You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2019/11/08 14:16:11 UTC

[hive] branch master updated: HIVE-22238: PK/FK selectivity estimation underscales estimations (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)

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

kgyrtkirk 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 3b4c2fe  HIVE-22238: PK/FK selectivity estimation underscales estimations (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
3b4c2fe is described below

commit 3b4c2fe0b34fbae89bffc3d766f61529c3b162c8
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Fri Nov 8 14:15:37 2019 +0000

    HIVE-22238: PK/FK selectivity estimation underscales estimations (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 data/files/starships.sql                           |  36 ++
 .../test/resources/testconfiguration.properties    |   3 +
 .../apache/hadoop/hive/ql/exec/OperatorUtils.java  |  21 +
 .../ql/hooks/AccurateEstimatesCheckerHook.java     | 146 +++++++
 .../stats/annotation/StatsRulesProcFactory.java    |  27 +-
 .../apache/hadoop/hive/ql/plan/ColStatistics.java  |  16 +
 .../apache/hadoop/hive/ql/stats/StatsUtils.java    |   1 +
 .../clientpositive/estimate_pkfk_filtered_fk.q     |  39 ++
 .../queries/clientpositive/estimate_pkfk_nocond.q  |  20 +
 .../queries/clientpositive/estimate_pkfk_push.q    |  21 +
 .../queries/clientpositive/stat_estimate_drill.q   |   7 +
 .../clientpositive/annotate_stats_join_pkfk.q.out  |   4 +-
 .../test/results/clientpositive/auto_join33.q.out  |   4 +-
 .../clientpositive/constprog_partitioner.q.out     |   6 +-
 .../clientpositive/correlationoptimizer9.q.out     |   6 +-
 .../clientpositive/estimate_pkfk_filtered_fk.q.out | 285 +++++++++++++
 .../clientpositive/estimate_pkfk_nocond.q.out      | 309 ++++++++++++++
 .../clientpositive/estimate_pkfk_push.q.out        | 281 +++++++++++++
 ql/src/test/results/clientpositive/join45.q.out    |   6 +-
 ql/src/test/results/clientpositive/join47.q.out    |   6 +-
 .../llap/estimate_pkfk_filtered_fk.q.out           | 312 +++++++++++++++
 .../clientpositive/llap/estimate_pkfk_nocond.q.out | 222 +++++++++++
 .../clientpositive/llap/estimate_pkfk_push.q.out   | 222 +++++++++++
 .../clientpositive/llap/explainuser_1.q.out        |  20 +-
 .../clientpositive/llap/explainuser_2.q.out        |  24 +-
 .../llap/groupby_groupingset_bug.q.out             |  61 ++-
 .../llap/hybridgrace_hashjoin_2.q.out              |  12 +-
 .../results/clientpositive/llap/reopt_dpp.q.out    |   4 +-
 .../clientpositive/llap/reopt_semijoin.q.out       |   4 +-
 .../llap/retry_failure_reorder.q.out               |   4 +-
 .../clientpositive/llap/subquery_exists.q.out      |   4 +-
 .../results/clientpositive/llap/subquery_in.q.out  |  20 +-
 .../clientpositive/llap/subquery_multi.q.out       |   6 +-
 .../clientpositive/llap/subquery_scalar.q.out      |   6 +-
 .../llap/vector_interval_mapjoin.q.out             |   6 +-
 .../llap/vector_mapjoin_reduce.q.out               |  18 +-
 ql/src/test/results/clientpositive/mapjoin47.q.out |   6 +-
 ql/src/test/results/clientpositive/masking_2.q.out |   4 +-
 .../clientpositive/masking_disablecbo_2.q.out      |   4 +-
 .../clientpositive/perf/tez/cbo_query23.q.out      |   4 +-
 .../perf/tez/constraints/cbo_query23.q.out         |   3 +-
 .../perf/tez/constraints/cbo_query54.q.out         |   4 +-
 .../perf/tez/constraints/query1.q.out              |  36 +-
 .../perf/tez/constraints/query10.q.out             | 266 ++++++-------
 .../perf/tez/constraints/query11.q.out             |  44 +-
 .../perf/tez/constraints/query12.q.out             |  12 +-
 .../perf/tez/constraints/query13.q.out             |  14 +-
 .../perf/tez/constraints/query14.q.out             | 194 ++++-----
 .../perf/tez/constraints/query15.q.out             |  16 +-
 .../perf/tez/constraints/query16.q.out             | 109 +++--
 .../perf/tez/constraints/query17.q.out             |  26 +-
 .../perf/tez/constraints/query18.q.out             |  18 +-
 .../perf/tez/constraints/query19.q.out             |  20 +-
 .../perf/tez/constraints/query1b.q.out             |  56 +--
 .../perf/tez/constraints/query2.q.out              |   6 +-
 .../perf/tez/constraints/query20.q.out             |  12 +-
 .../perf/tez/constraints/query23.q.out             | 442 +++++++++------------
 .../perf/tez/constraints/query25.q.out             |  22 +-
 .../perf/tez/constraints/query26.q.out             |   8 +-
 .../perf/tez/constraints/query27.q.out             |  18 +-
 .../perf/tez/constraints/query29.q.out             |  22 +-
 .../perf/tez/constraints/query3.q.out              |   8 +-
 .../perf/tez/constraints/query30.q.out             |  42 +-
 .../perf/tez/constraints/query31.q.out             |  66 +--
 .../perf/tez/constraints/query32.q.out             |  20 +-
 .../perf/tez/constraints/query33.q.out             |  48 +--
 .../perf/tez/constraints/query34.q.out             |  14 +-
 .../perf/tez/constraints/query35.q.out             | 206 +++++-----
 .../perf/tez/constraints/query36.q.out             |  10 +-
 .../perf/tez/constraints/query38.q.out             |  46 +--
 .../perf/tez/constraints/query4.q.out              |  58 +--
 .../perf/tez/constraints/query40.q.out             |  12 +-
 .../perf/tez/constraints/query42.q.out             |   6 +-
 .../perf/tez/constraints/query43.q.out             |  10 +-
 .../perf/tez/constraints/query45.q.out             |  20 +-
 .../perf/tez/constraints/query46.q.out             |  22 +-
 .../perf/tez/constraints/query47.q.out             |   8 +-
 .../perf/tez/constraints/query48.q.out             |  10 +-
 .../perf/tez/constraints/query49.q.out             |  94 ++---
 .../perf/tez/constraints/query5.q.out              |  18 +-
 .../perf/tez/constraints/query50.q.out             |  73 ++--
 .../perf/tez/constraints/query51.q.out             |  32 +-
 .../perf/tez/constraints/query52.q.out             |   4 +-
 .../perf/tez/constraints/query53.q.out             |  20 +-
 .../perf/tez/constraints/query54.q.out             | 287 +++++++------
 .../perf/tez/constraints/query55.q.out             |   4 +-
 .../perf/tez/constraints/query56.q.out             |  30 +-
 .../perf/tez/constraints/query57.q.out             |   8 +-
 .../perf/tez/constraints/query58.q.out             |  32 +-
 .../perf/tez/constraints/query6.q.out              |   8 +-
 .../perf/tez/constraints/query60.q.out             |  30 +-
 .../perf/tez/constraints/query61.q.out             |  18 +-
 .../perf/tez/constraints/query63.q.out             |  20 +-
 .../perf/tez/constraints/query64.q.out             |  84 ++--
 .../perf/tez/constraints/query65.q.out             |  32 +-
 .../perf/tez/constraints/query66.q.out             |  22 +-
 .../perf/tez/constraints/query67.q.out             |  22 +-
 .../perf/tez/constraints/query68.q.out             |  22 +-
 .../perf/tez/constraints/query69.q.out             | 196 ++++-----
 .../perf/tez/constraints/query7.q.out              |   8 +-
 .../perf/tez/constraints/query70.q.out             |  28 +-
 .../perf/tez/constraints/query71.q.out             |  24 +-
 .../perf/tez/constraints/query72.q.out             |  30 +-
 .../perf/tez/constraints/query73.q.out             |  14 +-
 .../perf/tez/constraints/query74.q.out             |  44 +-
 .../perf/tez/constraints/query75.q.out             |  72 ++--
 .../perf/tez/constraints/query77.q.out             |  70 ++--
 .../perf/tez/constraints/query78.q.out             |  36 +-
 .../perf/tez/constraints/query79.q.out             |  18 +-
 .../perf/tez/constraints/query8.q.out              |  18 +-
 .../perf/tez/constraints/query80.q.out             |  36 +-
 .../perf/tez/constraints/query81.q.out             |   6 +-
 .../perf/tez/constraints/query83.q.out             |  24 +-
 .../perf/tez/constraints/query84.q.out             |   6 +-
 .../perf/tez/constraints/query85.q.out             |  18 +-
 .../perf/tez/constraints/query86.q.out             |   8 +-
 .../perf/tez/constraints/query87.q.out             |  76 ++--
 .../perf/tez/constraints/query88.q.out             |  48 +--
 .../perf/tez/constraints/query89.q.out             |  20 +-
 .../perf/tez/constraints/query90.q.out             |  10 +-
 .../perf/tez/constraints/query91.q.out             |   8 +-
 .../perf/tez/constraints/query92.q.out             |  20 +-
 .../perf/tez/constraints/query93.q.out             |  57 +--
 .../perf/tez/constraints/query94.q.out             | 109 +++--
 .../perf/tez/constraints/query95.q.out             | 224 ++++-------
 .../perf/tez/constraints/query96.q.out             |   6 +-
 .../perf/tez/constraints/query97.q.out             |  16 +-
 .../perf/tez/constraints/query98.q.out             |  16 +-
 .../perf/tez/constraints/query99.q.out             |  18 +-
 .../results/clientpositive/perf/tez/query1.q.out   |  36 +-
 .../results/clientpositive/perf/tez/query10.q.out  | 266 ++++++-------
 .../results/clientpositive/perf/tez/query11.q.out  |  42 +-
 .../results/clientpositive/perf/tez/query12.q.out  |  12 +-
 .../results/clientpositive/perf/tez/query13.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query14.q.out  | 194 ++++-----
 .../results/clientpositive/perf/tez/query15.q.out  |  16 +-
 .../results/clientpositive/perf/tez/query16.q.out  | 111 +++---
 .../results/clientpositive/perf/tez/query17.q.out  |  26 +-
 .../results/clientpositive/perf/tez/query18.q.out  |  18 +-
 .../results/clientpositive/perf/tez/query19.q.out  |  20 +-
 .../results/clientpositive/perf/tez/query1b.q.out  | 136 +++----
 .../results/clientpositive/perf/tez/query2.q.out   |   6 +-
 .../results/clientpositive/perf/tez/query20.q.out  |  12 +-
 .../results/clientpositive/perf/tez/query23.q.out  | 373 ++++++++---------
 .../results/clientpositive/perf/tez/query24.q.out  |   2 +-
 .../results/clientpositive/perf/tez/query25.q.out  |  22 +-
 .../results/clientpositive/perf/tez/query26.q.out  |   8 +-
 .../results/clientpositive/perf/tez/query27.q.out  |  18 +-
 .../results/clientpositive/perf/tez/query29.q.out  |  24 +-
 .../results/clientpositive/perf/tez/query3.q.out   |   8 +-
 .../results/clientpositive/perf/tez/query30.q.out  |  42 +-
 .../results/clientpositive/perf/tez/query31.q.out  |  66 +--
 .../results/clientpositive/perf/tez/query32.q.out  |  20 +-
 .../results/clientpositive/perf/tez/query33.q.out  |  48 +--
 .../results/clientpositive/perf/tez/query34.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query35.q.out  |  38 +-
 .../results/clientpositive/perf/tez/query36.q.out  |  10 +-
 .../results/clientpositive/perf/tez/query38.q.out  |  46 +--
 .../results/clientpositive/perf/tez/query4.q.out   |  56 +--
 .../results/clientpositive/perf/tez/query40.q.out  |  44 +-
 .../results/clientpositive/perf/tez/query42.q.out  |   6 +-
 .../results/clientpositive/perf/tez/query43.q.out  |  10 +-
 .../results/clientpositive/perf/tez/query45.q.out  |  22 +-
 .../results/clientpositive/perf/tez/query46.q.out  |  22 +-
 .../results/clientpositive/perf/tez/query47.q.out  |   8 +-
 .../results/clientpositive/perf/tez/query48.q.out  |  10 +-
 .../results/clientpositive/perf/tez/query49.q.out  |  88 ++--
 .../results/clientpositive/perf/tez/query5.q.out   |  22 +-
 .../results/clientpositive/perf/tez/query50.q.out  |  89 ++---
 .../results/clientpositive/perf/tez/query51.q.out  |  32 +-
 .../results/clientpositive/perf/tez/query52.q.out  |   4 +-
 .../results/clientpositive/perf/tez/query53.q.out  |  22 +-
 .../results/clientpositive/perf/tez/query54.q.out  |  54 +--
 .../results/clientpositive/perf/tez/query55.q.out  |   4 +-
 .../results/clientpositive/perf/tez/query56.q.out  |  30 +-
 .../results/clientpositive/perf/tez/query57.q.out  |   8 +-
 .../results/clientpositive/perf/tez/query58.q.out  |  30 +-
 .../results/clientpositive/perf/tez/query6.q.out   |  22 +-
 .../results/clientpositive/perf/tez/query60.q.out  |  30 +-
 .../results/clientpositive/perf/tez/query61.q.out  |  18 +-
 .../results/clientpositive/perf/tez/query63.q.out  |  22 +-
 .../results/clientpositive/perf/tez/query64.q.out  |  76 ++--
 .../results/clientpositive/perf/tez/query65.q.out  |  36 +-
 .../results/clientpositive/perf/tez/query66.q.out  |  22 +-
 .../results/clientpositive/perf/tez/query67.q.out  |  22 +-
 .../results/clientpositive/perf/tez/query68.q.out  |  22 +-
 .../results/clientpositive/perf/tez/query69.q.out  | 196 ++++-----
 .../results/clientpositive/perf/tez/query7.q.out   |   8 +-
 .../results/clientpositive/perf/tez/query70.q.out  |  16 +-
 .../results/clientpositive/perf/tez/query71.q.out  |  24 +-
 .../results/clientpositive/perf/tez/query72.q.out  |  34 +-
 .../results/clientpositive/perf/tez/query73.q.out  |  14 +-
 .../results/clientpositive/perf/tez/query74.q.out  |  42 +-
 .../results/clientpositive/perf/tez/query75.q.out  |  72 ++--
 .../results/clientpositive/perf/tez/query77.q.out  |  70 ++--
 .../results/clientpositive/perf/tez/query78.q.out  |  38 +-
 .../results/clientpositive/perf/tez/query79.q.out  |  18 +-
 .../results/clientpositive/perf/tez/query8.q.out   |  18 +-
 .../results/clientpositive/perf/tez/query80.q.out  |  36 +-
 .../results/clientpositive/perf/tez/query81.q.out  |   6 +-
 .../results/clientpositive/perf/tez/query83.q.out  |  24 +-
 .../results/clientpositive/perf/tez/query84.q.out  |   8 +-
 .../results/clientpositive/perf/tez/query85.q.out  |  20 +-
 .../results/clientpositive/perf/tez/query86.q.out  |   8 +-
 .../results/clientpositive/perf/tez/query87.q.out  |  76 ++--
 .../results/clientpositive/perf/tez/query88.q.out  |  48 +--
 .../results/clientpositive/perf/tez/query89.q.out  |  20 +-
 .../results/clientpositive/perf/tez/query90.q.out  |  10 +-
 .../results/clientpositive/perf/tez/query91.q.out  |  12 +-
 .../results/clientpositive/perf/tez/query92.q.out  |  20 +-
 .../results/clientpositive/perf/tez/query93.q.out  |  59 ++-
 .../results/clientpositive/perf/tez/query94.q.out  | 111 +++---
 .../results/clientpositive/perf/tez/query95.q.out  | 236 +++++------
 .../results/clientpositive/perf/tez/query96.q.out  |   6 +-
 .../results/clientpositive/perf/tez/query97.q.out  |  16 +-
 .../results/clientpositive/perf/tez/query98.q.out  |  16 +-
 .../results/clientpositive/perf/tez/query99.q.out  |  18 +-
 .../clientpositive/spark/spark_explainuser_1.q.out |  20 +-
 .../results/clientpositive/subquery_exists.q.out   |   4 +-
 .../clientpositive/subquery_unqualcolumnrefs.q.out |   8 +-
 .../tez/hybridgrace_hashjoin_2.q.out               |  12 +-
 .../clientpositive/vector_interval_mapjoin.q.out   |   6 +-
 222 files changed, 5854 insertions(+), 4280 deletions(-)

diff --git a/data/files/starships.sql b/data/files/starships.sql
new file mode 100644
index 0000000..de56eaa
--- /dev/null
+++ b/data/files/starships.sql
@@ -0,0 +1,36 @@
+
+-- 10 torpedos carried by every ship
+-- 10 types of ships
+-- 10 admiral has 1 from each ship_type
+
+drop table if exists admirals;
+drop table if exists ship_types;
+drop table if exists ships;
+drop table if exists torpedos;
+
+create table ships (id integer,ship_type_id integer,crew_size integer);
+create table ship_types (id integer,type_name string);
+insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+    ;
+
+create table admirals as
+    select id from ship_types;
+
+create table torpedos (id integer,ship_id integer,admiral_id integer);
+
+insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2;
+
+insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2;
+
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 50dcf40..2528c28 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -531,6 +531,9 @@ minillaplocal.query.files=\
   enforce_constraint_notnull.q,\
   escape1.q,\
   escape2.q,\
+  estimate_pkfk_nocond.q,\
+  estimate_pkfk_filtered_fk.q,\
+  estimate_pkfk_push.q,\
   exchgpartition2lel.q,\
   explainanalyze_2.q,\
   explainuser_1.q,\
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
index f966def..2f114b8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
@@ -29,6 +29,8 @@ import java.util.Set;
 import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.exec.NodeUtils.Function;
+import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature;
+import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignatureFactory;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.SemiJoinBranchInfo;
 import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
@@ -611,4 +613,23 @@ public class OperatorUtils {
     }
     return null;
   }
+
+  /**
+   *  Determines if the two trees are using independent inputs.
+   */
+  public static boolean treesWithIndependentInputs(Operator<?> tree1, Operator<?> tree2) {
+    Set<String> tables1 = signaturesOf(OperatorUtils.findOperatorsUpstream(tree1, TableScanOperator.class));
+    Set<String> tables2 = signaturesOf(OperatorUtils.findOperatorsUpstream(tree2, TableScanOperator.class));
+
+    tables1.retainAll(tables2);
+    return tables1.isEmpty();
+  }
+
+  private static Set<String> signaturesOf(Set<TableScanOperator> ops) {
+    Set<String> ret = new HashSet<>();
+    for (TableScanOperator o : ops) {
+      ret.add(o.getConf().getQualifiedTable());
+    }
+    return ret;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/AccurateEstimatesCheckerHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/AccurateEstimatesCheckerHook.java
new file mode 100644
index 0000000..8299894
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/AccurateEstimatesCheckerHook.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.hooks;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ExplainTask;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
+import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
+import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook;
+import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.BaseWork;
+import org.apache.hadoop.hive.ql.plan.ExplainWork;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.plan.MapredWork;
+import org.apache.hadoop.hive.ql.plan.ReduceWork;
+import org.apache.hadoop.hive.ql.plan.Statistics;
+import org.apache.hadoop.hive.ql.plan.TezWork;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Adds an extra check for Explain Analyze queries.
+ */
+public class AccurateEstimatesCheckerHook extends AbstractSemanticAnalyzerHook {
+
+  private double absErr;
+  private double relErr;
+
+  class EstimateCheckerHook implements NodeProcessor {
+
+    Map<String, Operator<?>> opMap = new HashMap<>();
+
+    @Override
+    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
+        throws SemanticException {
+      Operator op = (Operator) nd;
+      Statistics stats = op.getStatistics();
+      if (stats != null && stats.getRunTimeNumRows() >= 0) {
+        try {
+          ensureEstimateAcceptable(stats.getNumRows(), stats.getRunTimeNumRows());
+        } catch (HiveException e) {
+          throw new SemanticException("On operator: " + op, e);
+        }
+      }
+      return null;
+    }
+
+  }
+
+  private void ensureEstimateAcceptable(long numRows, long runTimeNumRows) throws HiveException {
+    double currentDelta = Math.abs(runTimeNumRows - numRows);
+    if (currentDelta <= absErr) {
+      return;
+    }
+    if (runTimeNumRows > 0) {
+      double a = currentDelta / runTimeNumRows;
+      if (a <= relErr) {
+        return;
+      }
+    }
+    throw new HiveException("Estimation error is unacceptable " + numRows + " / " + runTimeNumRows + " with absErr:"
+        + absErr+ ", relErr:" + relErr);
+  }
+
+  @Override
+  public void postAnalyze(HiveSemanticAnalyzerHookContext context, List<Task<?>> rootTasks) throws SemanticException {
+
+    HiveSemanticAnalyzerHookContext hookContext = context;
+    HiveConf conf = (HiveConf) hookContext.getConf();
+    absErr = conf.getDouble("accurate.estimate.checker.absolute.error", 3.0);
+    relErr = conf.getDouble("accurate.estimate.checker.relative.error", .1);
+    List<Node> rootOps = Lists.newArrayList();
+
+    List<Task<?>> roots = rootTasks;
+    for (Task<?> task0 : roots) {
+
+      if (task0 instanceof ExplainTask) {
+        ExplainTask explainTask = (ExplainTask) task0;
+        ExplainWork w = explainTask.getWork();
+        List<Task<?>> explainRoots = w.getRootTasks();
+
+        for (Task<?> task : explainRoots) {
+
+          Object work = task.getWork();
+          if (work instanceof MapredWork) {
+            MapredWork mapredWork = (MapredWork) work;
+            MapWork mapWork = mapredWork.getMapWork();
+            if (mapWork != null) {
+              rootOps.addAll(mapWork.getAllRootOperators());
+            }
+            ReduceWork reduceWork = mapredWork.getReduceWork();
+            if (reduceWork != null) {
+              rootOps.addAll(reduceWork.getAllRootOperators());
+            }
+          }
+          if (work instanceof TezWork) {
+            for (BaseWork bw : ((TezWork) work).getAllWorkUnsorted()) {
+              rootOps.addAll(bw.getAllRootOperators());
+            }
+          }
+        }
+      }
+
+    }
+    if (rootOps.isEmpty()) {
+      return;
+    }
+
+    Dispatcher disp = new DefaultRuleDispatcher(new EstimateCheckerHook(), new HashMap<>(), null);
+    GraphWalker ogw = new DefaultGraphWalker(disp);
+
+    HashMap<Node, Object> nodeOutput = new HashMap<Node, Object>();
+    ogw.startWalking(rootOps, nodeOutput);
+
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
index c9284ad..5a7c1af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
@@ -238,7 +238,7 @@ public class StatsRulesProcFactory {
    * <li>T(S) - Number of tuples in relations S</li>
    * <li>V(S,A) - Number of distinct values of attribute A in relation S</li>
    * </ul>
-   * <i>Rules:</i> 
+   * <i>Rules:</i>
    * <ul>
    * <li><b>Column equals a constant</b> T(S) = T(R) / V(R,A)</li>
    * <li><b>Inequality conditions</b> T(S) = T(R) / 3</li>
@@ -879,11 +879,13 @@ public class StatsRulesProcFactory {
 
       if (pred instanceof ExprNodeColumnDesc) {
         ExprNodeColumnDesc encd = (ExprNodeColumnDesc) pred;
-        aspCtx.addAffectedColumn(encd);
         ColStatistics cs = stats.getColumnStatisticsFromColName(encd.getColumn());
         if (cs != null) {
           tmpNoNulls = cs.getNumNulls();
         }
+        if (cs == null || tmpNoNulls > 0) {
+          aspCtx.addAffectedColumn(encd);
+        }
       } else if (pred instanceof ExprNodeGenericFuncDesc || pred instanceof ExprNodeColumnListDesc) {
         long noNullsOfChild = 0;
         for (ExprNodeDesc childExpr : pred.getChildren()) {
@@ -1132,7 +1134,7 @@ public class StatsRulesProcFactory {
               if (aspCtx.isUniformWithinRange()) {
                 // Assuming uniform distribution, we can use the range to calculate
                 // new estimate for the number of rows
-                return Math.round(((double) (value - minValue) / (maxValue - minValue)) * numRows);
+                return Math.round(((value - minValue) / (maxValue - minValue)) * numRows);
               }
             } else {
               if (minValue > value || minValue == value && closedBound) {
@@ -1144,7 +1146,7 @@ public class StatsRulesProcFactory {
               if (aspCtx.isUniformWithinRange()) {
                 // Assuming uniform distribution, we can use the range to calculate
                 // new estimate for the number of rows
-                return Math.round(((double) (maxValue - value) / (maxValue - minValue)) * numRows);
+                return Math.round(((maxValue - value) / (maxValue - minValue)) * numRows);
               }
             }
           } else if (colTypeLowerCase.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
@@ -2239,6 +2241,7 @@ public class StatsRulesProcFactory {
         CommonJoinOperator<? extends JoinDesc> jop) {
       double pkfkSelectivity = Double.MAX_VALUE;
       int fkInd = -1;
+      boolean isFKIndependentFromPK = false;
       // 1. We iterate through all the operators that have candidate FKs and
       // choose the FK that has the minimum selectivity. We assume that PK and this FK
       // have the PK-FK relationship. This is heuristic and can be
@@ -2246,13 +2249,19 @@ public class StatsRulesProcFactory {
       for (Entry<Integer, ColStatistics> entry : csFKs.entrySet()) {
         int pos = entry.getKey();
         Operator<? extends OperatorDesc> opWithPK = ops.get(pkPos);
+        Operator<? extends OperatorDesc> opWithFK = jop.getParentOperators().get(pos);
         double selectivity = getSelectivitySimpleTree(opWithPK);
         double selectivityAdjustment = StatsUtils.getScaledSelectivity(csPK, entry.getValue());
         selectivity = selectivityAdjustment * selectivity > 1 ? selectivity : selectivityAdjustment
             * selectivity;
-        if (selectivity < pkfkSelectivity) {
+
+        boolean independent =
+            !entry.getValue().isFilteredColumn() && OperatorUtils.treesWithIndependentInputs(opWithFK, opWithPK);
+
+        if (fkInd < 0 || (independent && selectivity < pkfkSelectivity)) {
           pkfkSelectivity = selectivity;
           fkInd = pos;
+          isFKIndependentFromPK = independent;
         }
       }
       long newrows = 1;
@@ -2271,8 +2280,14 @@ public class StatsRulesProcFactory {
         Statistics parentStats = parent.getStatistics();
         if (fkInd == pos) {
           // 2.1 This is the new number of rows after PK is joining with FK
-          newrows = (long) Math.ceil(parentStats.getNumRows() * pkfkSelectivity);
+          if (!isFKIndependentFromPK) {
+            // if the foreign key is filtered by some condition we may not re-scale it
+            newrows = parentStats.getNumRows();
+          } else {
+            newrows = (long) Math.ceil(parentStats.getNumRows() * pkfkSelectivity);
+          }
           rowCounts.add(newrows);
+
           // 2.1 The ndv is the minimum of the PK and the FK.
           distinctVals.add(Math.min(csFK.getCountDistint(), csPK.getCountDistint()));
         } else {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ColStatistics.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ColStatistics.java
index a31f965..7e7b9a4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ColStatistics.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ColStatistics.java
@@ -30,6 +30,7 @@ public class ColStatistics {
   private Range range;
   private boolean isPrimaryKey;
   private boolean isEstimated;
+  private boolean isFilteredColumn;
 
   public ColStatistics(String colName, String colType) {
     this.setColumnName(colName);
@@ -149,6 +150,7 @@ public class ColStatistics {
     clone.setNumFalses(numFalses);
     clone.setPrimaryKey(isPrimaryKey);
     clone.setIsEstimated(isEstimated);
+    clone.setIsFilteredColumn(isFilteredColumn);
     if (range != null ) {
       clone.setRange(range.clone());
     }
@@ -196,4 +198,18 @@ public class ColStatistics {
       return sb.toString();
     }
   }
+
+  public void setFilterColumn() {
+    isFilteredColumn = true;
+  }
+
+  private void setIsFilteredColumn(boolean isFilteredColumn2) {
+    isFilteredColumn=isFilteredColumn2;
+    
+  }
+  
+  public boolean isFilteredColumn() {
+    return isFilteredColumn;
+  }
+  
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index 0e12f27..36c5cb7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -1975,6 +1975,7 @@ public class StatsUtils {
             newDV = (long) Math.ceil(ratio * oldDV);
           }
           cs.setCountDistint(newDV);
+          cs.setFilterColumn();
           oldDV = newDV;
         }
         if (oldDV > newNumRows) {
diff --git a/ql/src/test/queries/clientpositive/estimate_pkfk_filtered_fk.q b/ql/src/test/queries/clientpositive/estimate_pkfk_filtered_fk.q
new file mode 100644
index 0000000..70b7e79
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/estimate_pkfk_filtered_fk.q
@@ -0,0 +1,39 @@
+source ${system:hive.root}/data/files/starships.sql;
+
+-- conditions on both side of the join ; but not on the FK ; PK/FK scale should be used
+
+set hive.explain.user=true;
+
+--for this case this can't be enabled because min is used during join selectivity estimation - so the 2. join estimation wont be that good
+--set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.hooks.AccurateEstimatesCheckerHook;
+
+
+explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+;
+
+
+-- but this simpler case can be asserted
+set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.hooks.AccurateEstimatesCheckerHook;
+
+explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+;
diff --git a/ql/src/test/queries/clientpositive/estimate_pkfk_nocond.q b/ql/src/test/queries/clientpositive/estimate_pkfk_nocond.q
new file mode 100644
index 0000000..61a4a06
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/estimate_pkfk_nocond.q
@@ -0,0 +1,20 @@
+source ${system:hive.root}/data/files/starships.sql;
+
+-- conditions on one side of the join ; PK/FK scale should be used
+
+set hive.explain.user=true;
+set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.hooks.AccurateEstimatesCheckerHook;
+
+
+explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+;
diff --git a/ql/src/test/queries/clientpositive/estimate_pkfk_push.q b/ql/src/test/queries/clientpositive/estimate_pkfk_push.q
new file mode 100644
index 0000000..270c754
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/estimate_pkfk_push.q
@@ -0,0 +1,21 @@
+source ${system:hive.root}/data/files/starships.sql;
+
+-- conditions on the join key from one side ; is pushed to the other side to reduce computation
+-- however: we should not use the ratio twice (and loose accuracy)
+-- this was the primary issue of HIVE-22238
+
+set hive.explain.user=true;
+set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.hooks.AccurateEstimatesCheckerHook;
+
+explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+;
diff --git a/ql/src/test/queries/clientpositive/stat_estimate_drill.q b/ql/src/test/queries/clientpositive/stat_estimate_drill.q
index 2b711d6..7ba5fe9 100644
--- a/ql/src/test/queries/clientpositive/stat_estimate_drill.q
+++ b/ql/src/test/queries/clientpositive/stat_estimate_drill.q
@@ -1,3 +1,6 @@
+
+set hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.hooks.AccurateEstimatesCheckerHook;
+
 set hive.explain.user=true;
 set hive.strict.checks.cartesian.product=false;
 set hive.stats.fetch.column.stats=true;
@@ -25,4 +28,8 @@ explain analyze select sum(a) from t3 where a=1 group by b;
 explain analyze select sum(a) from t3 where a=1 and b=2 group by b;
 explain analyze select sum(a) from t3 where a=1 and b=2 and c=3 group by b;
 
+
+-- FIXME: for the below case; we right now get an IN which is estimated as 1/3 - might worth consider improving
+set hive.semantic.analyzer.hook=;
+
 explain analyze select sum(a) from t3 where (a=1 and b=2) or (a=2 and b=3) or (a=3 and b=4) group by b;
diff --git a/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out b/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
index 18f2c6a..8a1ebcd 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_join_pkfk.q.out
@@ -635,10 +635,10 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0
-          Statistics: Num rows: 90 Data size: 360 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 876 Data size: 3504 Basic stats: COMPLETE Column stats: PARTIAL
           File Output Operator
             compressed: false
-            Statistics: Num rows: 90 Data size: 360 Basic stats: COMPLETE Column stats: PARTIAL
+            Statistics: Num rows: 876 Data size: 3504 Basic stats: COMPLETE Column stats: PARTIAL
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/auto_join33.q.out b/ql/src/test/results/clientpositive/auto_join33.q.out
index 8af7e79..fb6ddab 100644
--- a/ql/src/test/results/clientpositive/auto_join33.q.out
+++ b/ql/src/test/results/clientpositive/auto_join33.q.out
@@ -67,10 +67,10 @@ STAGE PLANS:
                     0 (UDFToDouble(_col0) + 1.0D) (type: double)
                     1 (UDFToDouble(_col0) + 2.0D) (type: double)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/constprog_partitioner.q.out b/ql/src/test/results/clientpositive/constprog_partitioner.q.out
index e42287c..958f75d 100644
--- a/ql/src/test/results/clientpositive/constprog_partitioner.q.out
+++ b/ql/src/test/results/clientpositive/constprog_partitioner.q.out
@@ -144,14 +144,14 @@ STAGE PLANS:
             0 _col0 (type: int), 1 (type: int)
             1 _col0 (type: int), _col1 (type: int)
           outputColumnNames: _col1, _col2
-          Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col1 (type: int), _col2 (type: int)
             outputColumnNames: _col0, _col1
-            Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer9.q.out b/ql/src/test/results/clientpositive/correlationoptimizer9.q.out
index 73d3450..fa8819c 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer9.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer9.q.out
@@ -443,14 +443,14 @@ STAGE PLANS:
             0 _col0 (type: int), _col1 (type: string)
             1 _col0 (type: int), _col1 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 163 Data size: 33578 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 99 Data size: 20394 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: string), _col2 (type: bigint), _col5 (type: bigint)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 163 Data size: 33578 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 99 Data size: 20394 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 163 Data size: 33578 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 99 Data size: 20394 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/estimate_pkfk_filtered_fk.q.out b/ql/src/test/results/clientpositive/estimate_pkfk_filtered_fk.q.out
new file mode 100644
index 0000000..b0ecaa2
--- /dev/null
+++ b/ql/src/test/results/clientpositive/estimate_pkfk_filtered_fk.q.out
@@ -0,0 +1,285 @@
+PREHOOK: query: drop table if exists admirals
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists admirals
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ship_types
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ship_types
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ships
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ships
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists torpedos
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists torpedos
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ships
+POSTHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ships
+PREHOOK: query: create table ship_types (id integer,type_name string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: create table ship_types (id integer,type_name string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ship_types
+PREHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ship_types
+POSTHOOK: Lineage: ship_types.id SCRIPT []
+POSTHOOK: Lineage: ship_types.type_name SCRIPT []
+PREHOOK: query: create table admirals as
+    select id from ship_types
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: database:default
+PREHOOK: Output: default@admirals
+POSTHOOK: query: create table admirals as
+    select id from ship_types
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@admirals
+POSTHOOK: Lineage: admirals.id SIMPLE [(ship_types)ship_types.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@torpedos
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: default@ships
+POSTHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: default@ships
+POSTHOOK: Lineage: ships.crew_size SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.id SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.ship_type_id SIMPLE [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Output: default@torpedos
+POSTHOOK: Lineage: torpedos.admiral_id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.ship_id SIMPLE [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+PREHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: s
+            filterExpr: ((crew_size = 2) and ship_type_id is not null and id is not null) (type: boolean)
+            Statistics: Num rows: 100/100 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ((crew_size = 2) and ship_type_id is not null and id is not null) (type: boolean)
+              Statistics: Num rows: 10/10 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: id (type: int), ship_type_id (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 10/10 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 10/10 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int)
+          TableScan
+            alias: st
+            filterExpr: ((type_name = 'galaxy class') and id is not null) (type: boolean)
+            Statistics: Num rows: 10/10 Data size: 990 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ((type_name = 'galaxy class') and id is not null) (type: boolean)
+              Statistics: Num rows: 1/1 Data size: 99 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: id (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1/1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1/1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col1 (type: int)
+            1 _col0 (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 2/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: int)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 2/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          TableScan
+            alias: t
+            filterExpr: ship_id is not null (type: boolean)
+            Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ship_id is not null (type: boolean)
+              Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: ship_id (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: int)
+            1 _col0 (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 101/10 Data size: 404 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 101/10 Data size: 404 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/estimate_pkfk_nocond.q.out b/ql/src/test/results/clientpositive/estimate_pkfk_nocond.q.out
new file mode 100644
index 0000000..6e3a30f
--- /dev/null
+++ b/ql/src/test/results/clientpositive/estimate_pkfk_nocond.q.out
@@ -0,0 +1,309 @@
+PREHOOK: query: drop table if exists admirals
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists admirals
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ship_types
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ship_types
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ships
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ships
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists torpedos
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists torpedos
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ships
+POSTHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ships
+PREHOOK: query: create table ship_types (id integer,type_name string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: create table ship_types (id integer,type_name string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ship_types
+PREHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ship_types
+POSTHOOK: Lineage: ship_types.id SCRIPT []
+POSTHOOK: Lineage: ship_types.type_name SCRIPT []
+PREHOOK: query: create table admirals as
+    select id from ship_types
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: database:default
+PREHOOK: Output: default@admirals
+POSTHOOK: query: create table admirals as
+    select id from ship_types
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@admirals
+POSTHOOK: Lineage: admirals.id SIMPLE [(ship_types)ship_types.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@torpedos
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: default@ships
+POSTHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: default@ships
+POSTHOOK: Lineage: ships.crew_size SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.id SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.ship_type_id SIMPLE [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Output: default@torpedos
+POSTHOOK: Lineage: torpedos.admiral_id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.ship_id SIMPLE [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: explain 
+select 1
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: explain 
+select 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: _dummy_table
+          Row Limit Per Split: 1
+          Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: 1 (type: int)
+            outputColumnNames: _col0
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+            ListSink
+
+PREHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+PREHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: s
+            filterExpr: (ship_type_id is not null and id is not null) (type: boolean)
+            Statistics: Num rows: 100/100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: (ship_type_id is not null and id is not null) (type: boolean)
+              Statistics: Num rows: 100/100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: id (type: int), ship_type_id (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 100/100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 100/100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int)
+          TableScan
+            alias: st
+            filterExpr: ((type_name = 'galaxy class') and id is not null) (type: boolean)
+            Statistics: Num rows: 10/10 Data size: 990 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ((type_name = 'galaxy class') and id is not null) (type: boolean)
+              Statistics: Num rows: 1/1 Data size: 99 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: id (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1/1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1/1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col1 (type: int)
+            1 _col0 (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 11/10 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: int)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 11/10 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+          TableScan
+            alias: t
+            filterExpr: ship_id is not null (type: boolean)
+            Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ship_id is not null (type: boolean)
+              Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: ship_id (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: int)
+            1 _col0 (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 101/100 Data size: 404 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 101/100 Data size: 404 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/estimate_pkfk_push.q.out b/ql/src/test/results/clientpositive/estimate_pkfk_push.q.out
new file mode 100644
index 0000000..07c62f2
--- /dev/null
+++ b/ql/src/test/results/clientpositive/estimate_pkfk_push.q.out
@@ -0,0 +1,281 @@
+PREHOOK: query: drop table if exists admirals
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists admirals
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ship_types
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ship_types
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ships
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ships
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists torpedos
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists torpedos
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ships
+POSTHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ships
+PREHOOK: query: create table ship_types (id integer,type_name string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: create table ship_types (id integer,type_name string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ship_types
+PREHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ship_types
+POSTHOOK: Lineage: ship_types.id SCRIPT []
+POSTHOOK: Lineage: ship_types.type_name SCRIPT []
+PREHOOK: query: create table admirals as
+    select id from ship_types
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: database:default
+PREHOOK: Output: default@admirals
+POSTHOOK: query: create table admirals as
+    select id from ship_types
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@admirals
+POSTHOOK: Lineage: admirals.id SIMPLE [(ship_types)ship_types.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@torpedos
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: default@ships
+POSTHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: default@ships
+POSTHOOK: Lineage: ships.crew_size SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.id SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.ship_type_id SIMPLE [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Output: default@torpedos
+POSTHOOK: Lineage: torpedos.admiral_id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.ship_id SIMPLE [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+PREHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: s
+            filterExpr: ((ship_type_id) IN (1, 2) and id is not null) (type: boolean)
+            Statistics: Num rows: 100/100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ((ship_type_id) IN (1, 2) and id is not null) (type: boolean)
+              Statistics: Num rows: 20/20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: id (type: int), ship_type_id (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 20/20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 20/20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int)
+          TableScan
+            alias: st
+            filterExpr: (id) IN (1, 2) (type: boolean)
+            Statistics: Num rows: 10/10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: (id) IN (1, 2) (type: boolean)
+              Statistics: Num rows: 2/2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: id (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 2/2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 2/2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col1 (type: int)
+            1 _col0 (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 5/20 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: int)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 5/20 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+          TableScan
+            alias: t
+            filterExpr: ship_id is not null (type: boolean)
+            Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: ship_id is not null (type: boolean)
+              Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: ship_id (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1000/1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: int)
+            1 _col0 (type: int)
+          outputColumnNames: _col0
+          Statistics: Num rows: 201/200 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 201/200 Data size: 804 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/join45.q.out b/ql/src/test/results/clientpositive/join45.q.out
index 1bade2b..312474e 100644
--- a/ql/src/test/results/clientpositive/join45.q.out
+++ b/ql/src/test/results/clientpositive/join45.q.out
@@ -70,13 +70,13 @@ STAGE PLANS:
             0 _col0 (type: string)
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 10
-            Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/join47.q.out b/ql/src/test/results/clientpositive/join47.q.out
index 02b4e7e..e94ad43 100644
--- a/ql/src/test/results/clientpositive/join47.q.out
+++ b/ql/src/test/results/clientpositive/join47.q.out
@@ -70,13 +70,13 @@ STAGE PLANS:
             0 _col0 (type: string)
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
             Number of rows: 10
-            Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/estimate_pkfk_filtered_fk.q.out b/ql/src/test/results/clientpositive/llap/estimate_pkfk_filtered_fk.q.out
new file mode 100644
index 0000000..a9facbf
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/estimate_pkfk_filtered_fk.q.out
@@ -0,0 +1,312 @@
+PREHOOK: query: drop table if exists admirals
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists admirals
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ship_types
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ship_types
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ships
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ships
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists torpedos
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists torpedos
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ships
+POSTHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ships
+PREHOOK: query: create table ship_types (id integer,type_name string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: create table ship_types (id integer,type_name string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ship_types
+PREHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ship_types
+POSTHOOK: Lineage: ship_types.id SCRIPT []
+POSTHOOK: Lineage: ship_types.type_name SCRIPT []
+PREHOOK: query: create table admirals as
+    select id from ship_types
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: database:default
+PREHOOK: Output: default@admirals
+POSTHOOK: query: create table admirals as
+    select id from ship_types
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@admirals
+POSTHOOK: Lineage: admirals.id SIMPLE [(ship_types)ship_types.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@torpedos
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: default@ships
+POSTHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: default@ships
+POSTHOOK: Lineage: ships.crew_size SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.id SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.ship_type_id SIMPLE [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Output: default@torpedos
+POSTHOOK: Lineage: torpedos.admiral_id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.ship_id SIMPLE [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+PREHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 3 llap
+      File Output Operator [FS_16]
+        Merge Join Operator [MERGEJOIN_46] (rows=101/10 width=4)
+          Conds:RS_12._col0=RS_55._col0(Inner),Output:["_col0"]
+        <-Map 5 [SIMPLE_EDGE] vectorized, llap
+          SHUFFLE [RS_55]
+            PartitionCols:_col0
+            Select Operator [SEL_54] (rows=1000/1000 width=4)
+              Output:["_col0"]
+              Filter Operator [FIL_53] (rows=1000/1000 width=4)
+                predicate:ship_id is not null
+                TableScan [TS_6] (rows=1000/1000 width=4)
+                  default@torpedos,t,Tbl:COMPLETE,Col:COMPLETE,Output:["ship_id"]
+        <-Reducer 2 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_12]
+            PartitionCols:_col0
+            Merge Join Operator [MERGEJOIN_45] (rows=2/1 width=4)
+              Conds:RS_49._col1=RS_52._col0(Inner),Output:["_col0"]
+            <-Map 1 [SIMPLE_EDGE] vectorized, llap
+              SHUFFLE [RS_49]
+                PartitionCols:_col1
+                Select Operator [SEL_48] (rows=10/10 width=8)
+                  Output:["_col0","_col1"]
+                  Filter Operator [FIL_47] (rows=10/10 width=12)
+                    predicate:((crew_size = 2) and ship_type_id is not null and id is not null)
+                    TableScan [TS_0] (rows=100/100 width=12)
+                      default@ships,s,Tbl:COMPLETE,Col:COMPLETE,Output:["id","ship_type_id","crew_size"]
+            <-Map 4 [SIMPLE_EDGE] vectorized, llap
+              SHUFFLE [RS_52]
+                PartitionCols:_col0
+                Select Operator [SEL_51] (rows=1/1 width=4)
+                  Output:["_col0"]
+                  Filter Operator [FIL_50] (rows=1/1 width=99)
+                    predicate:((type_name = 'galaxy class') and id is not null)
+                    TableScan [TS_3] (rows=10/10 width=99)
+                      default@ship_types,st,Tbl:COMPLETE,Col:COMPLETE,Output:["id","type_name"]
+
+PREHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+#### A masked pattern was here ####
+POSTHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+#### A masked pattern was here ####
+PREHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+#### A masked pattern was here ####
+POSTHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st
+where
+    st.type_name='galaxy class' 
+    and s.crew_size=2
+    and ship_type_id=st.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 2 llap
+      File Output Operator [FS_10]
+        Merge Join Operator [MERGEJOIN_25] (rows=2/1 width=4)
+          Conds:RS_28._col1=RS_31._col0(Inner),Output:["_col0"]
+        <-Map 1 [SIMPLE_EDGE] vectorized, llap
+          SHUFFLE [RS_28]
+            PartitionCols:_col1
+            Select Operator [SEL_27] (rows=10/10 width=8)
+              Output:["_col0","_col1"]
+              Filter Operator [FIL_26] (rows=10/10 width=12)
+                predicate:((crew_size = 2) and ship_type_id is not null)
+                TableScan [TS_0] (rows=100/100 width=12)
+                  default@ships,s,Tbl:COMPLETE,Col:COMPLETE,Output:["id","ship_type_id","crew_size"]
+        <-Map 3 [SIMPLE_EDGE] vectorized, llap
+          SHUFFLE [RS_31]
+            PartitionCols:_col0
+            Select Operator [SEL_30] (rows=1/1 width=4)
+              Output:["_col0"]
+              Filter Operator [FIL_29] (rows=1/1 width=99)
+                predicate:((type_name = 'galaxy class') and id is not null)
+                TableScan [TS_3] (rows=10/10 width=99)
+                  default@ship_types,st,Tbl:COMPLETE,Col:COMPLETE,Output:["id","type_name"]
+
diff --git a/ql/src/test/results/clientpositive/llap/estimate_pkfk_nocond.q.out b/ql/src/test/results/clientpositive/llap/estimate_pkfk_nocond.q.out
new file mode 100644
index 0000000..bdd8c08
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/estimate_pkfk_nocond.q.out
@@ -0,0 +1,222 @@
+PREHOOK: query: drop table if exists admirals
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists admirals
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ship_types
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ship_types
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ships
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ships
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists torpedos
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists torpedos
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ships
+POSTHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ships
+PREHOOK: query: create table ship_types (id integer,type_name string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: create table ship_types (id integer,type_name string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ship_types
+PREHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ship_types
+POSTHOOK: Lineage: ship_types.id SCRIPT []
+POSTHOOK: Lineage: ship_types.type_name SCRIPT []
+PREHOOK: query: create table admirals as
+    select id from ship_types
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: database:default
+PREHOOK: Output: default@admirals
+POSTHOOK: query: create table admirals as
+    select id from ship_types
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@admirals
+POSTHOOK: Lineage: admirals.id SIMPLE [(ship_types)ship_types.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@torpedos
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: default@ships
+POSTHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: default@ships
+POSTHOOK: Lineage: ships.crew_size SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.id SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.ship_type_id SIMPLE [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Output: default@torpedos
+POSTHOOK: Lineage: torpedos.admiral_id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.ship_id SIMPLE [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+PREHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    st.type_name='galaxy class' 
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 3 llap
+      File Output Operator [FS_16]
+        Merge Join Operator [MERGEJOIN_46] (rows=101/100 width=4)
+          Conds:RS_12._col0=RS_55._col0(Inner),Output:["_col0"]
+        <-Map 5 [SIMPLE_EDGE] vectorized, llap
+          SHUFFLE [RS_55]
+            PartitionCols:_col0
+            Select Operator [SEL_54] (rows=1000/1000 width=4)
+              Output:["_col0"]
+              Filter Operator [FIL_53] (rows=1000/1000 width=4)
+                predicate:ship_id is not null
+                TableScan [TS_6] (rows=1000/1000 width=4)
+                  default@torpedos,t,Tbl:COMPLETE,Col:COMPLETE,Output:["ship_id"]
+        <-Reducer 2 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_12]
+            PartitionCols:_col0
+            Merge Join Operator [MERGEJOIN_45] (rows=11/10 width=4)
+              Conds:RS_49._col1=RS_52._col0(Inner),Output:["_col0"]
+            <-Map 1 [SIMPLE_EDGE] vectorized, llap
+              SHUFFLE [RS_49]
+                PartitionCols:_col1
+                Select Operator [SEL_48] (rows=100/100 width=8)
+                  Output:["_col0","_col1"]
+                  Filter Operator [FIL_47] (rows=100/100 width=8)
+                    predicate:(ship_type_id is not null and id is not null)
+                    TableScan [TS_0] (rows=100/100 width=8)
+                      default@ships,s,Tbl:COMPLETE,Col:COMPLETE,Output:["id","ship_type_id"]
+            <-Map 4 [SIMPLE_EDGE] vectorized, llap
+              SHUFFLE [RS_52]
+                PartitionCols:_col0
+                Select Operator [SEL_51] (rows=1/1 width=4)
+                  Output:["_col0"]
+                  Filter Operator [FIL_50] (rows=1/1 width=99)
+                    predicate:((type_name = 'galaxy class') and id is not null)
+                    TableScan [TS_3] (rows=10/10 width=99)
+                      default@ship_types,st,Tbl:COMPLETE,Col:COMPLETE,Output:["id","type_name"]
+
diff --git a/ql/src/test/results/clientpositive/llap/estimate_pkfk_push.q.out b/ql/src/test/results/clientpositive/llap/estimate_pkfk_push.q.out
new file mode 100644
index 0000000..2ba22d6
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/estimate_pkfk_push.q.out
@@ -0,0 +1,222 @@
+PREHOOK: query: drop table if exists admirals
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists admirals
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ship_types
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ship_types
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists ships
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists ships
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists torpedos
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists torpedos
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ships
+POSTHOOK: query: create table ships (id integer,ship_type_id integer,crew_size integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ships
+PREHOOK: query: create table ship_types (id integer,type_name string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: create table ship_types (id integer,type_name string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ship_types
+PREHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ship_types
+POSTHOOK: query: insert into ship_types values
+    (1,'galaxy class'),
+    (2,'nebula class'),
+    (3,'orion class'),
+    (4,'first class'),
+    (5,'last pass'),
+    (6,'last pass'),
+    (7,'akira class'),
+    (8,'aeon type'),
+    (9,'antares type'),
+    (10,'apollo class')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ship_types
+POSTHOOK: Lineage: ship_types.id SCRIPT []
+POSTHOOK: Lineage: ship_types.type_name SCRIPT []
+PREHOOK: query: create table admirals as
+    select id from ship_types
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: database:default
+PREHOOK: Output: default@admirals
+POSTHOOK: query: create table admirals as
+    select id from ship_types
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@admirals
+POSTHOOK: Lineage: admirals.id SIMPLE [(ship_types)ship_types.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: create table torpedos (id integer,ship_id integer,admiral_id integer)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@torpedos
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Output: default@ships
+POSTHOOK: query: insert into ships
+select row_number() over (),t.id,row_number() over (partition by t.id) from ship_types t join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Output: default@ships
+POSTHOOK: Lineage: ships.crew_size SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.id SCRIPT [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: ships.ship_type_id SIMPLE [(ship_types)t.FieldSchema(name:id, type:int, comment:null), ]
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Output: default@torpedos
+POSTHOOK: query: insert into torpedos
+select row_number() over (),s.id,row_number() over (partition by s.id) from ships s join ship_types t2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Output: default@torpedos
+POSTHOOK: Lineage: torpedos.admiral_id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.id SCRIPT [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+POSTHOOK: Lineage: torpedos.ship_id SIMPLE [(ships)s.FieldSchema(name:id, type:int, comment:null), ]
+PREHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+PREHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ship_types
+PREHOOK: Input: default@ships
+PREHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+POSTHOOK: query: explain analyze
+select
+    s.id
+from
+    ships s,
+    ship_types st,
+    torpedos t
+where
+    (st.id = 1 or st.id=2)
+    and ship_type_id=st.id
+    and ship_id=s.id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ship_types
+POSTHOOK: Input: default@ships
+POSTHOOK: Input: default@torpedos
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 3 llap
+      File Output Operator [FS_16]
+        Merge Join Operator [MERGEJOIN_46] (rows=201/200 width=4)
+          Conds:RS_12._col0=RS_55._col0(Inner),Output:["_col0"]
+        <-Map 5 [SIMPLE_EDGE] vectorized, llap
+          SHUFFLE [RS_55]
+            PartitionCols:_col0
+            Select Operator [SEL_54] (rows=1000/1000 width=4)
+              Output:["_col0"]
+              Filter Operator [FIL_53] (rows=1000/1000 width=4)
+                predicate:ship_id is not null
+                TableScan [TS_6] (rows=1000/1000 width=4)
+                  default@torpedos,t,Tbl:COMPLETE,Col:COMPLETE,Output:["ship_id"]
+        <-Reducer 2 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_12]
+            PartitionCols:_col0
+            Merge Join Operator [MERGEJOIN_45] (rows=20/20 width=4)
+              Conds:RS_49._col1=RS_52._col0(Inner),Output:["_col0"]
+            <-Map 1 [SIMPLE_EDGE] vectorized, llap
+              SHUFFLE [RS_49]
+                PartitionCols:_col1
+                Select Operator [SEL_48] (rows=20/20 width=8)
+                  Output:["_col0","_col1"]
+                  Filter Operator [FIL_47] (rows=20/20 width=8)
+                    predicate:((ship_type_id) IN (1, 2) and id is not null)
+                    TableScan [TS_0] (rows=100/100 width=8)
+                      default@ships,s,Tbl:COMPLETE,Col:COMPLETE,Output:["id","ship_type_id"]
+            <-Map 4 [SIMPLE_EDGE] vectorized, llap
+              SHUFFLE [RS_52]
+                PartitionCols:_col0
+                Select Operator [SEL_51] (rows=2/2 width=4)
+                  Output:["_col0"]
+                  Filter Operator [FIL_50] (rows=2/2 width=4)
+                    predicate:(id) IN (1, 2)
+                    TableScan [TS_3] (rows=10/10 width=4)
+                      default@ship_types,st,Tbl:COMPLETE,Col:COMPLETE,Output:["id"]
+
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index 0dc5b51..319bba6 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -2234,7 +2234,7 @@ Stage-0
     Stage-1
       Reducer 2 llap
       File Output Operator [FS_12]
-        Merge Join Operator [MERGEJOIN_37] (rows=131 width=178)
+        Merge Join Operator [MERGEJOIN_37] (rows=83 width=178)
           Conds:RS_8._col0, _col1=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1"]
         <-Map 1 [SIMPLE_EDGE] llap
           SHUFFLE [RS_8]
@@ -2290,7 +2290,7 @@ Stage-0
     Stage-1
       Reducer 2 llap
       File Output Operator [FS_12]
-        Merge Join Operator [MERGEJOIN_37] (rows=131 width=178)
+        Merge Join Operator [MERGEJOIN_37] (rows=83 width=178)
           Conds:RS_8._col0, _col1=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1"]
         <-Map 1 [SIMPLE_EDGE] llap
           SHUFFLE [RS_8]
@@ -2385,9 +2385,9 @@ Stage-0
     Stage-1
       Reducer 3 llap
       File Output Operator [FS_21]
-        Select Operator [SEL_20] (rows=3 width=8)
+        Select Operator [SEL_20] (rows=1 width=8)
           Output:["_col0","_col1"]
-          Merge Join Operator [MERGEJOIN_51] (rows=3 width=8)
+          Merge Join Operator [MERGEJOIN_51] (rows=1 width=8)
             Conds:RS_17._col1, _col4=RS_18._col0, _col1(Left Semi),Output:["_col0","_col3"]
           <-Map 5 [SIMPLE_EDGE] llap
             SHUFFLE [RS_18]
@@ -2704,16 +2704,16 @@ Stage-0
     Stage-1
       Reducer 3 llap
       File Output Operator [FS_24]
-        Select Operator [SEL_23] (rows=33 width=223)
+        Select Operator [SEL_23] (rows=38 width=223)
           Output:["_col0","_col1","_col2"]
-          Filter Operator [FIL_22] (rows=33 width=227)
+          Filter Operator [FIL_22] (rows=38 width=228)
             predicate:((_col7 is null or (_col4 = 0L) or _col4 is null) and ((_col5 < _col4) is not true or (_col4 = 0L) or _col4 is null or _col7 is not null or _col0 is null) and (_col0 is not null or (_col4 = 0L) or _col4 is null or _col7 is not null))
-            Merge Join Operator [MERGEJOIN_45] (rows=33 width=227)
+            Merge Join Operator [MERGEJOIN_45] (rows=38 width=228)
               Conds:RS_19._col0, _col1=RS_20._col0, _col2(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5","_col7"]
             <-Reducer 2 [SIMPLE_EDGE] llap
               SHUFFLE [RS_19]
                 PartitionCols:_col0, _col1
-                Merge Join Operator [MERGEJOIN_44] (rows=31 width=226)
+                Merge Join Operator [MERGEJOIN_44] (rows=36 width=227)
                   Conds:RS_16._col1=RS_17._col0(Left Outer),Output:["_col0","_col1","_col2","_col4","_col5"]
                 <-Map 1 [SIMPLE_EDGE] llap
                   SHUFFLE [RS_16]
@@ -2725,12 +2725,12 @@ Stage-0
                 <-Reducer 4 [SIMPLE_EDGE] llap
                   SHUFFLE [RS_17]
                     PartitionCols:_col0
-                    Group By Operator [GBY_7] (rows=1 width=114)
+                    Group By Operator [GBY_7] (rows=2 width=114)
                       Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
                     <-Map 1 [SIMPLE_EDGE] llap
                       SHUFFLE [RS_6]
                         PartitionCols:_col0
-                        Group By Operator [GBY_5] (rows=1 width=114)
+                        Group By Operator [GBY_5] (rows=2 width=114)
                           Output:["_col0","_col1","_col2"],aggregations:["count()","count(p_name)"],keys:p_mfgr
                           Select Operator [SEL_4] (rows=5 width=223)
                             Output:["p_name","p_mfgr"]
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_2.q.out b/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
index 74d8578..0aadc25 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
@@ -336,34 +336,34 @@ Stage-0
     Stage-1
       Reducer 5 vectorized, llap
       File Output Operator [FS_217]
-        Limit [LIM_216] (rows=5 width=285)
+        Limit [LIM_216] (rows=2 width=285)
           Number of rows:100
-          Select Operator [SEL_215] (rows=5 width=285)
+          Select Operator [SEL_215] (rows=2 width=285)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 4 [SIMPLE_EDGE] vectorized, llap
             SHUFFLE [RS_214]
-              Group By Operator [GBY_213] (rows=5 width=285)
+              Group By Operator [GBY_213] (rows=2 width=285)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"],keys:KEY._col0, KEY._col1, KEY._col2
               <-Reducer 3 [SIMPLE_EDGE] llap
                 SHUFFLE [RS_49]
                   PartitionCols:_col0, _col1, _col2
-                  Group By Operator [GBY_48] (rows=5 width=285)
+                  Group By Operator [GBY_48] (rows=2 width=285)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(_col8)","count(_col15)","count(_col3)"],keys:_col7, _col14, _col2
-                    Top N Key Operator [TNK_91] (rows=4704 width=534)
+                    Top N Key Operator [TNK_91] (rows=28 width=534)
                       keys:_col7, _col14, _col2,sort order:+++,top n:100
-                      Merge Join Operator [MERGEJOIN_188] (rows=4704 width=534)
+                      Merge Join Operator [MERGEJOIN_188] (rows=28 width=534)
                         Conds:RS_44._col1, _col3=RS_45._col10, _col12(Inner),Output:["_col2","_col3","_col7","_col8","_col14","_col15"]
                       <-Reducer 10 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_45]
                           PartitionCols:_col10, _col12
-                          Select Operator [SEL_40] (rows=336 width=447)
+                          Select Operator [SEL_40] (rows=2 width=447)
                             Output:["_col2","_col3","_col9","_col10","_col12"]
-                            Merge Join Operator [MERGEJOIN_187] (rows=336 width=447)
+                            Merge Join Operator [MERGEJOIN_187] (rows=2 width=447)
                               Conds:RS_37._col2, _col4=RS_38._col1, _col3(Inner),Output:["_col0","_col1","_col9","_col10","_col12"]
                             <-Reducer 11 [SIMPLE_EDGE] llap
                               SHUFFLE [RS_38]
                                 PartitionCols:_col1, _col3
-                                Merge Join Operator [MERGEJOIN_186] (rows=8 width=356)
+                                Merge Join Operator [MERGEJOIN_186] (rows=5 width=356)
                                   Conds:RS_212._col0=RS_200._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                 <-Map 6 [SIMPLE_EDGE] vectorized, llap
                                   SHUFFLE [RS_200]
@@ -386,7 +386,7 @@ Stage-0
                             <-Reducer 9 [SIMPLE_EDGE] llap
                               SHUFFLE [RS_37]
                                 PartitionCols:_col2, _col4
-                                Merge Join Operator [MERGEJOIN_185] (rows=42 width=352)
+                                Merge Join Operator [MERGEJOIN_185] (rows=2 width=352)
                                   Conds:RS_34._col1=RS_209._col0(Inner),Output:["_col0","_col1","_col2","_col4"]
                                 <-Map 14 [SIMPLE_EDGE] vectorized, llap
                                   SHUFFLE [RS_209]
@@ -400,7 +400,7 @@ Stage-0
                                 <-Reducer 8 [SIMPLE_EDGE] llap
                                   SHUFFLE [RS_34]
                                     PartitionCols:_col1
-                                    Merge Join Operator [MERGEJOIN_184] (rows=42 width=352)
+                                    Merge Join Operator [MERGEJOIN_184] (rows=2 width=352)
                                       Conds:RS_31._col3=RS_206._col0(Inner),Output:["_col0","_col1","_col2","_col4"]
                                     <-Map 13 [SIMPLE_EDGE] vectorized, llap
                                       SHUFFLE [RS_206]
@@ -414,7 +414,7 @@ Stage-0
                                     <-Reducer 7 [SIMPLE_EDGE] llap
                                       SHUFFLE [RS_31]
                                         PartitionCols:_col3
-                                        Merge Join Operator [MERGEJOIN_183] (rows=7 width=443)
+                                        Merge Join Operator [MERGEJOIN_183] (rows=2 width=443)
                                           Conds:RS_203._col0=RS_199._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4"]
                                         <-Map 6 [SIMPLE_EDGE] vectorized, llap
                                           SHUFFLE [RS_199]
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 d51d8a7..63d5617 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
@@ -215,8 +215,7 @@ Plan optimized by CBO.
 Vertex dependency in root stage
 Map 1 <- Reducer 3 (BROADCAST_EDGE)
 Map 4 <- Map 1 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE)
-Map 7 <- Reducer 10 (BROADCAST_EDGE), Reducer 9 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 9 (CUSTOM_SIMPLE_EDGE)
+Map 7 <- Reducer 9 (BROADCAST_EDGE)
 Reducer 3 <- Map 2 (SIMPLE_EDGE)
 Reducer 5 <- Map 4 (SIMPLE_EDGE)
 Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
@@ -227,38 +226,38 @@ Stage-0
     limit:100
     Stage-1
       Reducer 6 vectorized, llap
-      File Output Operator [FS_148]
-        Limit [LIM_147] (rows=1 width=8)
+      File Output Operator [FS_143]
+        Limit [LIM_142] (rows=2 width=8)
           Number of rows:100
-          Select Operator [SEL_146] (rows=1 width=8)
+          Select Operator [SEL_141] (rows=2 width=8)
             Output:["_col0"]
           <-Reducer 5 [SIMPLE_EDGE] vectorized, llap
-            SHUFFLE [RS_145]
-              Select Operator [SEL_144] (rows=1 width=8)
+            SHUFFLE [RS_140]
+              Select Operator [SEL_139] (rows=2 width=8)
                 Output:["_col0"]
-                Group By Operator [GBY_143] (rows=1 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_142]
+                  SHUFFLE [RS_137]
                     PartitionCols:_col0
-                    Group By Operator [GBY_141] (rows=1 width=12)
+                    Group By Operator [GBY_136] (rows=2 width=12)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col8
-                      Map Join Operator [MAPJOIN_140] (rows=2592597 width=4)
-                        Conds:MAPJOIN_139._col6=RS_133._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_133]
+                        BROADCAST [RS_128]
                           PartitionCols:_col0
-                          Map Join Operator [MAPJOIN_132] (rows=14 width=8)
-                            Conds:SEL_131._col1=RS_124._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
-                            PARTITION_ONLY_SHUFFLE [RS_124]
+                            BROADCAST [RS_124]
                               PartitionCols:_col0
-                              Group By Operator [GBY_123] (rows=1 width=4)
+                              Group By Operator [GBY_123] (rows=2 width=4)
                                 Output:["_col0"],keys:KEY._col0
                               <-Map 8 [SIMPLE_EDGE] vectorized, llap
                                 SHUFFLE [RS_122]
                                   PartitionCols:_col0
-                                  Group By Operator [GBY_121] (rows=1 width=4)
+                                  Group By Operator [GBY_121] (rows=2 width=4)
                                     Output:["_col0"],keys:d_month_seq
                                     Select Operator [SEL_120] (rows=4 width=12)
                                       Output:["d_month_seq"]
@@ -266,24 +265,20 @@ Stage-0
                                         predicate:((d_year = 2000) and ((d_year * d_moy) > 200000) and (d_moy = 2) and d_month_seq is not null)
                                         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_131] (rows=28 width=8)
+                          <-Select Operator [SEL_126] (rows=28 width=8)
                               Output:["_col0","_col1"]
-                              Filter Operator [FIL_130] (rows=28 width=8)
-                                predicate:(d_date_sk is not null and d_month_seq is not null and d_month_seq BETWEEN DynamicValue(RS_25_x1_date_dim_d_month_seq_min) AND DynamicValue(RS_25_x1_date_dim_d_month_seq_max) and in_bloom_filter(d_month_seq, DynamicValue(RS_25_x1_date_dim_d_month_seq_bloom_filter)))
+                              Filter Operator [FIL_125] (rows=28 width=8)
+                                predicate:(d_date_sk is not null and d_month_seq is not null)
                                 TableScan [TS_14] (rows=28 width=8)
                                   default@x1_date_dim,d,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_month_seq"]
-                                <-Reducer 10 [BROADCAST_EDGE] vectorized, llap
-                                  BROADCAST [RS_129]
-                                    Group By Operator [GBY_128] (rows=1 width=12)
-                                      Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                        Dynamic Partitioning Event Operator [EVENT_136] (rows=1 width=4)
-                          Group By Operator [GBY_135] (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_134] (rows=14 width=8)
+                            Select Operator [SEL_129] (rows=28 width=8)
                               Output:["_col0"]
-                               Please refer to the previous Map Join Operator [MAPJOIN_132]
-                      <-Map Join Operator [MAPJOIN_139] (rows=370371 width=4)
-                          Conds:RS_31._col0=SEL_138._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_31]
                             PartitionCols:_col0
@@ -313,9 +308,9 @@ Stage-0
                                   predicate:(i_item_sk is not null and i_category is not null and i_current_price 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_138] (rows=123457 width=8)
+                        <-Select Operator [SEL_133] (rows=123457 width=8)
                             Output:["_col0","_col1"]
-                            Filter Operator [FIL_137] (rows=123457 width=8)
+                            Filter Operator [FIL_132] (rows=123457 width=8)
                               predicate:ss_item_sk is not null
                               TableScan [TS_11] (rows=123457 width=8)
                                 default@x1_store_sales,s,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_item_sk"]
diff --git a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
index 44c4568..59a9dfb 100644
--- a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
@@ -1597,7 +1597,7 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 18 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 12 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -1607,7 +1607,7 @@ STAGE PLANS:
                           outputColumnNames: _col1
                           input vertices:
                             1 Map 5
-                          Statistics: Num rows: 114 Data size: 10146 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 76 Data size: 6764 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -1616,7 +1616,7 @@ STAGE PLANS:
                               1 value (type: string)
                             input vertices:
                               1 Map 6
-                            Statistics: Num rows: 185 Data size: 1480 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 123 Data size: 984 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
                               aggregations: count()
                               minReductionHashAggr: 0.99
@@ -1828,7 +1828,7 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 4
-                        Statistics: Num rows: 18 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 12 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
                         Map Join Operator
                           condition map:
                                Inner Join 0 to 1
@@ -1838,7 +1838,7 @@ STAGE PLANS:
                           outputColumnNames: _col1
                           input vertices:
                             1 Map 5
-                          Statistics: Num rows: 114 Data size: 10146 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 76 Data size: 6764 Basic stats: COMPLETE Column stats: COMPLETE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -1847,7 +1847,7 @@ STAGE PLANS:
                               1 value (type: string)
                             input vertices:
                               1 Map 6
-                            Statistics: Num rows: 185 Data size: 1480 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 123 Data size: 984 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
                               aggregations: count()
                               minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/llap/reopt_dpp.q.out b/ql/src/test/results/clientpositive/llap/reopt_dpp.q.out
index 62c93d7..c48a389 100644
--- a/ql/src/test/results/clientpositive/llap/reopt_dpp.q.out
+++ b/ql/src/test/results/clientpositive/llap/reopt_dpp.q.out
@@ -166,8 +166,8 @@ Stage-0
                       predicate:((d_year = 2000) and d_date_sk is not null)
                       TableScan [TS_3] (rows=56 width=8)
                         default@x1_date_dim,d,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
-                Dynamic Partitioning Event Operator [EVENT_29] (rows=1 width=4)
-                  Group By Operator [GBY_28] (rows=1 width=4)
+                Dynamic Partitioning Event Operator [EVENT_29] (rows=2 width=4)
+                  Group By Operator [GBY_28] (rows=2 width=4)
                     Output:["_col0"],keys:_col0
                     Select Operator [SEL_27] (rows=28 width=4)
                       Output:["_col0"]
diff --git a/ql/src/test/results/clientpositive/llap/reopt_semijoin.q.out b/ql/src/test/results/clientpositive/llap/reopt_semijoin.q.out
index 527395a..5ad0104 100644
--- a/ql/src/test/results/clientpositive/llap/reopt_semijoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/reopt_semijoin.q.out
@@ -191,7 +191,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=2)
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=4)
                           minReductionHashAggr: 0.75
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
@@ -242,7 +242,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=2)
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=4)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/retry_failure_reorder.q.out b/ql/src/test/results/clientpositive/llap/retry_failure_reorder.q.out
index f404daa..baeac43 100644
--- a/ql/src/test/results/clientpositive/llap/retry_failure_reorder.q.out
+++ b/ql/src/test/results/clientpositive/llap/retry_failure_reorder.q.out
@@ -276,9 +276,9 @@ Stage-0
             PARTITION_ONLY_SHUFFLE [RS_18]
               Group By Operator [GBY_17] (rows=1 width=8)
                 Output:["_col0"],aggregations:["sum(_col0)"]
-                Select Operator [SEL_15] (rows=105 width=12)
+                Select Operator [SEL_15] (rows=87 width=12)
                   Output:["_col0"]
-                  Merge Join Operator [MERGEJOIN_51] (rows=105 width=12)
+                  Merge Join Operator [MERGEJOIN_51] (rows=87 width=12)
                     Conds:RS_12._col0=RS_60._col0(Inner),Output:["_col2","_col4","_col6"]
                   <-Map 6 [SIMPLE_EDGE] vectorized, llap
                     SHUFFLE [RS_60]
diff --git a/ql/src/test/results/clientpositive/llap/subquery_exists.q.out b/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
index 3c042f6..859025a 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_exists.q.out
@@ -88,10 +88,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
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 e1fc35f..8d68a68 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
@@ -209,10 +209,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -723,10 +723,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 131 Data size: 23318 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1151,15 +1151,15 @@ STAGE PLANS:
                       Statistics: Num rows: 14 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.71428573
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
-                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1189,14 +1189,14 @@ STAGE PLANS:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col1, _col2
-                Statistics: Num rows: 14 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col1 (type: int), _col2 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 14 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 14 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
index b4c9656..ac253a2 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
@@ -3203,14 +3203,14 @@ STAGE PLANS:
                   0 _col1 (type: int), _col4 (type: int)
                   1 _col0 (type: int), _col1 (type: int)
                 outputColumnNames: _col0, _col3
-                Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col3 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
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 6aecf7e..b9c674f 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
@@ -4362,14 +4362,14 @@ STAGE PLANS:
                   1 _col0 (type: int)
                 outputColumnNames: _col1, _col2, _col5
                 residual filter predicates: {(_col1 > _col5)}
-                Statistics: Num rows: 5 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 8 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col2 (type: double)
                   outputColumnNames: _col2
-                  Statistics: Num rows: 5 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 8 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col2)
-                    minReductionHashAggr: 0.8
+                    minReductionHashAggr: 0.875
                     mode: hash
                     outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out
index 30c867e..4333410 100644
--- a/ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_interval_mapjoin.q.out
@@ -240,7 +240,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         input vertices:
                           1 Map 2
-                        Statistics: Num rows: 32218 Data size: 6443600 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 29831 Data size: 5966200 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: _col0 (type: string), _col2 (type: string), _col1 (type: interval_day_time)
                           outputColumnNames: _col0, _col1, _col2
@@ -248,13 +248,13 @@ STAGE PLANS:
                               className: VectorSelectOperator
                               native: true
                               projectedOutputColumnNums: [8, 8, 16]
-                          Statistics: Num rows: 32218 Data size: 6443600 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 29831 Data size: 5966200 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
                             compressed: false
                             File Sink Vectorization:
                                 className: VectorFileSinkOperator
                                 native: false
-                            Statistics: Num rows: 32218 Data size: 6443600 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 29831 Data size: 5966200 Basic stats: COMPLETE Column stats: COMPLETE
                             table:
                                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out b/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out
index eca66c0..4eef3d5 100644
--- a/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out
@@ -154,10 +154,10 @@ STAGE PLANS:
                             vectorProcessingMode: HASH
                             projectedOutputColumnNums: []
                         keys: _col0 (type: int)
-                        minReductionHashAggr: 0.71428573
+                        minReductionHashAggr: 0.5
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
                           sort order: +
@@ -166,7 +166,7 @@ STAGE PLANS:
                               className: VectorReduceSinkLongOperator
                               native: true
                               nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                          Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 7 Data size: 28 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -228,7 +228,7 @@ STAGE PLANS:
                     outputColumnNames: _col1, _col2
                     input vertices:
                       1 Map 4
-                    Statistics: Num rows: 14 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: _col1 (type: int), _col2 (type: int)
                       outputColumnNames: _col0, _col1
@@ -236,13 +236,13 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [0, 2]
-                      Statistics: Num rows: 14 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
                         File Sink Vectorization:
                             className: VectorFileSinkOperator
                             native: false
-                        Statistics: Num rows: 14 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -372,7 +372,7 @@ STAGE PLANS:
                             outputColumnNames: _col0, _col3
                             input vertices:
                               1 Map 3
-                            Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                             Select Operator
                               expressions: _col0 (type: int), _col3 (type: int)
                               outputColumnNames: _col0, _col1
@@ -380,13 +380,13 @@ STAGE PLANS:
                                   className: VectorSelectOperator
                                   native: true
                                   projectedOutputColumnNums: [1, 2]
-                              Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                               File Output Operator
                                 compressed: false
                                 File Sink Vectorization:
                                     className: VectorFileSinkOperator
                                     native: false
-                                Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                                 table:
                                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/mapjoin47.q.out b/ql/src/test/results/clientpositive/mapjoin47.q.out
index f6eafbc..9e19cef 100644
--- a/ql/src/test/results/clientpositive/mapjoin47.q.out
+++ b/ql/src/test/results/clientpositive/mapjoin47.q.out
@@ -71,13 +71,13 @@ STAGE PLANS:
                     0 _col0 (type: string)
                     1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 10
-                    Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
-                      Statistics: Num rows: 3 Data size: 1059 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 706 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/masking_2.q.out b/ql/src/test/results/clientpositive/masking_2.q.out
index 9bb3e73..8a96df7 100644
--- a/ql/src/test/results/clientpositive/masking_2.q.out
+++ b/ql/src/test/results/clientpositive/masking_2.q.out
@@ -272,10 +272,10 @@ STAGE PLANS:
             0 UDFToDouble(_col0) (type: double)
             1 UDFToDouble(_col1) (type: double)
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 42 Data size: 8064 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 27 Data size: 5184 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 42 Data size: 8064 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 27 Data size: 5184 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/masking_disablecbo_2.q.out b/ql/src/test/results/clientpositive/masking_disablecbo_2.q.out
index 013aa53..4938c8f 100644
--- a/ql/src/test/results/clientpositive/masking_disablecbo_2.q.out
+++ b/ql/src/test/results/clientpositive/masking_disablecbo_2.q.out
@@ -262,10 +262,10 @@ STAGE PLANS:
             0 UDFToDouble(_col0) (type: double)
             1 UDFToDouble(_col1) (type: double)
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 42 Data size: 8064 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 27 Data size: 5184 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 42 Data size: 8064 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 27 Data size: 5184 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
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 0ad4660..c233338 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[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
+Warning: Shuffle Join MERGEJOIN[454][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 23' is a cross product
+Warning: Shuffle Join MERGEJOIN[456][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 24' 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
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 bae58bd..06252d6 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,4 @@
-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
+Warning: Shuffle Join MERGEJOIN[358][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 15' 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
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 4f3965b..ece76c3 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[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
+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
 PREHOOK: query: explain cbo
 with my_customers as (
  select distinct c_customer_sk
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 21ad48c..3aea12d 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
@@ -76,13 +76,13 @@ Stage-0
       File Output Operator [FS_164]
         Limit [LIM_163] (rows=100 width=100)
           Number of rows:100
-          Select Operator [SEL_162] (rows=663970 width=100)
+          Select Operator [SEL_162] (rows=17457727 width=100)
             Output:["_col0"]
           <-Reducer 4 [SIMPLE_EDGE]
             SHUFFLE [RS_51]
-              Select Operator [SEL_50] (rows=663970 width=100)
+              Select Operator [SEL_50] (rows=17457727 width=100)
                 Output:["_col0"]
-                Merge Join Operator [MERGEJOIN_136] (rows=663970 width=100)
+                Merge Join Operator [MERGEJOIN_136] (rows=17457727 width=100)
                   Conds:RS_47._col1=RS_161._col0(Inner),Output:["_col7"]
                 <-Map 12 [SIMPLE_EDGE] vectorized
                   SHUFFLE [RS_161]
@@ -94,29 +94,29 @@ Stage-0
                 <-Reducer 3 [SIMPLE_EDGE]
                   SHUFFLE [RS_47]
                     PartitionCols:_col1
-                    Filter Operator [FIL_46] (rows=663970 width=225)
+                    Filter Operator [FIL_46] (rows=17457727 width=227)
                       predicate:(_col3 > _col4)
-                      Merge Join Operator [MERGEJOIN_135] (rows=1991910 width=225)
+                      Merge Join Operator [MERGEJOIN_135] (rows=52373181 width=227)
                         Conds:RS_43._col2=RS_159._col1(Inner),Output:["_col1","_col3","_col4"]
                       <-Reducer 10 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_159]
                           PartitionCols:_col1
-                          Select Operator [SEL_158] (rows=29 width=115)
+                          Select Operator [SEL_158] (rows=85 width=115)
                             Output:["_col0","_col1"]
-                            Filter Operator [FIL_157] (rows=29 width=123)
+                            Filter Operator [FIL_157] (rows=85 width=123)
                               predicate:(_col1 is not null and _col2 is not null)
-                              Group By Operator [GBY_156] (rows=29 width=123)
+                              Group By Operator [GBY_156] (rows=85 width=123)
                                 Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
-                                Select Operator [SEL_155] (rows=13369812 width=119)
+                                Select Operator [SEL_155] (rows=53634860 width=119)
                                   Output:["_col1","_col2"]
-                                  Group By Operator [GBY_154] (rows=13369812 width=119)
+                                  Group By Operator [GBY_154] (rows=53634860 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)
+                                      Group By Operator [GBY_28] (rows=53634860 width=119)
                                         Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col2, _col1
-                                        Merge Join Operator [MERGEJOIN_134] (rows=17467258 width=107)
+                                        Merge Join Operator [MERGEJOIN_134] (rows=53634860 width=115)
                                           Conds:RS_145._col0=RS_149._col0(Inner),Output:["_col1","_col2","_col3"]
                                         <-Map 11 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_149]
@@ -139,7 +139,7 @@ Stage-0
                       <-Reducer 2 [SIMPLE_EDGE]
                         SHUFFLE [RS_43]
                           PartitionCols:_col2
-                          Merge Join Operator [MERGEJOIN_133] (rows=1923224 width=114)
+                          Merge Join Operator [MERGEJOIN_133] (rows=51757026 width=119)
                             Conds:RS_139._col0=RS_153._col1(Inner),Output:["_col1","_col2","_col3"]
                           <-Map 1 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_139]
@@ -153,18 +153,18 @@ Stage-0
                           <-Reducer 8 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_153]
                               PartitionCols:_col1
-                              Select Operator [SEL_152] (rows=11601100 width=119)
+                              Select Operator [SEL_152] (rows=51757026 width=119)
                                 Output:["_col0","_col1","_col2"]
-                                Filter Operator [FIL_151] (rows=11601100 width=119)
+                                Filter Operator [FIL_151] (rows=51757026 width=119)
                                   predicate:_col2 is not null
-                                  Group By Operator [GBY_150] (rows=11601100 width=119)
+                                  Group By Operator [GBY_150] (rows=51757026 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)
+                                      Group By Operator [GBY_13] (rows=51757026 width=119)
                                         Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col2, _col1
-                                        Merge Join Operator [MERGEJOIN_132] (rows=16855704 width=107)
+                                        Merge Join Operator [MERGEJOIN_132] (rows=51757026 width=115)
                                           Conds:RS_144._col0=RS_148._col0(Inner),Output:["_col1","_col2","_col3"]
                                         <-Map 11 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_148]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query10.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query10.q.out
index 3277992..b373ca4 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query10.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query10.q.out
@@ -133,98 +133,174 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 13 <- Reducer 16 (BROADCAST_EDGE)
-Map 21 <- Reducer 10 (BROADCAST_EDGE)
-Map 22 <- Reducer 9 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 14 <- Map 13 (SIMPLE_EDGE), Map 15 (SIMPLE_EDGE)
-Reducer 16 <- Map 15 (CUSTOM_SIMPLE_EDGE)
-Reducer 17 <- Map 15 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
-Reducer 18 <- Reducer 17 (SIMPLE_EDGE)
-Reducer 19 <- Map 15 (SIMPLE_EDGE), Map 22 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
-Reducer 20 <- Reducer 19 (SIMPLE_EDGE)
-Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Reducer 14 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 18 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-Reducer 6 <- Reducer 20 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Map 11 <- Reducer 14 (BROADCAST_EDGE)
+Map 21 <- Reducer 17 (BROADCAST_EDGE)
+Map 22 <- Reducer 20 (BROADCAST_EDGE)
+Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 13 (SIMPLE_EDGE)
+Reducer 14 <- Map 13 (CUSTOM_SIMPLE_EDGE)
+Reducer 15 <- Map 13 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
+Reducer 16 <- Reducer 15 (SIMPLE_EDGE)
+Reducer 17 <- Map 13 (CUSTOM_SIMPLE_EDGE)
+Reducer 18 <- Map 13 (SIMPLE_EDGE), Map 22 (SIMPLE_EDGE)
+Reducer 19 <- Reducer 18 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 20 <- Map 13 (CUSTOM_SIMPLE_EDGE)
+Reducer 3 <- Map 10 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 16 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 19 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
-Reducer 9 <- Reducer 5 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:100
     Stage-1
       Reducer 8 vectorized
-      File Output Operator [FS_223]
-        Limit [LIM_222] (rows=1 width=419)
+      File Output Operator [FS_229]
+        Limit [LIM_228] (rows=1 width=419)
           Number of rows:100
-          Select Operator [SEL_221] (rows=1 width=419)
+          Select Operator [SEL_227] (rows=1 width=419)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13"]
           <-Reducer 7 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_220]
-              Select Operator [SEL_219] (rows=1 width=419)
+            SHUFFLE [RS_226]
+              Select Operator [SEL_225] (rows=1 width=419)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col8","_col10","_col12"]
-                Group By Operator [GBY_218] (rows=1 width=379)
+                Group By Operator [GBY_224] (rows=1 width=379)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7
                 <-Reducer 6 [SIMPLE_EDGE]
                   SHUFFLE [RS_66]
                     PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                    Group By Operator [GBY_65] (rows=1 width=379)
+                    Group By Operator [GBY_65] (rows=3 width=379)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["count()"],keys:_col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-                      Top N Key Operator [TNK_102] (rows=52 width=379)
+                      Top N Key Operator [TNK_102] (rows=1401496 width=379)
                         keys:_col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13,sort order:++++++++,top n:100
-                        Select Operator [SEL_64] (rows=52 width=379)
+                        Select Operator [SEL_64] (rows=1401496 width=379)
                           Output:["_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13"]
-                          Filter Operator [FIL_63] (rows=52 width=379)
+                          Filter Operator [FIL_63] (rows=1401496 width=379)
                             predicate:(_col14 is not null or _col16 is not null)
-                            Merge Join Operator [MERGEJOIN_180] (rows=52 width=379)
-                              Conds:RS_60._col0=RS_217._col1(Left Outer),Output:["_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col16"]
+                            Merge Join Operator [MERGEJOIN_180] (rows=1401496 width=379)
+                              Conds:RS_60._col0=RS_223._col1(Left Outer),Output:["_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col16"]
+                            <-Reducer 19 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_223]
+                                PartitionCols:_col1
+                                Select Operator [SEL_222] (rows=1401496 width=7)
+                                  Output:["_col0","_col1"]
+                                  Group By Operator [GBY_221] (rows=1401496 width=3)
+                                    Output:["_col0"],keys:KEY._col0
+                                  <-Reducer 18 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_43]
+                                      PartitionCols:_col0
+                                      Group By Operator [GBY_42] (rows=285115246 width=3)
+                                        Output:["_col0"],keys:_col1
+                                        Merge Join Operator [MERGEJOIN_177] (rows=285115246 width=3)
+                                          Conds:RS_220._col0=RS_195._col0(Inner),Output:["_col1"]
+                                        <-Map 13 [SIMPLE_EDGE] vectorized
+                                          PARTITION_ONLY_SHUFFLE [RS_195]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_190] (rows=201 width=4)
+                                              Output:["_col0"]
+                                              Filter Operator [FIL_189] (rows=201 width=12)
+                                                predicate:((d_year = 2002) and d_moy BETWEEN 4 AND 7)
+                                                TableScan [TS_11] (rows=73049 width=12)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
+                                        <-Map 22 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_220]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_219] (rows=285115246 width=7)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_218] (rows=285115246 width=7)
+                                                predicate:(cs_ship_customer_sk is not null and cs_sold_date_sk is not null and cs_sold_date_sk BETWEEN DynamicValue(RS_39_date_dim_d_date_sk_min) AND DynamicValue(RS_39_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_39_date_dim_d_date_sk_bloom_filter)))
+                                                TableScan [TS_32] (rows=287989836 width=7)
+                                                  default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_ship_customer_sk"]
+                                                <-Reducer 20 [BROADCAST_EDGE] vectorized
+                                                  BROADCAST [RS_217]
+                                                    Group By Operator [GBY_216] (rows=1 width=12)
+                                                      Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                    <-Map 13 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                      PARTITION_ONLY_SHUFFLE [RS_202]
+                                                        Group By Operator [GBY_199] (rows=1 width=12)
+                                                          Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                          Select Operator [SEL_196] (rows=201 width=4)
+                                                            Output:["_col0"]
+                                                             Please refer to the previous Select Operator [SEL_190]
                             <-Reducer 5 [SIMPLE_EDGE]
-                              PARTITION_ONLY_SHUFFLE [RS_60]
+                              SHUFFLE [RS_60]
                                 PartitionCols:_col0
-                                Merge Join Operator [MERGEJOIN_179] (rows=53 width=379)
-                                  Conds:RS_57._col0=RS_209._col1(Left Outer),Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"]
+                                Merge Join Operator [MERGEJOIN_179] (rows=1414922 width=379)
+                                  Conds:RS_57._col0=RS_215._col1(Left Outer),Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"]
+                                <-Reducer 16 [SIMPLE_EDGE] vectorized
+                                  SHUFFLE [RS_215]
+                                    PartitionCols:_col1
+                                    Select Operator [SEL_214] (rows=1414922 width=7)
+                                      Output:["_col0","_col1"]
+                                      Group By Operator [GBY_213] (rows=1414922 width=3)
+                                        Output:["_col0"],keys:KEY._col0
+                                      <-Reducer 15 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_29]
+                                          PartitionCols:_col0
+                                          Group By Operator [GBY_28] (rows=143930993 width=3)
+                                            Output:["_col0"],keys:_col1
+                                            Merge Join Operator [MERGEJOIN_176] (rows=143930993 width=3)
+                                              Conds:RS_212._col0=RS_193._col0(Inner),Output:["_col1"]
+                                            <-Map 13 [SIMPLE_EDGE] vectorized
+                                              PARTITION_ONLY_SHUFFLE [RS_193]
+                                                PartitionCols:_col0
+                                                 Please refer to the previous Select Operator [SEL_190]
+                                            <-Map 21 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_212]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_211] (rows=143930993 width=7)
+                                                  Output:["_col0","_col1"]
+                                                  Filter Operator [FIL_210] (rows=143930993 width=7)
+                                                    predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_sold_date_sk BETWEEN DynamicValue(RS_25_date_dim_d_date_sk_min) AND DynamicValue(RS_25_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_25_date_dim_d_date_sk_bloom_filter)))
+                                                    TableScan [TS_18] (rows=144002668 width=7)
+                                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_bill_customer_sk"]
+                                                    <-Reducer 17 [BROADCAST_EDGE] vectorized
+                                                      BROADCAST [RS_209]
+                                                        Group By Operator [GBY_208] (rows=1 width=12)
+                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                        <-Map 13 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                          PARTITION_ONLY_SHUFFLE [RS_201]
+                                                            Group By Operator [GBY_198] (rows=1 width=12)
+                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                              Select Operator [SEL_194] (rows=201 width=4)
+                                                                Output:["_col0"]
+                                                                 Please refer to the previous Select Operator [SEL_190]
                                 <-Reducer 4 [SIMPLE_EDGE]
                                   SHUFFLE [RS_57]
                                     PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_178] (rows=20705 width=375)
+                                    Merge Join Operator [MERGEJOIN_178] (rows=525327388 width=375)
                                       Conds:RS_54._col0=RS_55._col0(Left Semi),Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13"]
-                                    <-Reducer 14 [SIMPLE_EDGE]
+                                    <-Reducer 12 [SIMPLE_EDGE]
                                       SHUFFLE [RS_55]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_53] (rows=142119 width=2)
+                                        Group By Operator [GBY_53] (rows=525327388 width=3)
                                           Output:["_col0"],keys:_col0
-                                          Select Operator [SEL_17] (rows=57825495 width=2)
+                                          Select Operator [SEL_17] (rows=525327388 width=3)
                                             Output:["_col0"]
-                                            Merge Join Operator [MERGEJOIN_175] (rows=57825495 width=2)
-                                              Conds:RS_201._col0=RS_191._col0(Inner),Output:["_col1"]
-                                            <-Map 15 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_191]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_190] (rows=201 width=4)
-                                                  Output:["_col0"]
-                                                  Filter Operator [FIL_189] (rows=201 width=12)
-                                                    predicate:((d_year = 2002) and d_moy BETWEEN 4 AND 7)
-                                                    TableScan [TS_11] (rows=73049 width=12)
-                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
+                                            Merge Join Operator [MERGEJOIN_175] (rows=525327388 width=3)
+                                              Conds:RS_207._col0=RS_191._col0(Inner),Output:["_col1"]
                                             <-Map 13 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_201]
+                                              PARTITION_ONLY_SHUFFLE [RS_191]
                                                 PartitionCols:_col0
-                                                Select Operator [SEL_200] (rows=525327388 width=7)
+                                                 Please refer to the previous Select Operator [SEL_190]
+                                            <-Map 11 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_207]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_206] (rows=525327388 width=7)
                                                   Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_199] (rows=525327388 width=7)
+                                                  Filter Operator [FIL_205] (rows=525327388 width=7)
                                                     predicate:(ss_sold_date_sk is not null and ss_customer_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_15_date_dim_d_date_sk_min) AND DynamicValue(RS_15_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_15_date_dim_d_date_sk_bloom_filter)))
                                                     TableScan [TS_8] (rows=575995635 width=7)
                                                       default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_customer_sk"]
-                                                    <-Reducer 16 [BROADCAST_EDGE] vectorized
-                                                      BROADCAST [RS_198]
-                                                        Group By Operator [GBY_197] (rows=1 width=12)
+                                                    <-Reducer 14 [BROADCAST_EDGE] vectorized
+                                                      BROADCAST [RS_204]
+                                                        Group By Operator [GBY_203] (rows=1 width=12)
                                                           Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                        <-Map 15 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                          SHUFFLE [RS_196]
-                                                            Group By Operator [GBY_195] (rows=1 width=12)
+                                                        <-Map 13 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                          PARTITION_ONLY_SHUFFLE [RS_200]
+                                                            Group By Operator [GBY_197] (rows=1 width=12)
                                                               Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
                                                               Select Operator [SEL_192] (rows=201 width=4)
                                                                 Output:["_col0"]
@@ -234,7 +310,7 @@ Stage-0
                                         PartitionCols:_col0
                                         Merge Join Operator [MERGEJOIN_174] (rows=228127 width=375)
                                           Conds:RS_49._col1=RS_188._col0(Inner),Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13"]
-                                        <-Map 12 [SIMPLE_EDGE] vectorized
+                                        <-Map 10 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_188]
                                             PartitionCols:_col0
                                             Select Operator [SEL_187] (rows=1861800 width=375)
@@ -255,7 +331,7 @@ Stage-0
                                                     predicate:(c_current_cdemo_sk is not null and c_current_addr_sk is not null)
                                                     TableScan [TS_0] (rows=80000000 width=11)
                                                       default@customer,c,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_current_cdemo_sk","c_current_addr_sk"]
-                                            <-Map 11 [SIMPLE_EDGE] vectorized
+                                            <-Map 9 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_186]
                                                 PartitionCols:_col0
                                                 Select Operator [SEL_185] (rows=116550 width=102)
@@ -264,80 +340,4 @@ Stage-0
                                                     predicate:(ca_county) IN ('Walker County', 'Richland County', 'Gaines County', 'Douglas County', 'Dona Ana County')
                                                     TableScan [TS_3] (rows=40000000 width=102)
                                                       default@customer_address,ca,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_county"]
-                                <-Reducer 18 [SIMPLE_EDGE] vectorized
-                                  SHUFFLE [RS_209]
-                                    PartitionCols:_col1
-                                    Select Operator [SEL_208] (rows=155749 width=7)
-                                      Output:["_col0","_col1"]
-                                      Group By Operator [GBY_207] (rows=155749 width=3)
-                                        Output:["_col0"],keys:KEY._col0
-                                      <-Reducer 17 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_29]
-                                          PartitionCols:_col0
-                                          Group By Operator [GBY_28] (rows=155749 width=3)
-                                            Output:["_col0"],keys:_col1
-                                            Merge Join Operator [MERGEJOIN_176] (rows=15843227 width=3)
-                                              Conds:RS_206._col0=RS_193._col0(Inner),Output:["_col1"]
-                                            <-Map 15 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_193]
-                                                PartitionCols:_col0
-                                                 Please refer to the previous Select Operator [SEL_190]
-                                            <-Map 21 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_206]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_205] (rows=143930993 width=7)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_204] (rows=143930993 width=7)
-                                                    predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_bill_customer_sk BETWEEN DynamicValue(RS_57_c_c_customer_sk_min) AND DynamicValue(RS_57_c_c_customer_sk_max) and in_bloom_filter(ws_bill_customer_sk, DynamicValue(RS_57_c_c_customer_sk_bloom_filter)))
-                                                    TableScan [TS_18] (rows=144002668 width=7)
-                                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_bill_customer_sk"]
-                                                    <-Reducer 10 [BROADCAST_EDGE] vectorized
-                                                      BROADCAST [RS_203]
-                                                        Group By Operator [GBY_202] (rows=1 width=12)
-                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                        <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-                                                          SHUFFLE [RS_150]
-                                                            Group By Operator [GBY_149] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                              Select Operator [SEL_148] (rows=20705 width=4)
-                                                                Output:["_col0"]
-                                                                 Please refer to the previous Merge Join Operator [MERGEJOIN_178]
-                            <-Reducer 20 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_217]
-                                PartitionCols:_col1
-                                Select Operator [SEL_216] (rows=153181 width=7)
-                                  Output:["_col0","_col1"]
-                                  Group By Operator [GBY_215] (rows=153181 width=3)
-                                    Output:["_col0"],keys:KEY._col0
-                                  <-Reducer 19 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_43]
-                                      PartitionCols:_col0
-                                      Group By Operator [GBY_42] (rows=153181 width=3)
-                                        Output:["_col0"],keys:_col1
-                                        Merge Join Operator [MERGEJOIN_177] (rows=31162251 width=3)
-                                          Conds:RS_214._col0=RS_194._col0(Inner),Output:["_col1"]
-                                        <-Map 15 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_194]
-                                            PartitionCols:_col0
-                                             Please refer to the previous Select Operator [SEL_190]
-                                        <-Map 22 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_214]
-                                            PartitionCols:_col0
-                                            Select Operator [SEL_213] (rows=285115246 width=7)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_212] (rows=285115246 width=7)
-                                                predicate:(cs_ship_customer_sk is not null and cs_sold_date_sk is not null and cs_ship_customer_sk BETWEEN DynamicValue(RS_60_c_c_customer_sk_min) AND DynamicValue(RS_60_c_c_customer_sk_max) and in_bloom_filter(cs_ship_customer_sk, DynamicValue(RS_60_c_c_customer_sk_bloom_filter)))
-                                                TableScan [TS_32] (rows=287989836 width=7)
-                                                  default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_ship_customer_sk"]
-                                                <-Reducer 9 [BROADCAST_EDGE] vectorized
-                                                  BROADCAST [RS_211]
-                                                    Group By Operator [GBY_210] (rows=1 width=12)
-                                                      Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                    <-Reducer 5 [CUSTOM_SIMPLE_EDGE]
-                                                      PARTITION_ONLY_SHUFFLE [RS_165]
-                                                        Group By Operator [GBY_164] (rows=1 width=12)
-                                                          Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                          Select Operator [SEL_163] (rows=53 width=4)
-                                                            Output:["_col0"]
-                                                             Please refer to the previous Merge Join Operator [MERGEJOIN_179]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query11.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query11.q.out
index 4d0ff49..cc47c7e 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query11.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query11.q.out
@@ -192,15 +192,15 @@ Stage-0
       File Output Operator [FS_349]
         Limit [LIM_348] (rows=100 width=85)
           Number of rows:100
-          Select Operator [SEL_347] (rows=12248094 width=85)
+          Select Operator [SEL_347] (rows=19066162 width=85)
             Output:["_col0"]
           <-Reducer 7 [SIMPLE_EDGE]
             SHUFFLE [RS_89]
-              Select Operator [SEL_88] (rows=12248094 width=85)
+              Select Operator [SEL_88] (rows=19066162 width=85)
                 Output:["_col0"]
-                Filter Operator [FIL_87] (rows=12248094 width=537)
+                Filter Operator [FIL_87] (rows=19066162 width=537)
                   predicate:CASE WHEN (_col4 is not null) THEN (CASE WHEN (_col2) THEN (((_col6 / _col1) > (_col9 / _col4))) ELSE (false) END) ELSE (false) END
-                  Merge Join Operator [MERGEJOIN_283] (rows=24496188 width=537)
+                  Merge Join Operator [MERGEJOIN_283] (rows=38132324 width=537)
                     Conds:RS_84._col3=RS_346._col0(Inner),Output:["_col1","_col2","_col4","_col6","_col8","_col9"]
                   <-Reducer 20 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_346]
@@ -214,7 +214,7 @@ Stage-0
                             PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
                             Group By Operator [GBY_74] (rows=80000000 width=764)
                               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                              Merge Join Operator [MERGEJOIN_280] (rows=187573258 width=764)
+                              Merge Join Operator [MERGEJOIN_280] (rows=525327388 width=764)
                                 Conds:RS_70._col1=RS_312._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                               <-Map 26 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_312]
@@ -226,7 +226,7 @@ Stage-0
                               <-Reducer 18 [SIMPLE_EDGE]
                                 SHUFFLE [RS_70]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_279] (rows=187573258 width=115)
+                                  Merge Join Operator [MERGEJOIN_279] (rows=525327388 width=115)
                                     Conds:RS_343._col0=RS_290._col0(Inner),Output:["_col1","_col2"]
                                   <-Map 21 [SIMPLE_EDGE] vectorized
                                     PARTITION_ONLY_SHUFFLE [RS_290]
@@ -260,21 +260,21 @@ Stage-0
                   <-Reducer 6 [SIMPLE_EDGE]
                     SHUFFLE [RS_84]
                       PartitionCols:_col3
-                      Merge Join Operator [MERGEJOIN_282] (rows=20485012 width=440)
+                      Merge Join Operator [MERGEJOIN_282] (rows=31888273 width=440)
                         Conds:RS_81._col3=RS_338._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6"]
                       <-Reducer 16 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_338]
                           PartitionCols:_col0
-                          Select Operator [SEL_337] (rows=51391963 width=212)
+                          Select Operator [SEL_337] (rows=80000000 width=212)
                             Output:["_col0","_col1"]
-                            Group By Operator [GBY_336] (rows=51391963 width=764)
+                            Group By Operator [GBY_336] (rows=80000000 width=764)
                               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6
                             <-Reducer 15 [SIMPLE_EDGE]
                               SHUFFLE [RS_56]
                                 PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
-                                Group By Operator [GBY_55] (rows=51391963 width=764)
+                                Group By Operator [GBY_55] (rows=80000000 width=764)
                                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                                  Merge Join Operator [MERGEJOIN_278] (rows=51391963 width=764)
+                                  Merge Join Operator [MERGEJOIN_278] (rows=143930993 width=764)
                                     Conds:RS_51._col1=RS_313._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                   <-Map 26 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_313]
@@ -283,7 +283,7 @@ Stage-0
                                   <-Reducer 14 [SIMPLE_EDGE]
                                     SHUFFLE [RS_51]
                                       PartitionCols:_col1
-                                      Merge Join Operator [MERGEJOIN_277] (rows=51391963 width=115)
+                                      Merge Join Operator [MERGEJOIN_277] (rows=143930993 width=115)
                                         Conds:RS_335._col0=RS_292._col0(Inner),Output:["_col1","_col2"]
                                       <-Map 21 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_292]
@@ -312,7 +312,7 @@ Stage-0
                       <-Reducer 5 [SIMPLE_EDGE]
                         SHUFFLE [RS_81]
                           PartitionCols:_col3
-                          Merge Join Operator [MERGEJOIN_281] (rows=17130654 width=328)
+                          Merge Join Operator [MERGEJOIN_281] (rows=26666666 width=328)
                             Conds:RS_320._col0=RS_330._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                           <-Reducer 12 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_330]
@@ -330,7 +330,7 @@ Stage-0
                                         PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
                                         Group By Operator [GBY_35] (rows=80000000 width=764)
                                           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                                          Merge Join Operator [MERGEJOIN_276] (rows=187573258 width=764)
+                                          Merge Join Operator [MERGEJOIN_276] (rows=525327388 width=764)
                                             Conds:RS_31._col1=RS_315._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                           <-Map 26 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_315]
@@ -339,7 +339,7 @@ Stage-0
                                           <-Reducer 10 [SIMPLE_EDGE]
                                             SHUFFLE [RS_31]
                                               PartitionCols:_col1
-                                              Merge Join Operator [MERGEJOIN_275] (rows=187573258 width=115)
+                                              Merge Join Operator [MERGEJOIN_275] (rows=525327388 width=115)
                                                 Conds:RS_325._col0=RS_296._col0(Inner),Output:["_col1","_col2"]
                                               <-Map 21 [SIMPLE_EDGE] vectorized
                                                 PARTITION_ONLY_SHUFFLE [RS_296]
@@ -372,20 +372,20 @@ Stage-0
                           <-Reducer 4 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_320]
                               PartitionCols:_col0
-                              Select Operator [SEL_319] (rows=17130654 width=216)
+                              Select Operator [SEL_319] (rows=26666666 width=216)
                                 Output:["_col0","_col1","_col2"]
-                                Filter Operator [FIL_318] (rows=17130654 width=212)
+                                Filter Operator [FIL_318] (rows=26666666 width=212)
                                   predicate:(_col7 > 0)
-                                  Select Operator [SEL_317] (rows=51391963 width=212)
+                                  Select Operator [SEL_317] (rows=80000000 width=212)
                                     Output:["_col0","_col7"]
-                                    Group By Operator [GBY_316] (rows=51391963 width=764)
+                                    Group By Operator [GBY_316] (rows=80000000 width=764)
                                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6
                                     <-Reducer 3 [SIMPLE_EDGE]
                                       SHUFFLE [RS_16]
                                         PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
-                                        Group By Operator [GBY_15] (rows=51391963 width=764)
+                                        Group By Operator [GBY_15] (rows=80000000 width=764)
                                           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                                          Merge Join Operator [MERGEJOIN_274] (rows=51391963 width=764)
+                                          Merge Join Operator [MERGEJOIN_274] (rows=143930993 width=764)
                                             Conds:RS_11._col1=RS_314._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                           <-Map 26 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_314]
@@ -394,7 +394,7 @@ Stage-0
                                           <-Reducer 2 [SIMPLE_EDGE]
                                             SHUFFLE [RS_11]
                                               PartitionCols:_col1
-                                              Merge Join Operator [MERGEJOIN_273] (rows=51391963 width=115)
+                                              Merge Join Operator [MERGEJOIN_273] (rows=143930993 width=115)
                                                 Conds:RS_310._col0=RS_294._col0(Inner),Output:["_col1","_col2"]
                                               <-Map 21 [SIMPLE_EDGE] vectorized
                                                 PARTITION_ONLY_SHUFFLE [RS_294]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query12.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query12.q.out
index b62589b..57c1634 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query12.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query12.q.out
@@ -95,21 +95,21 @@ Stage-0
             SHUFFLE [RS_78]
               Select Operator [SEL_77] (rows=138600 width=801)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                PTF Operator [PTF_76] (rows=138600 width=689)
+                PTF Operator [PTF_76] (rows=138600 width=690)
                   Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col1"}]
-                  Select Operator [SEL_75] (rows=138600 width=689)
+                  Select Operator [SEL_75] (rows=138600 width=690)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                   <-Reducer 4 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_74]
                       PartitionCols:_col1
-                      Group By Operator [GBY_73] (rows=138600 width=689)
+                      Group By Operator [GBY_73] (rows=138600 width=690)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
                       <-Reducer 3 [SIMPLE_EDGE]
                         SHUFFLE [RS_17]
                           PartitionCols:_col0, _col1, _col2, _col3, _col4
-                          Group By Operator [GBY_16] (rows=138600 width=689)
+                          Group By Operator [GBY_16] (rows=138600 width=690)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col2)"],keys:_col9, _col8, _col5, _col6, _col7
-                            Merge Join Operator [MERGEJOIN_58] (rows=4798568 width=689)
+                            Merge Join Operator [MERGEJOIN_58] (rows=43190061 width=689)
                               Conds:RS_12._col1=RS_72._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9"]
                             <-Map 9 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_72]
@@ -123,7 +123,7 @@ Stage-0
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_12]
                                 PartitionCols:_col1
-                                Merge Join Operator [MERGEJOIN_57] (rows=15995224 width=115)
+                                Merge Join Operator [MERGEJOIN_57] (rows=143966864 width=115)
                                   Conds:RS_69._col0=RS_61._col0(Inner),Output:["_col1","_col2"]
                                 <-Map 7 [SIMPLE_EDGE] vectorized
                                   PARTITION_ONLY_SHUFFLE [RS_61]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query13.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query13.q.out
index c6274c0..b23af5e 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query13.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query13.q.out
@@ -137,11 +137,11 @@ Stage-0
             PARTITION_ONLY_SHUFFLE [RS_31]
               Group By Operator [GBY_30] (rows=1 width=256)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","count(_col4)","sum(_col5)","count(_col5)","sum(_col6)","count(_col6)"]
-                Select Operator [SEL_29] (rows=368553 width=44)
+                Select Operator [SEL_29] (rows=11734953 width=215)
                   Output:["_col4","_col5","_col6"]
-                  Filter Operator [FIL_28] (rows=368553 width=44)
+                  Filter Operator [FIL_28] (rows=11734953 width=215)
                     predicate:((_col19 and _col20 and _col10 and _col26) or (_col21 and _col22 and _col11 and _col27) or (_col23 and _col24 and _col12 and _col27))
-                    Merge Join Operator [MERGEJOIN_97] (rows=1965626 width=44)
+                    Merge Join Operator [MERGEJOIN_97] (rows=62586416 width=215)
                       Conds:RS_25._col2=RS_117._col0(Inner),Output:["_col4","_col5","_col6","_col10","_col11","_col12","_col19","_col20","_col21","_col22","_col23","_col24","_col26","_col27"]
                     <-Map 11 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_117]
@@ -155,7 +155,7 @@ Stage-0
                     <-Reducer 4 [SIMPLE_EDGE]
                       SHUFFLE [RS_25]
                         PartitionCols:_col2
-                        Merge Join Operator [MERGEJOIN_96] (rows=10811694 width=36)
+                        Merge Join Operator [MERGEJOIN_96] (rows=62586416 width=210)
                           Conds:RS_22._col1=RS_114._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col10","_col11","_col12","_col19","_col20","_col21","_col22","_col23","_col24"]
                         <-Map 10 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_114]
@@ -169,9 +169,9 @@ Stage-0
                         <-Reducer 3 [SIMPLE_EDGE]
                           SHUFFLE [RS_22]
                             PartitionCols:_col1
-                            Filter Operator [FIL_21] (rows=10811694 width=36)
+                            Filter Operator [FIL_21] (rows=343170969 width=261)
                               predicate:((_col15 and _col7) or (_col16 and _col8) or (_col17 and _col9))
-                              Merge Join Operator [MERGEJOIN_95] (rows=14415593 width=36)
+                              Merge Join Operator [MERGEJOIN_95] (rows=457561292 width=261)
                                 Conds:RS_18._col3=RS_111._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17"]
                               <-Map 9 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_111]
@@ -185,7 +185,7 @@ Stage-0
                               <-Reducer 2 [SIMPLE_EDGE]
                                 SHUFFLE [RS_18]
                                   PartitionCols:_col3
-                                  Merge Join Operator [MERGEJOIN_94] (rows=163376714 width=233)
+                                  Merge Join Operator [MERGEJOIN_94] (rows=457561292 width=253)
                                     Conds:RS_108._col0=RS_100._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"]
                                   <-Map 7 [SIMPLE_EDGE] vectorized
                                     PARTITION_ONLY_SHUFFLE [RS_100]
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 6cfd12c..9b1cefe 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
@@ -297,44 +297,44 @@ Stage-0
     Stage-1
       Reducer 9 vectorized
       File Output Operator [FS_1361]
-        Limit [LIM_1360] (rows=7 width=212)
+        Limit [LIM_1360] (rows=100 width=223)
           Number of rows:100
-          Select Operator [SEL_1359] (rows=7 width=212)
+          Select Operator [SEL_1359] (rows=304320 width=223)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 8 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_1358]
-              Select Operator [SEL_1357] (rows=7 width=212)
+              Select Operator [SEL_1357] (rows=304320 width=223)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                Group By Operator [GBY_1356] (rows=7 width=220)
+                Group By Operator [GBY_1356] (rows=304320 width=231)
                   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_1204]
                       PartitionCols:_col0, _col1, _col2, _col3, _col4
-                      Group By Operator [GBY_1203] (rows=7 width=220)
+                      Group By Operator [GBY_1203] (rows=304320 width=231)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L
-                        Top N Key Operator [TNK_1202] (rows=3 width=221)
+                        Top N Key Operator [TNK_1202] (rows=121728 width=221)
                           keys:_col0, _col1, _col2, _col3, 0L,sort order:+++++,top n:100
-                          Select Operator [SEL_1200] (rows=1 width=223)
+                          Select Operator [SEL_1200] (rows=40576 width=223)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                            Filter Operator [FIL_1199] (rows=1 width=244)
+                            Filter Operator [FIL_1199] (rows=40576 width=244)
                               predicate:(_col3 > _col5)
-                              Merge Join Operator [MERGEJOIN_1198] (rows=1 width=244)
+                              Merge Join Operator [MERGEJOIN_1198] (rows=121728 width=244)
                                 Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                               <-Reducer 15 [CUSTOM_SIMPLE_EDGE] vectorized
                                 PARTITION_ONLY_SHUFFLE [RS_1373]
-                                  Filter Operator [FIL_1372] (rows=1 width=132)
+                                  Filter Operator [FIL_1372] (rows=121728 width=132)
                                     predicate:_col3 is not null
-                                    Group By Operator [GBY_1371] (rows=1 width=132)
+                                    Group By Operator [GBY_1371] (rows=121728 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)
+                                        Group By Operator [GBY_237] (rows=17407104 width=131)
                                           Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2
-                                          Select Operator [SEL_235] (rows=1 width=128)
+                                          Select Operator [SEL_235] (rows=286549727 width=127)
                                             Output:["_col0","_col1","_col2","_col3"]
-                                            Merge Join Operator [MERGEJOIN_1171] (rows=1 width=128)
+                                            Merge Join Operator [MERGEJOIN_1171] (rows=286549727 width=127)
                                               Conds:RS_232._col1=RS_1339._col0(Inner),Output:["_col2","_col3","_col7","_col8","_col9"]
                                             <-Map 65 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_1339]
@@ -346,12 +346,12 @@ Stage-0
                                             <-Reducer 13 [SIMPLE_EDGE]
                                               SHUFFLE [RS_232]
                                                 PartitionCols:_col1
-                                                Merge Join Operator [MERGEJOIN_1170] (rows=1 width=120)
+                                                Merge Join Operator [MERGEJOIN_1170] (rows=286549727 width=119)
                                                   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)
+                                                    Merge Join Operator [MERGEJOIN_1162] (rows=286549727 width=119)
                                                       Conds:RS_1366._col0=RS_1306._col0(Inner),Output:["_col1","_col2","_col3"]
                                                     <-Map 10 [SIMPLE_EDGE] vectorized
                                                       PARTITION_ONLY_SHUFFLE [RS_1306]
@@ -402,22 +402,22 @@ Stage-0
                                                           Output:["_col0","_col1","_col2"]
                                                           Filter Operator [FIL_1368] (rows=1 width=20)
                                                             predicate:(_col3 = 3L)
-                                                            Group By Operator [GBY_1367] (rows=120960 width=19)
+                                                            Group By Operator [GBY_1367] (rows=120960 width=20)
                                                               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=120960 width=19)
+                                                                  Group By Operator [GBY_1423] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1422] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1422] (rows=120960 width=20)
                                                                       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=120960 width=19)
+                                                                        Group By Operator [GBY_25] (rows=3144960 width=19)
                                                                           Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
-                                                                          Merge Join Operator [MERGEJOIN_1151] (rows=14628613 width=11)
+                                                                          Merge Join Operator [MERGEJOIN_1151] (rows=546042657 width=11)
                                                                             Conds:RS_21._col1=RS_1344._col0(Inner),Output:["_col4","_col5","_col6"]
                                                                           <-Map 65 [SIMPLE_EDGE] vectorized
                                                                             SHUFFLE [RS_1344]
@@ -430,7 +430,7 @@ Stage-0
                                                                           <-Reducer 25 [SIMPLE_EDGE]
                                                                             SHUFFLE [RS_21]
                                                                               PartitionCols:_col1
-                                                                              Merge Join Operator [MERGEJOIN_1150] (rows=14736682 width=4)
+                                                                              Merge Join Operator [MERGEJOIN_1150] (rows=550076554 width=4)
                                                                                 Conds:RS_1418._col0=RS_1396._col0(Inner),Output:["_col1"]
                                                                               <-Map 39 [SIMPLE_EDGE] vectorized
                                                                                 SHUFFLE [RS_1396]
@@ -464,16 +464,16 @@ Stage-0
                                                               <-Reducer 44 [CONTAINS] vectorized
                                                                 Reduce Output Operator [RS_1438]
                                                                   PartitionCols:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1437] (rows=120960 width=19)
+                                                                  Group By Operator [GBY_1437] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1436] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1436] (rows=120960 width=20)
                                                                       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=120960 width=19)
+                                                                        Group By Operator [GBY_45] (rows=1693440 width=19)
                                                                           Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
-                                                                          Merge Join Operator [MERGEJOIN_1153] (rows=7620440 width=11)
+                                                                          Merge Join Operator [MERGEJOIN_1153] (rows=284448361 width=11)
                                                                             Conds:RS_41._col1=RS_1345._col0(Inner),Output:["_col4","_col5","_col6"]
                                                                           <-Map 65 [SIMPLE_EDGE] vectorized
                                                                             SHUFFLE [RS_1345]
@@ -486,7 +486,7 @@ Stage-0
                                                                           <-Reducer 41 [SIMPLE_EDGE]
                                                                             SHUFFLE [RS_41]
                                                                               PartitionCols:_col1
-                                                                              Merge Join Operator [MERGEJOIN_1152] (rows=7676736 width=4)
+                                                                              Merge Join Operator [MERGEJOIN_1152] (rows=286549727 width=4)
                                                                                 Conds:RS_1432._col0=RS_1398._col0(Inner),Output:["_col1"]
                                                                               <-Map 39 [SIMPLE_EDGE] vectorized
                                                                                 SHUFFLE [RS_1398]
@@ -515,16 +515,16 @@ Stage-0
                                                               <-Reducer 50 [CONTAINS] vectorized
                                                                 Reduce Output Operator [RS_1452]
                                                                   PartitionCols:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1451] (rows=120960 width=19)
+                                                                  Group By Operator [GBY_1451] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1450] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1450] (rows=120960 width=20)
                                                                       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=120960 width=19)
+                                                                        Group By Operator [GBY_66] (rows=846720 width=19)
                                                                           Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6
-                                                                          Merge Join Operator [MERGEJOIN_1155] (rows=3828623 width=11)
+                                                                          Merge Join Operator [MERGEJOIN_1155] (rows=142911107 width=11)
                                                                             Conds:RS_62._col1=RS_1346._col0(Inner),Output:["_col4","_col5","_col6"]
                                                                           <-Map 65 [SIMPLE_EDGE] vectorized
                                                                             SHUFFLE [RS_1346]
@@ -537,7 +537,7 @@ Stage-0
                                                                           <-Reducer 47 [SIMPLE_EDGE]
                                                                             SHUFFLE [RS_62]
                                                                               PartitionCols:_col1
-                                                                              Merge Join Operator [MERGEJOIN_1154] (rows=3856907 width=4)
+                                                                              Merge Join Operator [MERGEJOIN_1154] (rows=143966864 width=4)
                                                                                 Conds:RS_1446._col0=RS_1400._col0(Inner),Output:["_col1"]
                                                                               <-Map 39 [SIMPLE_EDGE] vectorized
                                                                                 SHUFFLE [RS_1400]
@@ -576,11 +576,11 @@ Stage-0
                                           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)
+                                              Select Operator [SEL_1257] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1255] (rows=14736682 width=0)
+                                                Select Operator [SEL_1255] (rows=550076554 width=110)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1254] (rows=14736682 width=0)
+                                                  Merge Join Operator [MERGEJOIN_1254] (rows=550076554 width=110)
                                                     Conds:RS_1461._col0=RS_1404._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 39 [SIMPLE_EDGE] vectorized
                                                     SHUFFLE [RS_1404]
@@ -610,11 +610,11 @@ Stage-0
                                           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)
+                                              Select Operator [SEL_1275] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1273] (rows=7676736 width=94)
+                                                Select Operator [SEL_1273] (rows=286549727 width=115)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1272] (rows=7676736 width=94)
+                                                  Merge Join Operator [MERGEJOIN_1272] (rows=286549727 width=115)
                                                     Conds:RS_1476._col0=RS_1467._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 71 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_1467]
@@ -649,11 +649,11 @@ Stage-0
                                           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)
+                                              Select Operator [SEL_1293] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1291] (rows=3856907 width=114)
+                                                Select Operator [SEL_1291] (rows=143966864 width=115)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1290] (rows=3856907 width=114)
+                                                  Merge Join Operator [MERGEJOIN_1290] (rows=143966864 width=115)
                                                     Conds:RS_1491._col0=RS_1482._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 77 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_1482]
@@ -687,30 +687,30 @@ Stage-0
                   <-Reducer 22 [CONTAINS]
                     Reduce Output Operator [RS_1211]
                       PartitionCols:_col0, _col1, _col2, _col3, _col4
-                      Group By Operator [GBY_1210] (rows=7 width=220)
+                      Group By Operator [GBY_1210] (rows=304320 width=231)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L
-                        Top N Key Operator [TNK_1209] (rows=3 width=221)
+                        Top N Key Operator [TNK_1209] (rows=121728 width=221)
                           keys:_col0, _col1, _col2, _col3, 0L,sort order:+++++,top n:100
-                          Select Operator [SEL_1207] (rows=1 width=219)
+                          Select Operator [SEL_1207] (rows=40576 width=219)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                            Filter Operator [FIL_1206] (rows=1 width=244)
+                            Filter Operator [FIL_1206] (rows=40576 width=244)
                               predicate:(_col3 > _col5)
-                              Merge Join Operator [MERGEJOIN_1205] (rows=1 width=244)
+                              Merge Join Operator [MERGEJOIN_1205] (rows=121728 width=244)
                                 Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                               <-Reducer 21 [CUSTOM_SIMPLE_EDGE] vectorized
                                 PARTITION_ONLY_SHUFFLE [RS_1389]
-                                  Filter Operator [FIL_1388] (rows=1 width=132)
+                                  Filter Operator [FIL_1388] (rows=121728 width=132)
                                     predicate:_col3 is not null
-                                    Group By Operator [GBY_1387] (rows=1 width=132)
+                                    Group By Operator [GBY_1387] (rows=121728 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)
+                                        Group By Operator [GBY_381] (rows=8764416 width=131)
                                           Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2
-                                          Select Operator [SEL_379] (rows=1 width=128)
+                                          Select Operator [SEL_379] (rows=143966864 width=127)
                                             Output:["_col0","_col1","_col2","_col3"]
-                                            Merge Join Operator [MERGEJOIN_1184] (rows=1 width=128)
+                                            Merge Join Operator [MERGEJOIN_1184] (rows=143966864 width=127)
                                               Conds:RS_376._col1=RS_1341._col0(Inner),Output:["_col2","_col3","_col7","_col8","_col9"]
                                             <-Map 65 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_1341]
@@ -721,12 +721,12 @@ Stage-0
                                             <-Reducer 19 [SIMPLE_EDGE]
                                               SHUFFLE [RS_376]
                                                 PartitionCols:_col1
-                                                Merge Join Operator [MERGEJOIN_1183] (rows=1 width=120)
+                                                Merge Join Operator [MERGEJOIN_1183] (rows=143966864 width=119)
                                                   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)
+                                                    Merge Join Operator [MERGEJOIN_1175] (rows=143966864 width=119)
                                                       Conds:RS_1382._col0=RS_1308._col0(Inner),Output:["_col1","_col2","_col3"]
                                                     <-Map 10 [SIMPLE_EDGE] vectorized
                                                       PARTITION_ONLY_SHUFFLE [RS_1308]
@@ -772,15 +772,15 @@ Stage-0
                                                           Output:["_col0","_col1","_col2"]
                                                           Filter Operator [FIL_1384] (rows=1 width=20)
                                                             predicate:(_col3 = 3L)
-                                                            Group By Operator [GBY_1383] (rows=120960 width=19)
+                                                            Group By Operator [GBY_1383] (rows=120960 width=20)
                                                               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=120960 width=19)
+                                                                  Group By Operator [GBY_1426] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1425] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1425] (rows=120960 width=20)
                                                                       Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                                     <-Reducer 26 [SIMPLE_EDGE]
                                                                       SHUFFLE [RS_313]
@@ -789,9 +789,9 @@ Stage-0
                                                               <-Reducer 45 [CONTAINS] vectorized
                                                                 Reduce Output Operator [RS_1441]
                                                                   PartitionCols:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1440] (rows=120960 width=19)
+                                                                  Group By Operator [GBY_1440] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1439] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1439] (rows=120960 width=20)
                                                                       Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                                     <-Reducer 42 [SIMPLE_EDGE]
                                                                       SHUFFLE [RS_333]
@@ -800,9 +800,9 @@ Stage-0
                                                               <-Reducer 51 [CONTAINS] vectorized
                                                                 Reduce Output Operator [RS_1455]
                                                                   PartitionCols:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1454] (rows=120960 width=19)
+                                                                  Group By Operator [GBY_1454] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1453] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1453] (rows=120960 width=20)
                                                                       Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                                     <-Reducer 48 [SIMPLE_EDGE]
                                                                       SHUFFLE [RS_354]
@@ -821,11 +821,11 @@ Stage-0
                                           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)
+                                              Select Operator [SEL_1263] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1261] (rows=14736682 width=0)
+                                                Select Operator [SEL_1261] (rows=550076554 width=110)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1260] (rows=14736682 width=0)
+                                                  Merge Join Operator [MERGEJOIN_1260] (rows=550076554 width=110)
                                                     Conds:RS_1462._col0=RS_1405._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 39 [SIMPLE_EDGE] vectorized
                                                     SHUFFLE [RS_1405]
@@ -839,11 +839,11 @@ Stage-0
                                           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)
+                                              Select Operator [SEL_1281] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1279] (rows=7676736 width=94)
+                                                Select Operator [SEL_1279] (rows=286549727 width=115)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1278] (rows=7676736 width=94)
+                                                  Merge Join Operator [MERGEJOIN_1278] (rows=286549727 width=115)
                                                     Conds:RS_1477._col0=RS_1468._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 71 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_1468]
@@ -857,11 +857,11 @@ Stage-0
                                           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)
+                                              Select Operator [SEL_1299] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1297] (rows=3856907 width=114)
+                                                Select Operator [SEL_1297] (rows=143966864 width=115)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1296] (rows=3856907 width=114)
+                                                  Merge Join Operator [MERGEJOIN_1296] (rows=143966864 width=115)
                                                     Conds:RS_1492._col0=RS_1483._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 77 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_1483]
@@ -874,30 +874,30 @@ Stage-0
                   <-Reducer 6 [CONTAINS]
                     Reduce Output Operator [RS_1197]
                       PartitionCols:_col0, _col1, _col2, _col3, _col4
-                      Group By Operator [GBY_1196] (rows=7 width=220)
+                      Group By Operator [GBY_1196] (rows=304320 width=231)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L
-                        Top N Key Operator [TNK_1195] (rows=3 width=221)
+                        Top N Key Operator [TNK_1195] (rows=121728 width=221)
                           keys:_col0, _col1, _col2, _col3, 0L,sort order:+++++,top n:100
-                          Select Operator [SEL_1193] (rows=1 width=221)
+                          Select Operator [SEL_1193] (rows=40576 width=221)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                            Filter Operator [FIL_1192] (rows=1 width=244)
+                            Filter Operator [FIL_1192] (rows=40576 width=244)
                               predicate:(_col3 > _col5)
-                              Merge Join Operator [MERGEJOIN_1191] (rows=1 width=244)
+                              Merge Join Operator [MERGEJOIN_1191] (rows=121728 width=244)
                                 Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                               <-Reducer 5 [CUSTOM_SIMPLE_EDGE] vectorized
                                 PARTITION_ONLY_SHUFFLE [RS_1351]
-                                  Filter Operator [FIL_1350] (rows=1 width=132)
+                                  Filter Operator [FIL_1350] (rows=121728 width=132)
                                     predicate:_col3 is not null
-                                    Group By Operator [GBY_1349] (rows=1 width=132)
+                                    Group By Operator [GBY_1349] (rows=121728 width=132)
                                       Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2
                                     <-Reducer 4 [SIMPLE_EDGE]
                                       SHUFFLE [RS_95]
                                         PartitionCols:_col0, _col1, _col2
-                                        Group By Operator [GBY_94] (rows=1 width=132)
+                                        Group By Operator [GBY_94] (rows=32136192 width=131)
                                           Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2
-                                          Select Operator [SEL_92] (rows=1 width=128)
+                                          Select Operator [SEL_92] (rows=550076554 width=122)
                                             Output:["_col0","_col1","_col2","_col3"]
-                                            Merge Join Operator [MERGEJOIN_1158] (rows=1 width=128)
+                                            Merge Join Operator [MERGEJOIN_1158] (rows=550076554 width=122)
                                               Conds:RS_89._col1=RS_1334._col0(Inner),Output:["_col2","_col3","_col7","_col8","_col9"]
                                             <-Map 65 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_1334]
@@ -908,12 +908,12 @@ Stage-0
                                             <-Reducer 3 [SIMPLE_EDGE]
                                               SHUFFLE [RS_89]
                                                 PartitionCols:_col1
-                                                Merge Join Operator [MERGEJOIN_1157] (rows=1 width=120)
+                                                Merge Join Operator [MERGEJOIN_1157] (rows=550076554 width=114)
                                                   Conds:RS_86._col1=RS_87._col0(Inner),Output:["_col1","_col2","_col3"]
                                                 <-Reducer 2 [SIMPLE_EDGE]
                                                   SHUFFLE [RS_86]
                                                     PartitionCols:_col1
-                                                    Merge Join Operator [MERGEJOIN_1149] (rows=15062131 width=4)
+                                                    Merge Join Operator [MERGEJOIN_1149] (rows=550076554 width=114)
                                                       Conds:RS_1320._col0=RS_1304._col0(Inner),Output:["_col1","_col2","_col3"]
                                                     <-Map 10 [SIMPLE_EDGE] vectorized
                                                       PARTITION_ONLY_SHUFFLE [RS_1304]
@@ -959,15 +959,15 @@ Stage-0
                                                           Output:["_col0","_col1","_col2"]
                                                           Filter Operator [FIL_1322] (rows=1 width=20)
                                                             predicate:(_col3 = 3L)
-                                                            Group By Operator [GBY_1321] (rows=120960 width=19)
+                                                            Group By Operator [GBY_1321] (rows=120960 width=20)
                                                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                             <-Union 28 [SIMPLE_EDGE]
                                                               <-Reducer 27 [CONTAINS] vectorized
                                                                 Reduce Output Operator [RS_1421]
                                                                   PartitionCols:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1420] (rows=120960 width=19)
+                                                                  Group By Operator [GBY_1420] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1419] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1419] (rows=120960 width=20)
                                                                       Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                                     <-Reducer 26 [SIMPLE_EDGE]
                                                                       SHUFFLE [RS_26]
@@ -976,9 +976,9 @@ Stage-0
                                                               <-Reducer 43 [CONTAINS] vectorized
                                                                 Reduce Output Operator [RS_1435]
                                                                   PartitionCols:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1434] (rows=120960 width=19)
+                                                                  Group By Operator [GBY_1434] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1433] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1433] (rows=120960 width=20)
                                                                       Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                                     <-Reducer 42 [SIMPLE_EDGE]
                                                                       SHUFFLE [RS_46]
@@ -987,9 +987,9 @@ Stage-0
                                                               <-Reducer 49 [CONTAINS] vectorized
                                                                 Reduce Output Operator [RS_1449]
                                                                   PartitionCols:_col0, _col1, _col2
-                                                                  Group By Operator [GBY_1448] (rows=120960 width=19)
+                                                                  Group By Operator [GBY_1448] (rows=120960 width=20)
                                                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                                                                    Group By Operator [GBY_1447] (rows=120960 width=19)
+                                                                    Group By Operator [GBY_1447] (rows=120960 width=20)
                                                                       Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                                     <-Reducer 48 [SIMPLE_EDGE]
                                                                       SHUFFLE [RS_67]
@@ -1008,11 +1008,11 @@ Stage-0
                                           Reduce Output Operator [RS_1253]
                                             Group By Operator [GBY_1252] (rows=1 width=120)
                                               Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                              Select Operator [SEL_1251] (rows=26270325 width=44)
+                                              Select Operator [SEL_1251] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1249] (rows=14736682 width=0)
+                                                Select Operator [SEL_1249] (rows=550076554 width=110)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1248] (rows=14736682 width=0)
+                                                  Merge Join Operator [MERGEJOIN_1248] (rows=550076554 width=110)
                                                     Conds:RS_1460._col0=RS_1402._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 39 [SIMPLE_EDGE] vectorized
                                                     SHUFFLE [RS_1402]
@@ -1026,11 +1026,11 @@ Stage-0
                                           Reduce Output Operator [RS_1271]
                                             Group By Operator [GBY_1270] (rows=1 width=120)
                                               Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                              Select Operator [SEL_1269] (rows=26270325 width=44)
+                                              Select Operator [SEL_1269] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1267] (rows=7676736 width=94)
+                                                Select Operator [SEL_1267] (rows=286549727 width=115)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1266] (rows=7676736 width=94)
+                                                  Merge Join Operator [MERGEJOIN_1266] (rows=286549727 width=115)
                                                     Conds:RS_1475._col0=RS_1465._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 71 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_1465]
@@ -1044,11 +1044,11 @@ Stage-0
                                           Reduce Output Operator [RS_1289]
                                             Group By Operator [GBY_1288] (rows=1 width=120)
                                               Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"]
-                                              Select Operator [SEL_1287] (rows=26270325 width=44)
+                                              Select Operator [SEL_1287] (rows=980593145 width=112)
                                                 Output:["_col0"]
-                                                Select Operator [SEL_1285] (rows=3856907 width=114)
+                                                Select Operator [SEL_1285] (rows=143966864 width=115)
                                                   Output:["_col0","_col1"]
-                                                  Merge Join Operator [MERGEJOIN_1284] (rows=3856907 width=114)
+                                                  Merge Join Operator [MERGEJOIN_1284] (rows=143966864 width=115)
                                                     Conds:RS_1490._col0=RS_1480._col0(Inner),Output:["_col1","_col2"]
                                                   <-Map 77 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_1480]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query15.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query15.q.out
index 2ca3250..587fa14 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query15.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query15.q.out
@@ -65,24 +65,24 @@ Stage-0
       File Output Operator [FS_97]
         Limit [LIM_96] (rows=100 width=201)
           Number of rows:100
-          Select Operator [SEL_95] (rows=2555 width=201)
+          Select Operator [SEL_95] (rows=10141 width=201)
             Output:["_col0","_col1"]
           <-Reducer 4 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_94]
-              Group By Operator [GBY_93] (rows=2555 width=201)
+              Group By Operator [GBY_93] (rows=10141 width=201)
                 Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
               <-Reducer 3 [SIMPLE_EDGE]
                 SHUFFLE [RS_24]
                   PartitionCols:_col0
-                  Group By Operator [GBY_23] (rows=43435 width=201)
+                  Group By Operator [GBY_23] (rows=2403417 width=201)
                     Output:["_col0","_col1"],aggregations:["sum(_col8)"],keys:_col3
-                    Top N Key Operator [TNK_43] (rows=20154874 width=205)
+                    Top N Key Operator [TNK_43] (rows=285117831 width=212)
                       keys:_col3,sort order:+,top n:100
-                      Select Operator [SEL_22] (rows=20154874 width=205)
+                      Select Operator [SEL_22] (rows=285117831 width=212)
                         Output:["_col3","_col8"]
-                        Filter Operator [FIL_21] (rows=20154874 width=205)
+                        Filter Operator [FIL_21] (rows=285117831 width=212)
                           predicate:(_col9 or _col4 or _col5)
-                          Merge Join Operator [MERGEJOIN_76] (rows=20154874 width=205)
+                          Merge Join Operator [MERGEJOIN_76] (rows=285117831 width=212)
                             Conds:RS_18._col0=RS_19._col1(Inner),Output:["_col3","_col4","_col5","_col8","_col9"]
                           <-Reducer 2 [SIMPLE_EDGE]
                             SHUFFLE [RS_18]
@@ -108,7 +108,7 @@ Stage-0
                           <-Reducer 8 [SIMPLE_EDGE]
                             SHUFFLE [RS_19]
                               PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_75] (rows=20154874 width=111)
+                              Merge Join Operator [MERGEJOIN_75] (rows=285117831 width=119)
                                 Conds:RS_92._col0=RS_84._col0(Inner),Output:["_col1","_col2","_col3"]
                               <-Map 9 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_84]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query16.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query16.q.out
index f9b2dc1..cc3d388e 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query16.q.out
@@ -73,71 +73,80 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 12 (BROADCAST_EDGE)
-Map 14 <- Reducer 9 (BROADCAST_EDGE)
-Reducer 12 <- Map 11 (CUSTOM_SIMPLE_EDGE)
-Reducer 16 <- Map 15 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
-Reducer 3 <- Map 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 14 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-Reducer 6 <- Reducer 16 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Map 1 <- Reducer 11 (BROADCAST_EDGE)
+Reducer 11 <- Map 10 (CUSTOM_SIMPLE_EDGE)
+Reducer 15 <- Map 14 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 3 <- Map 10 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 13 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 15 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
-Reducer 9 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
       Reducer 8 vectorized
-      File Output Operator [FS_158]
-        Group By Operator [GBY_157] (rows=1 width=232)
+      File Output Operator [FS_156]
+        Group By Operator [GBY_155] (rows=1 width=232)
           Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
         <-Reducer 7 [CUSTOM_SIMPLE_EDGE] vectorized
-          PARTITION_ONLY_SHUFFLE [RS_156]
-            Group By Operator [GBY_155] (rows=1 width=232)
+          PARTITION_ONLY_SHUFFLE [RS_154]
+            Group By Operator [GBY_153] (rows=1 width=232)
               Output:["_col0","_col1","_col2"],aggregations:["count(_col0)","sum(_col1)","sum(_col2)"]
-              Group By Operator [GBY_154] (rows=5150256 width=228)
+              Group By Operator [GBY_152] (rows=283695062 width=228)
                 Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
               <-Reducer 6 [SIMPLE_EDGE]
                 SHUFFLE [RS_69]
                   PartitionCols:_col0
-                  Group By Operator [GBY_68] (rows=5150256 width=228)
+                  Group By Operator [GBY_68] (rows=283695062 width=228)
                     Output:["_col0","_col2","_col3"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col4
-                    Select Operator [SEL_41] (rows=5150256 width=214)
+                    Select Operator [SEL_41] (rows=283695062 width=228)
                       Output:["_col4","_col5","_col6"]
-                      Filter Operator [FIL_40] (rows=5150256 width=214)
+                      Filter Operator [FIL_40] (rows=283695062 width=228)
                         predicate:_col13 is null
-                        Merge Join Operator [MERGEJOIN_125] (rows=10300512 width=214)
-                          Conds:RS_37._col4=RS_153._col1(Left Outer),Output:["_col4","_col5","_col6","_col13"]
-                        <-Reducer 16 [SIMPLE_EDGE] vectorized
-                          SHUFFLE [RS_153]
+                        Merge Join Operator [MERGEJOIN_125] (rows=397917099 width=228)
+                          Conds:RS_37._col4=RS_151._col1(Left Outer),Output:["_col4","_col5","_col6","_col13"]
+                        <-Reducer 15 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_151]
                             PartitionCols:_col1
-                            Select Operator [SEL_152] (rows=18238808 width=8)
+                            Select Operator [SEL_150] (rows=18238808 width=8)
                               Output:["_col0","_col1"]
-                              Group By Operator [GBY_151] (rows=18238808 width=4)
+                              Group By Operator [GBY_149] (rows=18238808 width=4)
                                 Output:["_col0"],keys:KEY._col0
-                              <-Map 15 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_150]
+                              <-Map 14 [SIMPLE_EDGE] vectorized
+                                SHUFFLE [RS_148]
                                   PartitionCols:_col0
-                                  Group By Operator [GBY_149] (rows=28798881 width=4)
+                                  Group By Operator [GBY_147] (rows=28798881 width=4)
                                     Output:["_col0"],keys:cr_order_number
                                     TableScan [TS_25] (rows=28798881 width=4)
                                       default@catalog_returns,cr1,Tbl:COMPLETE,Col:COMPLETE,Output:["cr_order_number"]
                         <-Reducer 5 [SIMPLE_EDGE]
                           SHUFFLE [RS_37]
                             PartitionCols:_col4
-                            Select Operator [SEL_36] (rows=5150256 width=200)
+                            Select Operator [SEL_36] (rows=283695062 width=231)
                               Output:["_col4","_col5","_col6"]
-                              Merge Join Operator [MERGEJOIN_124] (rows=5150256 width=202)
-                                Conds:RS_33._col4=RS_148._col0(Left Semi),Output:["_col3","_col4","_col5","_col6","_col14"],residual filter predicates:{(_col3 <> _col14)}
+                              Merge Join Operator [MERGEJOIN_124] (rows=283695062 width=235)
+                                Conds:RS_33._col4=RS_146._col0(Left Semi),Output:["_col3","_col4","_col5","_col6","_col14"],residual filter predicates:{(_col3 <> _col14)}
+                              <-Map 13 [SIMPLE_EDGE] vectorized
+                                SHUFFLE [RS_146]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_145] (rows=286548719 width=7)
+                                    Output:["_col0","_col1"],keys:_col0, _col1
+                                    Select Operator [SEL_144] (rows=286548719 width=7)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_143] (rows=286548719 width=7)
+                                        predicate:cs_warehouse_sk is not null
+                                        TableScan [TS_22] (rows=287989836 width=7)
+                                          default@catalog_sales,cs2,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_warehouse_sk","cs_order_number"]
                               <-Reducer 4 [SIMPLE_EDGE]
-                                PARTITION_ONLY_SHUFFLE [RS_33]
+                                SHUFFLE [RS_33]
                                   PartitionCols:_col4
-                                  Merge Join Operator [MERGEJOIN_123] (rows=5150256 width=200)
+                                  Merge Join Operator [MERGEJOIN_123] (rows=283695062 width=231)
                                     Conds:RS_18._col2=RS_142._col0(Inner),Output:["_col3","_col4","_col5","_col6"]
-                                  <-Map 13 [SIMPLE_EDGE] vectorized
+                                  <-Map 12 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_142]
                                       PartitionCols:_col0
                                       Select Operator [SEL_141] (rows=10 width=102)
@@ -149,9 +158,9 @@ Stage-0
                                   <-Reducer 3 [SIMPLE_EDGE]
                                     SHUFFLE [RS_18]
                                       PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_122] (rows=30901534 width=230)
+                                      Merge Join Operator [MERGEJOIN_122] (rows=283695062 width=235)
                                         Conds:RS_15._col1=RS_128._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
-                                      <-Map 11 [SIMPLE_EDGE] vectorized
+                                      <-Map 10 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_128]
                                           PartitionCols:_col0
                                           Select Operator [SEL_127] (rows=784314 width=90)
@@ -163,7 +172,7 @@ Stage-0
                                       <-Reducer 2 [SIMPLE_EDGE]
                                         SHUFFLE [RS_15]
                                           PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_121] (rows=31519516 width=234)
+                                          Merge Join Operator [MERGEJOIN_121] (rows=283695062 width=239)
                                             Conds:RS_136._col0=RS_139._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
                                           <-Map 1 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_136]
@@ -174,18 +183,18 @@ Stage-0
                                                   predicate:(cs_ship_date_sk is not null and cs_call_center_sk is not null and cs_ship_addr_sk is not null and cs_ship_addr_sk BETWEEN DynamicValue(RS_16_customer_address_ca_address_sk_min) AND DynamicValue(RS_16_customer_address_ca_address_sk_max) and in_bloom_filter(cs_ship_addr_sk, DynamicValue(RS_16_customer_address_ca_address_sk_bloom_filter)))
                                                   TableScan [TS_0] (rows=287989836 width=243)
                                                     default@catalog_sales,cs1,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_ship_date_sk","cs_ship_addr_sk","cs_call_center_sk","cs_warehouse_sk","cs_order_number","cs_ext_ship_cost","cs_net_profit"]
-                                                  <-Reducer 12 [BROADCAST_EDGE] vectorized
+                                                  <-Reducer 11 [BROADCAST_EDGE] vectorized
                                                     BROADCAST [RS_133]
                                                       Group By Operator [GBY_132] (rows=1 width=12)
                                                         Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                      <-Map 11 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                      <-Map 10 [CUSTOM_SIMPLE_EDGE] vectorized
                                                         SHUFFLE [RS_131]
                                                           Group By Operator [GBY_130] (rows=1 width=12)
                                                             Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
                                                             Select Operator [SEL_129] (rows=784314 width=4)
                                                               Output:["_col0"]
                                                                Please refer to the previous Select Operator [SEL_127]
-                                          <-Map 10 [SIMPLE_EDGE] vectorized
+                                          <-Map 9 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_139]
                                               PartitionCols:_col0
                                               Select Operator [SEL_138] (rows=8116 width=98)
@@ -194,26 +203,4 @@ Stage-0
                                                   predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'2001-04-01 00:00:00' AND TIMESTAMP'2001-05-31 00:00:00'
                                                   TableScan [TS_3] (rows=73049 width=98)
                                                     default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_date"]
-                              <-Map 14 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_148]
-                                  PartitionCols:_col0
-                                  Group By Operator [GBY_147] (rows=286548719 width=7)
-                                    Output:["_col0","_col1"],keys:_col0, _col1
-                                    Select Operator [SEL_146] (rows=286548719 width=7)
-                                      Output:["_col0","_col1"]
-                                      Filter Operator [FIL_145] (rows=286548719 width=7)
-                                        predicate:(cs_warehouse_sk is not null and cs_order_number BETWEEN DynamicValue(RS_33_cs1_cs_order_number_min) AND DynamicValue(RS_33_cs1_cs_order_number_max) and in_bloom_filter(cs_order_number, DynamicValue(RS_33_cs1_cs_order_number_bloom_filter)))
-                                        TableScan [TS_22] (rows=287989836 width=7)
-                                          default@catalog_sales,cs2,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_warehouse_sk","cs_order_number"]
-                                        <-Reducer 9 [BROADCAST_EDGE] vectorized
-                                          BROADCAST [RS_144]
-                                            Group By Operator [GBY_143] (rows=1 width=12)
-                                              Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                            <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-                                              PARTITION_ONLY_SHUFFLE [RS_111]
-                                                Group By Operator [GBY_110] (rows=1 width=12)
-                                                  Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                  Select Operator [SEL_109] (rows=5150256 width=8)
-                                                    Output:["_col0"]
-                                                     Please refer to the previous Merge Join Operator [MERGEJOIN_123]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query17.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query17.q.out
index 07b6748..5c13e11 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query17.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query17.q.out
@@ -125,24 +125,24 @@ Stage-0
       File Output Operator [FS_248]
         Limit [LIM_247] (rows=100 width=466)
           Number of rows:100
-          Select Operator [SEL_246] (rows=8667503740 width=466)
+          Select Operator [SEL_246] (rows=97302218301 width=466)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_245]
-              Select Operator [SEL_244] (rows=8667503740 width=466)
+              Select Operator [SEL_244] (rows=97302218301 width=466)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13"]
-                Group By Operator [GBY_243] (rows=8667503740 width=466)
+                Group By Operator [GBY_243] (rows=97302218301 width=466)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","count(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","count(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)"],keys:KEY._col0, KEY._col1, KEY._col2
                 <-Reducer 5 [SIMPLE_EDGE]
                   SHUFFLE [RS_48]
                     PartitionCols:_col0, _col1, _col2
-                    Group By Operator [GBY_47] (rows=8667503740 width=466)
+                    Group By Operator [GBY_47] (rows=97302218301 width=466)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["count(_col3)","sum(_col3)","sum(_col7)","sum(_col6)","count(_col4)","sum(_col4)","sum(_col9)","sum(_col8)","count(_col5)","sum(_col5)","sum(_col11)","sum(_col10)"],keys:_col0, _col1, _col2
-                      Top N Key Operator [TNK_94] (rows=8667503740 width=381)
+                      Top N Key Operator [TNK_94] (rows=97302218301 width=381)
                         keys:_col0, _col1, _col2,sort order:+++,top n:100
-                        Select Operator [SEL_45] (rows=8667503740 width=381)
+                        Select Operator [SEL_45] (rows=97302218301 width=381)
                           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                          Merge Join Operator [MERGEJOIN_210] (rows=8667503740 width=381)
+                          Merge Join Operator [MERGEJOIN_210] (rows=97302218301 width=381)
                             Conds:RS_42._col6=RS_242._col0(Inner),Output:["_col3","_col10","_col16","_col19","_col21","_col22"]
                           <-Map 17 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_242]
@@ -154,7 +154,7 @@ Stage-0
                           <-Reducer 4 [SIMPLE_EDGE]
                             SHUFFLE [RS_42]
                               PartitionCols:_col6
-                              Merge Join Operator [MERGEJOIN_209] (rows=8667503740 width=101)
+                              Merge Join Operator [MERGEJOIN_209] (rows=97302218301 width=101)
                                 Conds:RS_39._col8=RS_240._col0(Inner),Output:["_col3","_col6","_col10","_col16","_col19"]
                               <-Map 16 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_240]
@@ -166,17 +166,17 @@ Stage-0
                               <-Reducer 3 [SIMPLE_EDGE]
                                 SHUFFLE [RS_39]
                                   PartitionCols:_col8
-                                  Merge Join Operator [MERGEJOIN_208] (rows=8667503740 width=19)
+                                  Merge Join Operator [MERGEJOIN_208] (rows=97302218301 width=19)
                                     Conds:RS_36._col1, _col2=RS_37._col9, _col8(Inner),Output:["_col3","_col6","_col8","_col10","_col16"]
                                   <-Reducer 11 [SIMPLE_EDGE]
                                     SHUFFLE [RS_37]
                                       PartitionCols:_col9, _col8
-                                      Merge Join Operator [MERGEJOIN_207] (rows=42605597 width=19)
+                                      Merge Join Operator [MERGEJOIN_207] (rows=478292911 width=23)
                                         Conds:RS_25._col2, _col1, _col4=RS_26._col2, _col1, _col3(Inner),Output:["_col1","_col3","_col5","_col8","_col9","_col11"]
                                       <-Reducer 10 [SIMPLE_EDGE]
                                         SHUFFLE [RS_25]
                                           PartitionCols:_col2, _col1, _col4
-                                          Merge Join Operator [MERGEJOIN_205] (rows=27749405 width=10)
+                                          Merge Join Operator [MERGEJOIN_205] (rows=501694138 width=19)
                                             Conds:RS_235._col0=RS_219._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
                                           <-Map 8 [SIMPLE_EDGE] vectorized
                                             PARTITION_ONLY_SHUFFLE [RS_219]
@@ -210,7 +210,7 @@ Stage-0
                                       <-Reducer 13 [SIMPLE_EDGE]
                                         SHUFFLE [RS_26]
                                           PartitionCols:_col2, _col1, _col3
-                                          Merge Join Operator [MERGEJOIN_206] (rows=8143830 width=14)
+                                          Merge Join Operator [MERGEJOIN_206] (rows=53632139 width=15)
                                             Conds:RS_238._col0=RS_221._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                           <-Map 8 [SIMPLE_EDGE] vectorized
                                             PARTITION_ONLY_SHUFFLE [RS_221]
@@ -232,7 +232,7 @@ Stage-0
                                   <-Reducer 2 [SIMPLE_EDGE]
                                     SHUFFLE [RS_36]
                                       PartitionCols:_col1, _col2
-                                      Merge Join Operator [MERGEJOIN_204] (rows=47131396 width=11)
+                                      Merge Join Operator [MERGEJOIN_204] (rows=285117831 width=11)
                                         Conds:RS_230._col0=RS_217._col0(Inner),Output:["_col1","_col2","_col3"]
                                       <-Map 8 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_217]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query18.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query18.q.out
index 133971d..c3c17a2 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query18.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query18.q.out
@@ -100,20 +100,20 @@ Stage-0
       File Output Operator [FS_174]
         Limit [LIM_173] (rows=100 width=1165)
           Number of rows:100
-          Select Operator [SEL_172] (rows=11292800 width=1165)
+          Select Operator [SEL_172] (rows=82276185 width=1165)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_171]
-              Select Operator [SEL_170] (rows=11292800 width=1165)
+              Select Operator [SEL_170] (rows=82276185 width=1165)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"]
-                Group By Operator [GBY_169] (rows=11292800 width=1229)
+                Group By Operator [GBY_169] (rows=82276185 width=1229)
                   Output:["_col0","_col1","_col2","_col3","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","sum(VALUE._col2)","count(VALUE._col3)","sum(VALUE._col4)","count(VALUE._col5)","sum(VALUE._col6)","count(VALUE._col7)","sum(VALUE._col8)","count(VALUE._col9)","sum(VALUE._col10)","count(VALUE._col11)","sum(VALUE._col12)","count(VALUE._col13)"],keys:KEY._col [...]
                 <-Reducer 5 [SIMPLE_EDGE]
                   SHUFFLE [RS_40]
                     PartitionCols:_col0, _col1, _col2, _col3, _col4
-                    Group By Operator [GBY_39] (rows=11292800 width=1229)
+                    Group By Operator [GBY_39] (rows=82276185 width=1229)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"],aggregations:["sum(_col12)","count(_col12)","sum(_col13)","count(_col13)","sum(_col14)","count(_col14)","sum(_col15)","count(_col15)","sum(_col16)","count(_col16)","sum(_col3)","count(_col3)","sum(_col19)","count(_col19)"],keys:_col21, _col5, _col6, _col7, 0L
-                      Merge Join Operator [MERGEJOIN_144] (rows=2258560 width=821)
+                      Merge Join Operator [MERGEJOIN_144] (rows=16455237 width=1117)
                         Conds:RS_35._col1=RS_168._col0(Inner),Output:["_col3","_col5","_col6","_col7","_col12","_col13","_col14","_col15","_col16","_col19","_col21"]
                       <-Map 16 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_168]
@@ -125,7 +125,7 @@ Stage-0
                       <-Reducer 4 [SIMPLE_EDGE]
                         SHUFFLE [RS_35]
                           PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_143] (rows=2227067 width=818)
+                          Merge Join Operator [MERGEJOIN_143] (rows=16225785 width=1120)
                             Conds:RS_32._col11=RS_166._col0(Inner),Output:["_col1","_col3","_col5","_col6","_col7","_col12","_col13","_col14","_col15","_col16","_col19","_col21"]
                           <-Map 15 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_166]
@@ -137,12 +137,12 @@ Stage-0
                           <-Reducer 3 [SIMPLE_EDGE]
                             SHUFFLE [RS_32]
                               PartitionCols:_col11
-                              Merge Join Operator [MERGEJOIN_142] (rows=2227067 width=722)
+                              Merge Join Operator [MERGEJOIN_142] (rows=16225785 width=1024)
                                 Conds:RS_29._col0=RS_30._col1(Inner),Output:["_col1","_col3","_col5","_col6","_col7","_col11","_col12","_col13","_col14","_col15","_col16","_col19"]
                               <-Reducer 11 [SIMPLE_EDGE]
                                 SHUFFLE [RS_30]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_141] (rows=16225774 width=640)
+                                  Merge Join Operator [MERGEJOIN_141] (rows=16225785 width=640)
                                     Conds:RS_18._col2=RS_164._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col11"]
                                   <-Map 14 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_164]
@@ -156,7 +156,7 @@ Stage-0
                                   <-Reducer 10 [SIMPLE_EDGE]
                                     SHUFFLE [RS_18]
                                       PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_140] (rows=100578970 width=565)
+                                      Merge Join Operator [MERGEJOIN_140] (rows=283692098 width=569)
                                         Conds:RS_161._col0=RS_153._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
                                       <-Map 12 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_153]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query19.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query19.q.out
index 7d89946..fff6c49 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query19.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query19.q.out
@@ -81,24 +81,24 @@ Stage-0
       File Output Operator [FS_148]
         Limit [LIM_147] (rows=100 width=419)
           Number of rows:100
-          Select Operator [SEL_146] (rows=2098703 width=418)
+          Select Operator [SEL_146] (rows=76645658 width=418)
             Output:["_col0","_col1","_col2","_col3","_col4"]
           <-Reducer 5 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_145]
-              Select Operator [SEL_144] (rows=2098703 width=418)
+              Select Operator [SEL_144] (rows=76645658 width=418)
                 Output:["_col2","_col3","_col4","_col5","_col6"]
-                Group By Operator [GBY_143] (rows=2098703 width=314)
+                Group By Operator [GBY_143] (rows=76645658 width=314)
                   Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
                 <-Reducer 4 [SIMPLE_EDGE]
                   SHUFFLE [RS_35]
                     PartitionCols:_col0, _col1, _col2, _col3
-                    Group By Operator [GBY_34] (rows=2098703 width=314)
+                    Group By Operator [GBY_34] (rows=76645658 width=314)
                       Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col8)"],keys:_col12, _col11, _col13, _col14
-                      Select Operator [SEL_33] (rows=2098703 width=570)
+                      Select Operator [SEL_33] (rows=76645658 width=650)
                         Output:["_col8","_col11","_col12","_col13","_col14"]
-                        Filter Operator [FIL_32] (rows=2098703 width=570)
+                        Filter Operator [FIL_32] (rows=76645658 width=650)
                           predicate:(_col3 <> _col16)
-                          Merge Join Operator [MERGEJOIN_121] (rows=2098703 width=570)
+                          Merge Join Operator [MERGEJOIN_121] (rows=76645658 width=650)
                             Conds:RS_29._col7=RS_142._col0(Inner),Output:["_col3","_col8","_col11","_col12","_col13","_col14","_col16"]
                           <-Map 14 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_142]
@@ -110,12 +110,12 @@ Stage-0
                           <-Reducer 3 [SIMPLE_EDGE]
                             SHUFFLE [RS_29]
                               PartitionCols:_col7
-                              Merge Join Operator [MERGEJOIN_120] (rows=2098703 width=386)
+                              Merge Join Operator [MERGEJOIN_120] (rows=76645658 width=468)
                                 Conds:RS_26._col0=RS_27._col2(Inner),Output:["_col3","_col7","_col8","_col11","_col12","_col13","_col14"]
                               <-Reducer 10 [SIMPLE_EDGE]
                                 SHUFFLE [RS_27]
                                   PartitionCols:_col2
-                                  Merge Join Operator [MERGEJOIN_119] (rows=2098703 width=202)
+                                  Merge Join Operator [MERGEJOIN_119] (rows=76645658 width=287)
                                     Conds:RS_17._col1=RS_140._col0(Inner),Output:["_col2","_col3","_col4","_col7","_col8","_col9","_col10"]
                                   <-Map 13 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_140]
@@ -129,7 +129,7 @@ Stage-0
                                   <-Reducer 9 [SIMPLE_EDGE]
                                     SHUFFLE [RS_17]
                                       PartitionCols:_col1
-                                      Merge Join Operator [MERGEJOIN_118] (rows=13737330 width=4)
+                                      Merge Join Operator [MERGEJOIN_118] (rows=501694138 width=118)
                                         Conds:RS_137._col0=RS_129._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                       <-Map 11 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_129]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query1b.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query1b.q.out
index bef5ef1..64bbe52 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query1b.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query1b.q.out
@@ -169,19 +169,19 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2, _col3
                         input vertices:
                           1 Map 9
-                        Statistics: Num rows: 16855704 Data size: 1805298496 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 51757026 Data size: 5993457136 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: sum(_col3)
                           keys: _col2 (type: int), _col1 (type: int)
-                          minReductionHashAggr: 0.8699312
+                          minReductionHashAggr: 0.777413
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 16855704 Data size: 2008197920 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 51757026 Data size: 6196356560 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col0 (type: int), _col1 (type: int)
                             sort order: ++
                             Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
-                            Statistics: Num rows: 16855704 Data size: 2008197920 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 51757026 Data size: 6196356560 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col2 (type: decimal(17,2))
                   Filter Operator
                     predicate: (sr_store_sk is not null and sr_returned_date_sk is not null and sr_store_sk BETWEEN DynamicValue(RS_40_store_s_store_sk_min) AND DynamicValue(RS_40_store_s_store_sk_max) and in_bloom_filter(sr_store_sk, DynamicValue(RS_40_store_s_store_sk_bloom_filter))) (type: boolean)
@@ -199,19 +199,19 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col2, _col3
                         input vertices:
                           1 Map 10
-                        Statistics: Num rows: 17467258 Data size: 1870797840 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 53634860 Data size: 6210910080 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
                           aggregations: sum(_col3)
                           keys: _col2 (type: int), _col1 (type: int)
-                          minReductionHashAggr: 0.85786486
+                          minReductionHashAggr: 0.75666153
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2
-                          Statistics: Num rows: 17467258 Data size: 2081058800 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 53634860 Data size: 6421171040 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col0 (type: int), _col1 (type: int)
                             sort order: ++
                             Map-reduce partition columns: _col0 (type: int)
-                            Statistics: Num rows: 17467258 Data size: 2081058800 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 53634860 Data size: 6421171040 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col2 (type: decimal(17,2))
             Execution mode: vectorized
         Map 9 
@@ -253,14 +253,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int), KEY._col1 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 11601100 Data size: 1382161488 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 51757026 Data size: 6196356560 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
                   predicate: _col2 is not null (type: boolean)
-                  Statistics: Num rows: 11601100 Data size: 1382161488 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 51757026 Data size: 6196356560 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col1 (type: int), _col0 (type: int), _col2 (type: decimal(17,2))
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 11601100 Data size: 1382161488 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 51757026 Data size: 6196356560 Basic stats: COMPLETE Column stats: COMPLETE
                     Map Join Operator
                       condition map:
                            Inner Join 0 to 1
@@ -270,7 +270,7 @@ STAGE PLANS:
                       outputColumnNames: _col1, _col2, _col3
                       input vertices:
                         0 Map 1
-                      Statistics: Num rows: 1923224 Data size: 220816368 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 51757026 Data size: 6196356560 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -280,21 +280,21 @@ STAGE PLANS:
                         outputColumnNames: _col1, _col3, _col4
                         input vertices:
                           1 Reducer 8
-                        Statistics: Num rows: 1991910 Data size: 449166736 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 52373181 Data size: 11933836920 Basic stats: COMPLETE Column stats: COMPLETE
                         Filter Operator
                           predicate: (_col3 > _col4) (type: boolean)
-                          Statistics: Num rows: 663970 Data size: 149722248 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 17457727 Data size: 3977945644 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col1 (type: int)
                             sort order: +
                             Map-reduce partition columns: _col1 (type: int)
-                            Statistics: Num rows: 663970 Data size: 149722248 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 17457727 Data size: 3977945644 Basic stats: COMPLETE Column stats: COMPLETE
                           Select Operator
                             expressions: _col1 (type: int)
                             outputColumnNames: _col0
-                            Statistics: Num rows: 663970 Data size: 1985936 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 17457727 Data size: 134829592 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=68687)
+                              aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1272219)
                               minReductionHashAggr: 0.99
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2
@@ -312,15 +312,15 @@ STAGE PLANS:
                   0 _col1 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col7
-                Statistics: Num rows: 663970 Data size: 66397000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 17457727 Data size: 1745772700 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col7 (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 663970 Data size: 66397000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 17457727 Data size: 1745772700 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     sort order: +
-                    Statistics: Num rows: 663970 Data size: 66397000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 17457727 Data size: 1745772700 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 0.1
         Reducer 6 
             Execution mode: vectorized
@@ -328,7 +328,7 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 663970 Data size: 66397000 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 17457727 Data size: 1745772700 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
                   Number of rows: 100
                   Statistics: Num rows: 100 Data size: 10000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -343,7 +343,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=68687)
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1272219)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
@@ -359,29 +359,29 @@ STAGE PLANS:
                 keys: KEY._col0 (type: int), KEY._col1 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 13369812 Data size: 1592886816 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 53634860 Data size: 6421171040 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col2 (type: decimal(17,2))
                   outputColumnNames: _col1, _col2
-                  Statistics: Num rows: 13369812 Data size: 1592886816 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 53634860 Data size: 6421171040 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: sum(_col2), count(_col2)
                     keys: _col1 (type: int)
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 29 Data size: 3588 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 85 Data size: 10532 Basic stats: COMPLETE Column stats: COMPLETE
                     Filter Operator
                       predicate: (_col1 is not null and _col2 is not null) (type: boolean)
-                      Statistics: Num rows: 29 Data size: 3588 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 85 Data size: 10532 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: ((_col1 / _col2) * 1.2) (type: decimal(38,11)), _col0 (type: int)
                         outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 29 Data size: 3356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 85 Data size: 9852 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
                           sort order: +
                           Map-reduce partition columns: _col1 (type: int)
-                          Statistics: Num rows: 29 Data size: 3356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 85 Data size: 9852 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: decimal(38,11))
 
   Stage: Stage-0
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query2.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query2.q.out
index 0e5ad6d..6677776 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query2.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query2.q.out
@@ -142,13 +142,13 @@ Stage-0
     Stage-1
       Reducer 7 vectorized
       File Output Operator [FS_173]
-        Select Operator [SEL_172] (rows=70850 width=788)
+        Select Operator [SEL_172] (rows=12881 width=788)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
         <-Reducer 6 [SIMPLE_EDGE]
           SHUFFLE [RS_57]
-            Select Operator [SEL_56] (rows=70850 width=788)
+            Select Operator [SEL_56] (rows=12881 width=788)
               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
-              Merge Join Operator [MERGEJOIN_146] (rows=70850 width=1572)
+              Merge Join Operator [MERGEJOIN_146] (rows=12881 width=1572)
                 Conds:RS_53._col0=RS_54.(_col0 - 53)(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col10","_col11","_col12","_col13","_col14","_col15","_col16"]
               <-Reducer 5 [SIMPLE_EDGE]
                 SHUFFLE [RS_53]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query20.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query20.q.out
index b383dd6..b71831c 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query20.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query20.q.out
@@ -87,21 +87,21 @@ Stage-0
             SHUFFLE [RS_78]
               Select Operator [SEL_77] (rows=138600 width=801)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                PTF Operator [PTF_76] (rows=138600 width=689)
+                PTF Operator [PTF_76] (rows=138600 width=690)
                   Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col1"}]
-                  Select Operator [SEL_75] (rows=138600 width=689)
+                  Select Operator [SEL_75] (rows=138600 width=690)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                   <-Reducer 4 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_74]
                       PartitionCols:_col1
-                      Group By Operator [GBY_73] (rows=138600 width=689)
+                      Group By Operator [GBY_73] (rows=138600 width=690)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
                       <-Reducer 3 [SIMPLE_EDGE]
                         SHUFFLE [RS_17]
                           PartitionCols:_col0, _col1, _col2, _col3, _col4
-                          Group By Operator [GBY_16] (rows=138600 width=689)
+                          Group By Operator [GBY_16] (rows=138600 width=690)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col2)"],keys:_col9, _col8, _col5, _col6, _col7
-                            Merge Join Operator [MERGEJOIN_58] (rows=9551005 width=673)
+                            Merge Join Operator [MERGEJOIN_58] (rows=85964922 width=688)
                               Conds:RS_12._col1=RS_72._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9"]
                             <-Map 9 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_72]
@@ -115,7 +115,7 @@ Stage-0
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_12]
                                 PartitionCols:_col1
-                                Merge Join Operator [MERGEJOIN_57] (rows=31836679 width=110)
+                                Merge Join Operator [MERGEJOIN_57] (rows=286549727 width=115)
                                   Conds:RS_69._col0=RS_61._col0(Inner),Output:["_col1","_col2"]
                                 <-Map 7 [SIMPLE_EDGE] vectorized
                                   PARTITION_ONLY_SHUFFLE [RS_61]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query23.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query23.q.out
index c8844c0..aced542 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query23.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query23.q.out
@@ -1,5 +1,4 @@
-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
+Warning: Shuffle Join MERGEJOIN[358][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 15' 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
@@ -119,272 +118,275 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 9 (BROADCAST_EDGE)
-Map 15 <- Reducer 7 (BROADCAST_EDGE)
-Map 19 <- Reducer 26 (BROADCAST_EDGE)
-Map 27 <- Reducer 33 (BROADCAST_EDGE)
-Map 35 <- Reducer 14 (BROADCAST_EDGE)
-Map 36 <- Reducer 13 (BROADCAST_EDGE)
-Reducer 10 <- Map 35 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
-Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
-Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 31 (SIMPLE_EDGE), Union 5 (CONTAINS)
-Reducer 13 <- Reducer 10 (CUSTOM_SIMPLE_EDGE)
-Reducer 14 <- Map 8 (CUSTOM_SIMPLE_EDGE)
-Reducer 16 <- Map 15 (SIMPLE_EDGE)
-Reducer 17 <- Reducer 16 (CUSTOM_SIMPLE_EDGE), Reducer 22 (CUSTOM_SIMPLE_EDGE)
-Reducer 18 <- Reducer 17 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
-Reducer 20 <- Map 19 (SIMPLE_EDGE), Map 25 (SIMPLE_EDGE)
-Reducer 21 <- Reducer 20 (SIMPLE_EDGE)
-Reducer 22 <- Reducer 21 (CUSTOM_SIMPLE_EDGE)
-Reducer 23 <- Reducer 22 (CUSTOM_SIMPLE_EDGE), Reducer 37 (CUSTOM_SIMPLE_EDGE)
-Reducer 24 <- Reducer 23 (SIMPLE_EDGE)
-Reducer 26 <- Map 25 (CUSTOM_SIMPLE_EDGE)
-Reducer 28 <- Map 27 (SIMPLE_EDGE), Map 32 (SIMPLE_EDGE)
-Reducer 29 <- Map 34 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 18 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 30 <- Reducer 29 (SIMPLE_EDGE)
-Reducer 31 <- Reducer 29 (SIMPLE_EDGE)
-Reducer 33 <- Map 32 (CUSTOM_SIMPLE_EDGE)
-Reducer 37 <- Map 36 (SIMPLE_EDGE)
-Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 30 (SIMPLE_EDGE), Union 5 (CONTAINS)
+Map 1 <- Reducer 8 (BROADCAST_EDGE)
+Map 17 <- Reducer 22 (BROADCAST_EDGE)
+Map 23 <- Reducer 29 (BROADCAST_EDGE)
+Map 31 <- Reducer 12 (BROADCAST_EDGE)
+Reducer 10 <- Reducer 16 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE), Union 5 (CONTAINS)
+Reducer 12 <- Map 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 14 <- Map 13 (SIMPLE_EDGE)
+Reducer 15 <- Reducer 14 (CUSTOM_SIMPLE_EDGE), Reducer 20 (CUSTOM_SIMPLE_EDGE)
+Reducer 16 <- Reducer 15 (SIMPLE_EDGE)
+Reducer 18 <- Map 17 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
+Reducer 19 <- Reducer 18 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 20 <- Reducer 19 (CUSTOM_SIMPLE_EDGE)
+Reducer 22 <- Map 21 (CUSTOM_SIMPLE_EDGE)
+Reducer 24 <- Map 23 (SIMPLE_EDGE), Map 28 (SIMPLE_EDGE)
+Reducer 25 <- Map 30 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
+Reducer 26 <- Reducer 25 (SIMPLE_EDGE)
+Reducer 27 <- Reducer 25 (SIMPLE_EDGE)
+Reducer 29 <- Map 28 (CUSTOM_SIMPLE_EDGE)
+Reducer 3 <- Reducer 16 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 26 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE), Union 5 (CONTAINS)
 Reducer 6 <- Union 5 (CUSTOM_SIMPLE_EDGE)
-Reducer 7 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
-Reducer 9 <- Map 8 (CUSTOM_SIMPLE_EDGE)
+Reducer 8 <- Map 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 9 <- Map 31 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
       Reducer 6 vectorized
-      File Output Operator [FS_440]
-        Group By Operator [GBY_439] (rows=1 width=112)
+      File Output Operator [FS_438]
+        Group By Operator [GBY_437] (rows=1 width=112)
           Output:["_col0"],aggregations:["sum(VALUE._col0)"]
         <-Union 5 [CUSTOM_SIMPLE_EDGE]
-          <-Reducer 12 [CONTAINS]
+          <-Reducer 11 [CONTAINS]
             Reduce Output Operator [RS_373]
               Group By Operator [GBY_372] (rows=1 width=112)
                 Output:["_col0"],aggregations:["sum(_col0)"]
-                Select Operator [SEL_370] (rows=155 width=112)
+                Select Operator [SEL_370] (rows=22920586 width=112)
                   Output:["_col0"]
-                  Merge Join Operator [MERGEJOIN_369] (rows=155 width=0)
-                    Conds:RS_152._col1=RS_462._col0(Left Semi),Output:["_col3","_col4"]
-                  <-Reducer 11 [SIMPLE_EDGE]
+                  Merge Join Operator [MERGEJOIN_369] (rows=22920586 width=115)
+                    Conds:RS_152._col1=RS_449._col0(Left Semi),Output:["_col3","_col4"]
+                  <-Reducer 10 [SIMPLE_EDGE]
                     SHUFFLE [RS_152]
                       PartitionCols:_col1
-                      Merge Join Operator [MERGEJOIN_361] (rows=3941102 width=118)
-                        Conds:RS_147._col2=RS_456._col0(Inner),Output:["_col1","_col3","_col4"]
-                      <-Reducer 10 [SIMPLE_EDGE]
-                        PARTITION_ONLY_SHUFFLE [RS_147]
-                          PartitionCols:_col2
-                          Merge Join Operator [MERGEJOIN_354] (rows=3941102 width=122)
-                            Conds:RS_445._col0=RS_378._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                          <-Map 8 [SIMPLE_EDGE] vectorized
-                            PARTITION_ONLY_SHUFFLE [RS_378]
-                              PartitionCols:_col0
-                              Select Operator [SEL_375] (rows=50 width=4)
-                                Output:["_col0"]
-                                Filter Operator [FIL_374] (rows=50 width=12)
-                                  predicate:((d_year = 1999) and (d_moy = 1))
-                                  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 35 [SIMPLE_EDGE] vectorized
-                            SHUFFLE [RS_445]
-                              PartitionCols:_col0
-                              Select Operator [SEL_444] (rows=143930993 width=127)
-                                Output:["_col0","_col1","_col2","_col3","_col4"]
-                                Filter Operator [FIL_443] (rows=143930993 width=127)
-                                  predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_sold_date_sk BETWEEN DynamicValue(RS_145_date_dim_d_date_sk_min) AND DynamicValue(RS_145_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_145_date_dim_d_date_sk_bloom_filter)))
-                                  TableScan [TS_78] (rows=144002668 width=127)
-                                    default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk","ws_bill_customer_sk","ws_quantity","ws_list_price"]
-                                  <-Reducer 14 [BROADCAST_EDGE] vectorized
-                                    BROADCAST [RS_442]
-                                      Group By Operator [GBY_441] (rows=1 width=12)
-                                        Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                      <-Map 8 [CUSTOM_SIMPLE_EDGE] vectorized
-                                        PARTITION_ONLY_SHUFFLE [RS_383]
-                                          Group By Operator [GBY_381] (rows=1 width=12)
-                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                            Select Operator [SEL_379] (rows=50 width=4)
-                                              Output:["_col0"]
-                                               Please refer to the previous Select Operator [SEL_375]
-                      <-Reducer 24 [SIMPLE_EDGE] vectorized
-                        SHUFFLE [RS_456]
+                      Merge Join Operator [MERGEJOIN_361] (rows=22920586 width=119)
+                        Conds:RS_147._col2=RS_417._col0(Inner),Output:["_col1","_col3","_col4"]
+                      <-Reducer 16 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_417]
                           PartitionCols:_col0
-                          Group By Operator [GBY_455] (rows=225322 width=3)
+                          Group By Operator [GBY_415] (rows=225322 width=3)
                             Output:["_col0"],keys:KEY._col0
-                          <-Reducer 23 [SIMPLE_EDGE]
-                            SHUFFLE [RS_120]
+                          <-Reducer 15 [SIMPLE_EDGE]
+                            SHUFFLE [RS_42]
                               PartitionCols:_col0
-                              Group By Operator [GBY_119] (rows=225322 width=3)
+                              Group By Operator [GBY_41] (rows=225322 width=3)
                                 Output:["_col0"],keys:_col0
-                                Select Operator [SEL_118] (rows=450644 width=227)
+                                Select Operator [SEL_40] (rows=450644 width=227)
                                   Output:["_col0"]
-                                  Filter Operator [FIL_117] (rows=450644 width=227)
+                                  Filter Operator [FIL_39] (rows=450644 width=227)
                                     predicate:(_col1 > _col2)
-                                    Merge Join Operator [MERGEJOIN_360] (rows=1351934 width=227)
+                                    Merge Join Operator [MERGEJOIN_358] (rows=1351934 width=227)
                                       Conds:(Inner),Output:["_col0","_col1","_col2"]
-                                    <-Reducer 22 [CUSTOM_SIMPLE_EDGE] vectorized
-                                      PARTITION_ONLY_SHUFFLE [RS_417]
-                                        Select Operator [SEL_415] (rows=1 width=112)
+                                    <-Reducer 14 [CUSTOM_SIMPLE_EDGE] vectorized
+                                      PARTITION_ONLY_SHUFFLE [RS_395]
+                                        Filter Operator [FIL_394] (rows=1351934 width=115)
+                                          predicate:_col1 is not null
+                                          Group By Operator [GBY_393] (rows=1351934 width=115)
+                                            Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
+                                          <-Map 13 [SIMPLE_EDGE] vectorized
+                                            SHUFFLE [RS_392]
+                                              PartitionCols:_col0
+                                              Group By Operator [GBY_391] (rows=550080312 width=115)
+                                                Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
+                                                Select Operator [SEL_390] (rows=550080312 width=114)
+                                                  Output:["_col0","_col1"]
+                                                  Filter Operator [FIL_389] (rows=550080312 width=114)
+                                                    predicate:ss_customer_sk is not null
+                                                    TableScan [TS_6] (rows=575995635 width=114)
+                                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_customer_sk","ss_quantity","ss_sales_price"]
+                                    <-Reducer 20 [CUSTOM_SIMPLE_EDGE] vectorized
+                                      PARTITION_ONLY_SHUFFLE [RS_414]
+                                        Select Operator [SEL_413] (rows=1 width=112)
                                           Output:["_col0"]
-                                          Filter Operator [FIL_414] (rows=1 width=112)
+                                          Filter Operator [FIL_412] (rows=1 width=112)
                                             predicate:_col0 is not null
-                                            Group By Operator [GBY_413] (rows=1 width=112)
+                                            Group By Operator [GBY_411] (rows=1 width=112)
                                               Output:["_col0"],aggregations:["max(VALUE._col0)"]
-                                            <-Reducer 21 [CUSTOM_SIMPLE_EDGE] vectorized
-                                              PARTITION_ONLY_SHUFFLE [RS_412]
-                                                Group By Operator [GBY_411] (rows=1 width=112)
+                                            <-Reducer 19 [CUSTOM_SIMPLE_EDGE] vectorized
+                                              PARTITION_ONLY_SHUFFLE [RS_410]
+                                                Group By Operator [GBY_409] (rows=1 width=112)
                                                   Output:["_col0"],aggregations:["max(_col1)"]
-                                                  Select Operator [SEL_410] (rows=46114 width=112)
+                                                  Select Operator [SEL_408] (rows=1291099 width=115)
                                                     Output:["_col1"]
-                                                    Group By Operator [GBY_409] (rows=46114 width=112)
+                                                    Group By Operator [GBY_407] (rows=1291099 width=115)
                                                       Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                                                    <-Reducer 20 [SIMPLE_EDGE]
+                                                    <-Reducer 18 [SIMPLE_EDGE]
                                                       SHUFFLE [RS_26]
                                                         PartitionCols:_col0
-                                                        Group By Operator [GBY_25] (rows=415026 width=112)
+                                                        Group By Operator [GBY_25] (rows=525327388 width=115)
                                                           Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col1
-                                                          Merge Join Operator [MERGEJOIN_351] (rows=18762463 width=112)
-                                                            Conds:RS_408._col0=RS_400._col0(Inner),Output:["_col1","_col2"]
-                                                          <-Map 25 [SIMPLE_EDGE] vectorized
-                                                            PARTITION_ONLY_SHUFFLE [RS_400]
+                                                          Merge Join Operator [MERGEJOIN_351] (rows=525327388 width=115)
+                                                            Conds:RS_406._col0=RS_398._col0(Inner),Output:["_col1","_col2"]
+                                                          <-Map 21 [SIMPLE_EDGE] vectorized
+                                                            PARTITION_ONLY_SHUFFLE [RS_398]
                                                               PartitionCols:_col0
-                                                              Select Operator [SEL_399] (rows=2609 width=4)
+                                                              Select Operator [SEL_397] (rows=2609 width=4)
                                                                 Output:["_col0"]
-                                                                Filter Operator [FIL_398] (rows=2609 width=8)
+                                                                Filter Operator [FIL_396] (rows=2609 width=8)
                                                                   predicate:(d_year) IN (1999, 2000, 2001, 2002)
                                                                   TableScan [TS_18] (rows=73049 width=8)
                                                                     default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
-                                                          <-Map 19 [SIMPLE_EDGE] vectorized
-                                                            SHUFFLE [RS_408]
+                                                          <-Map 17 [SIMPLE_EDGE] vectorized
+                                                            SHUFFLE [RS_406]
                                                               PartitionCols:_col0
-                                                              Select Operator [SEL_407] (rows=525327388 width=119)
+                                                              Select Operator [SEL_405] (rows=525327388 width=119)
                                                                 Output:["_col0","_col1","_col2"]
-                                                                Filter Operator [FIL_406] (rows=525327388 width=118)
+                                                                Filter Operator [FIL_404] (rows=525327388 width=118)
                                                                   predicate:(ss_sold_date_sk is not null and ss_customer_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_22_date_dim_d_date_sk_min) AND DynamicValue(RS_22_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_22_date_dim_d_date_sk_bloom_filter)))
                                                                   TableScan [TS_15] (rows=575995635 width=118)
                                                                     default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_customer_sk","ss_quantity","ss_sales_price"]
-                                                                  <-Reducer 26 [BROADCAST_EDGE] vectorized
-                                                                    BROADCAST [RS_405]
-                                                                      Group By Operator [GBY_404] (rows=1 width=12)
+                                                                  <-Reducer 22 [BROADCAST_EDGE] vectorized
+                                                                    BROADCAST [RS_403]
+                                                                      Group By Operator [GBY_402] (rows=1 width=12)
                                                                         Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                      <-Map 25 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                        PARTITION_ONLY_SHUFFLE [RS_403]
-                                                                          Group By Operator [GBY_402] (rows=1 width=12)
+                                                                      <-Map 21 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                        PARTITION_ONLY_SHUFFLE [RS_401]
+                                                                          Group By Operator [GBY_400] (rows=1 width=12)
                                                                             Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                            Select Operator [SEL_401] (rows=2609 width=4)
+                                                                            Select Operator [SEL_399] (rows=2609 width=4)
                                                                               Output:["_col0"]
-                                                                               Please refer to the previous Select Operator [SEL_399]
-                                    <-Reducer 37 [CUSTOM_SIMPLE_EDGE] vectorized
-                                      PARTITION_ONLY_SHUFFLE [RS_454]
-                                        Filter Operator [FIL_453] (rows=1351934 width=115)
-                                          predicate:_col1 is not null
-                                          Group By Operator [GBY_452] (rows=1351934 width=115)
-                                            Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                                          <-Map 36 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_451]
-                                              PartitionCols:_col0
-                                              Group By Operator [GBY_450] (rows=550080312 width=115)
-                                                Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
-                                                Select Operator [SEL_449] (rows=550080312 width=114)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_448] (rows=550080312 width=114)
-                                                    predicate:(ss_customer_sk is not null and ss_customer_sk BETWEEN DynamicValue(RS_147_web_sales_ws_bill_customer_sk_min) AND DynamicValue(RS_147_web_sales_ws_bill_customer_sk_max) and in_bloom_filter(ss_customer_sk, DynamicValue(RS_147_web_sales_ws_bill_customer_sk_bloom_filter)))
-                                                    TableScan [TS_84] (rows=575995635 width=114)
-                                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_customer_sk","ss_quantity","ss_sales_price"]
-                                                    <-Reducer 13 [BROADCAST_EDGE] vectorized
-                                                      BROADCAST [RS_447]
-                                                        Group By Operator [GBY_446] (rows=1 width=12)
-                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                        <-Reducer 10 [CUSTOM_SIMPLE_EDGE]
-                                                          PARTITION_ONLY_SHUFFLE [RS_319]
-                                                            Group By Operator [GBY_318] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                              Select Operator [SEL_317] (rows=3941102 width=7)
-                                                                Output:["_col0"]
-                                                                 Please refer to the previous Merge Join Operator [MERGEJOIN_354]
-                  <-Reducer 31 [SIMPLE_EDGE] vectorized
-                    SHUFFLE [RS_462]
+                                                                               Please refer to the previous Select Operator [SEL_397]
+                      <-Reducer 9 [SIMPLE_EDGE]
+                        SHUFFLE [RS_147]
+                          PartitionCols:_col2
+                          Merge Join Operator [MERGEJOIN_354] (rows=143930993 width=123)
+                            Conds:RS_443._col0=RS_378._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                          <-Map 7 [SIMPLE_EDGE] vectorized
+                            SHUFFLE [RS_378]
+                              PartitionCols:_col0
+                              Select Operator [SEL_375] (rows=50 width=4)
+                                Output:["_col0"]
+                                Filter Operator [FIL_374] (rows=50 width=12)
+                                  predicate:((d_year = 1999) and (d_moy = 1))
+                                  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 31 [SIMPLE_EDGE] vectorized
+                            SHUFFLE [RS_443]
+                              PartitionCols:_col0
+                              Select Operator [SEL_442] (rows=143930993 width=127)
+                                Output:["_col0","_col1","_col2","_col3","_col4"]
+                                Filter Operator [FIL_441] (rows=143930993 width=127)
+                                  predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_sold_date_sk BETWEEN DynamicValue(RS_145_date_dim_d_date_sk_min) AND DynamicValue(RS_145_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_145_date_dim_d_date_sk_bloom_filter)))
+                                  TableScan [TS_78] (rows=144002668 width=127)
+                                    default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk","ws_bill_customer_sk","ws_quantity","ws_list_price"]
+                                  <-Reducer 12 [BROADCAST_EDGE] vectorized
+                                    BROADCAST [RS_440]
+                                      Group By Operator [GBY_439] (rows=1 width=12)
+                                        Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                      <-Map 7 [CUSTOM_SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_383]
+                                          Group By Operator [GBY_381] (rows=1 width=12)
+                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                            Select Operator [SEL_379] (rows=50 width=4)
+                                              Output:["_col0"]
+                                               Please refer to the previous Select Operator [SEL_375]
+                  <-Reducer 27 [SIMPLE_EDGE] vectorized
+                    SHUFFLE [RS_449]
                       PartitionCols:_col0
-                      Group By Operator [GBY_461] (rows=2235 width=4)
+                      Group By Operator [GBY_448] (rows=62562 width=4)
                         Output:["_col0"],keys:_col0
-                        Select Operator [SEL_460] (rows=1943705 width=4)
+                        Select Operator [SEL_447] (rows=54408086 width=4)
                           Output:["_col0"]
-                          Filter Operator [FIL_459] (rows=1943705 width=106)
+                          Filter Operator [FIL_446] (rows=54408086 width=106)
                             predicate:(_col2 > 4L)
-                            Select Operator [SEL_458] (rows=5831115 width=106)
+                            Select Operator [SEL_445] (rows=163224258 width=106)
                               Output:["_col0","_col2"]
-                              Group By Operator [GBY_457] (rows=5831115 width=106)
+                              Group By Operator [GBY_444] (rows=163224258 width=106)
                                 Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
-                              <-Reducer 29 [SIMPLE_EDGE]
+                              <-Reducer 25 [SIMPLE_EDGE]
                                 SHUFFLE [RS_139]
                                   PartitionCols:_col0, _col1
-                                  Group By Operator [GBY_60] (rows=19646398 width=106)
+                                  Group By Operator [GBY_60] (rows=550076554 width=106)
                                     Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col4, _col3
-                                    Merge Join Operator [MERGEJOIN_353] (rows=19646398 width=98)
-                                      Conds:RS_56._col1=RS_432._col0(Inner),Output:["_col3","_col4"]
-                                    <-Map 34 [SIMPLE_EDGE] vectorized
-                                      SHUFFLE [RS_432]
+                                    Merge Join Operator [MERGEJOIN_353] (rows=550076554 width=98)
+                                      Conds:RS_56._col1=RS_430._col0(Inner),Output:["_col3","_col4"]
+                                    <-Map 30 [SIMPLE_EDGE] vectorized
+                                      SHUFFLE [RS_430]
                                         PartitionCols:_col0
-                                        Select Operator [SEL_431] (rows=462000 width=188)
+                                        Select Operator [SEL_429] (rows=462000 width=188)
                                           Output:["_col0"]
                                           TableScan [TS_51] (rows=462000 width=4)
                                             default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk"]
-                                    <-Reducer 28 [SIMPLE_EDGE]
+                                    <-Reducer 24 [SIMPLE_EDGE]
                                       SHUFFLE [RS_56]
                                         PartitionCols:_col1
-                                        Merge Join Operator [MERGEJOIN_352] (rows=19646398 width=98)
-                                          Conds:RS_430._col0=RS_422._col0(Inner),Output:["_col1","_col3"]
-                                        <-Map 32 [SIMPLE_EDGE] vectorized
-                                          PARTITION_ONLY_SHUFFLE [RS_422]
+                                        Merge Join Operator [MERGEJOIN_352] (rows=550076554 width=98)
+                                          Conds:RS_428._col0=RS_420._col0(Inner),Output:["_col1","_col3"]
+                                        <-Map 28 [SIMPLE_EDGE] vectorized
+                                          PARTITION_ONLY_SHUFFLE [RS_420]
                                             PartitionCols:_col0
-                                            Select Operator [SEL_421] (rows=2609 width=98)
+                                            Select Operator [SEL_419] (rows=2609 width=98)
                                               Output:["_col0","_col1"]
-                                              Filter Operator [FIL_420] (rows=2609 width=102)
+                                              Filter Operator [FIL_418] (rows=2609 width=102)
                                                 predicate:(d_year) IN (1999, 2000, 2001, 2002)
                                                 TableScan [TS_48] (rows=73049 width=102)
                                                   default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_date","d_year"]
-                                        <-Map 27 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_430]
+                                        <-Map 23 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_428]
                                             PartitionCols:_col0
-                                            Select Operator [SEL_429] (rows=550076554 width=7)
+                                            Select Operator [SEL_427] (rows=550076554 width=7)
                                               Output:["_col0","_col1"]
-                                              Filter Operator [FIL_428] (rows=550076554 width=7)
+                                              Filter Operator [FIL_426] (rows=550076554 width=7)
                                                 predicate:(ss_sold_date_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_54_date_dim_d_date_sk_min) AND DynamicValue(RS_54_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_54_date_dim_d_date_sk_bloom_filter)))
                                                 TableScan [TS_45] (rows=575995635 width=7)
                                                   default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk"]
-                                                <-Reducer 33 [BROADCAST_EDGE] vectorized
-                                                  BROADCAST [RS_427]
-                                                    Group By Operator [GBY_426] (rows=1 width=12)
+                                                <-Reducer 29 [BROADCAST_EDGE] vectorized
+                                                  BROADCAST [RS_425]
+                                                    Group By Operator [GBY_424] (rows=1 width=12)
                                                       Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                    <-Map 32 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                      PARTITION_ONLY_SHUFFLE [RS_425]
-                                                        Group By Operator [GBY_424] (rows=1 width=12)
+                                                    <-Map 28 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                      PARTITION_ONLY_SHUFFLE [RS_423]
+                                                        Group By Operator [GBY_422] (rows=1 width=12)
                                                           Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                          Select Operator [SEL_423] (rows=2609 width=4)
+                                                          Select Operator [SEL_421] (rows=2609 width=4)
                                                             Output:["_col0"]
-                                                             Please refer to the previous Select Operator [SEL_421]
+                                                             Please refer to the previous Select Operator [SEL_419]
           <-Reducer 4 [CONTAINS]
             Reduce Output Operator [RS_368]
               Group By Operator [GBY_367] (rows=1 width=112)
                 Output:["_col0"],aggregations:["sum(_col0)"]
-                Select Operator [SEL_365] (rows=304 width=112)
+                Select Operator [SEL_365] (rows=45838710 width=112)
                   Output:["_col0"]
-                  Merge Join Operator [MERGEJOIN_364] (rows=304 width=0)
-                    Conds:RS_74._col2=RS_438._col0(Left Semi),Output:["_col3","_col4"]
+                  Merge Join Operator [MERGEJOIN_364] (rows=45838710 width=112)
+                    Conds:RS_74._col2=RS_436._col0(Left Semi),Output:["_col3","_col4"]
+                  <-Reducer 26 [SIMPLE_EDGE] vectorized
+                    SHUFFLE [RS_436]
+                      PartitionCols:_col0
+                      Group By Operator [GBY_435] (rows=62562 width=4)
+                        Output:["_col0"],keys:_col0
+                        Select Operator [SEL_434] (rows=54408086 width=4)
+                          Output:["_col0"]
+                          Filter Operator [FIL_433] (rows=54408086 width=106)
+                            predicate:(_col2 > 4L)
+                            Select Operator [SEL_432] (rows=163224258 width=106)
+                              Output:["_col0","_col2"]
+                              Group By Operator [GBY_431] (rows=163224258 width=106)
+                                Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
+                              <-Reducer 25 [SIMPLE_EDGE]
+                                SHUFFLE [RS_61]
+                                  PartitionCols:_col0, _col1
+                                   Please refer to the previous Group By Operator [GBY_60]
                   <-Reducer 3 [SIMPLE_EDGE]
                     SHUFFLE [RS_74]
                       PartitionCols:_col2
-                      Merge Join Operator [MERGEJOIN_359] (rows=7751875 width=98)
-                        Conds:RS_69._col1=RS_419._col0(Inner),Output:["_col2","_col3","_col4"]
+                      Merge Join Operator [MERGEJOIN_359] (rows=45838710 width=116)
+                        Conds:RS_69._col1=RS_416._col0(Inner),Output:["_col2","_col3","_col4"]
+                      <-Reducer 16 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_416]
+                          PartitionCols:_col0
+                           Please refer to the previous Group By Operator [GBY_415]
                       <-Reducer 2 [SIMPLE_EDGE]
-                        PARTITION_ONLY_SHUFFLE [RS_69]
+                        SHUFFLE [RS_69]
                           PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_350] (rows=7751875 width=101)
+                          Merge Join Operator [MERGEJOIN_350] (rows=285117831 width=123)
                             Conds:RS_388._col0=RS_376._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                          <-Map 8 [SIMPLE_EDGE] vectorized
-                            PARTITION_ONLY_SHUFFLE [RS_376]
+                          <-Map 7 [SIMPLE_EDGE] vectorized
+                            SHUFFLE [RS_376]
                               PartitionCols:_col0
                                Please refer to the previous Select Operator [SEL_375]
                           <-Map 1 [SIMPLE_EDGE] vectorized
@@ -396,79 +398,15 @@ Stage-0
                                   predicate:(cs_sold_date_sk is not null and cs_bill_customer_sk is not null and cs_sold_date_sk BETWEEN DynamicValue(RS_67_date_dim_d_date_sk_min) AND DynamicValue(RS_67_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_67_date_dim_d_date_sk_bloom_filter)))
                                   TableScan [TS_0] (rows=287989836 width=127)
                                     default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_bill_customer_sk","cs_item_sk","cs_quantity","cs_list_price"]
-                                  <-Reducer 9 [BROADCAST_EDGE] vectorized
+                                  <-Reducer 8 [BROADCAST_EDGE] vectorized
                                     BROADCAST [RS_385]
                                       Group By Operator [GBY_384] (rows=1 width=12)
                                         Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                      <-Map 8 [CUSTOM_SIMPLE_EDGE] vectorized
-                                        PARTITION_ONLY_SHUFFLE [RS_382]
+                                      <-Map 7 [CUSTOM_SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_382]
                                           Group By Operator [GBY_380] (rows=1 width=12)
                                             Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
                                             Select Operator [SEL_377] (rows=50 width=4)
                                               Output:["_col0"]
                                                Please refer to the previous Select Operator [SEL_375]
-                      <-Reducer 18 [SIMPLE_EDGE] vectorized
-                        SHUFFLE [RS_419]
-                          PartitionCols:_col0
-                          Group By Operator [GBY_418] (rows=225322 width=3)
-                            Output:["_col0"],keys:KEY._col0
-                          <-Reducer 17 [SIMPLE_EDGE]
-                            SHUFFLE [RS_42]
-                              PartitionCols:_col0
-                              Group By Operator [GBY_41] (rows=225322 width=3)
-                                Output:["_col0"],keys:_col0
-                                Select Operator [SEL_40] (rows=450644 width=227)
-                                  Output:["_col0"]
-                                  Filter Operator [FIL_39] (rows=450644 width=227)
-                                    predicate:(_col1 > _col2)
-                                    Merge Join Operator [MERGEJOIN_358] (rows=1351934 width=227)
-                                      Conds:(Inner),Output:["_col0","_col1","_col2"]
-                                    <-Reducer 22 [CUSTOM_SIMPLE_EDGE] vectorized
-                                      PARTITION_ONLY_SHUFFLE [RS_416]
-                                         Please refer to the previous Select Operator [SEL_415]
-                                    <-Reducer 16 [CUSTOM_SIMPLE_EDGE] vectorized
-                                      PARTITION_ONLY_SHUFFLE [RS_397]
-                                        Filter Operator [FIL_396] (rows=1351934 width=115)
-                                          predicate:_col1 is not null
-                                          Group By Operator [GBY_395] (rows=1351934 width=115)
-                                            Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                                          <-Map 15 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_394]
-                                              PartitionCols:_col0
-                                              Group By Operator [GBY_393] (rows=550080312 width=115)
-                                                Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
-                                                Select Operator [SEL_392] (rows=550080312 width=114)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_391] (rows=550080312 width=114)
-                                                    predicate:(ss_customer_sk is not null and ss_customer_sk BETWEEN DynamicValue(RS_69_catalog_sales_cs_bill_customer_sk_min) AND DynamicValue(RS_69_catalog_sales_cs_bill_customer_sk_max) and in_bloom_filter(ss_customer_sk, DynamicValue(RS_69_catalog_sales_cs_bill_customer_sk_bloom_filter)))
-                                                    TableScan [TS_6] (rows=575995635 width=114)
-                                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_customer_sk","ss_quantity","ss_sales_price"]
-                                                    <-Reducer 7 [BROADCAST_EDGE] vectorized
-                                                      BROADCAST [RS_390]
-                                                        Group By Operator [GBY_389] (rows=1 width=12)
-                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                        <-Reducer 2 [CUSTOM_SIMPLE_EDGE]
-                                                          PARTITION_ONLY_SHUFFLE [RS_256]
-                                                            Group By Operator [GBY_255] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                              Select Operator [SEL_254] (rows=7751875 width=6)
-                                                                Output:["_col0"]
-                                                                 Please refer to the previous Merge Join Operator [MERGEJOIN_350]
-                  <-Reducer 30 [SIMPLE_EDGE] vectorized
-                    SHUFFLE [RS_438]
-                      PartitionCols:_col0
-                      Group By Operator [GBY_437] (rows=2235 width=4)
-                        Output:["_col0"],keys:_col0
-                        Select Operator [SEL_436] (rows=1943705 width=4)
-                          Output:["_col0"]
-                          Filter Operator [FIL_435] (rows=1943705 width=106)
-                            predicate:(_col2 > 4L)
-                            Select Operator [SEL_434] (rows=5831115 width=106)
-                              Output:["_col0","_col2"]
-                              Group By Operator [GBY_433] (rows=5831115 width=106)
-                                Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
-                              <-Reducer 29 [SIMPLE_EDGE]
-                                SHUFFLE [RS_61]
-                                  PartitionCols:_col0, _col1
-                                   Please refer to the previous Group By Operator [GBY_60]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query25.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query25.q.out
index ab95fb8..c2cee80 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query25.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query25.q.out
@@ -131,20 +131,20 @@ Stage-0
       File Output Operator [FS_246]
         Limit [LIM_245] (rows=100 width=808)
           Number of rows:100
-          Select Operator [SEL_244] (rows=4290851668 width=808)
+          Select Operator [SEL_244] (rows=97302218301 width=808)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_243]
-              Group By Operator [GBY_242] (rows=4290851668 width=808)
+              Group By Operator [GBY_242] (rows=97302218301 width=808)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
               <-Reducer 5 [SIMPLE_EDGE]
                 SHUFFLE [RS_47]
                   PartitionCols:_col0, _col1, _col2, _col3
-                  Group By Operator [GBY_46] (rows=4290851668 width=808)
+                  Group By Operator [GBY_46] (rows=97302218301 width=808)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col10)","sum(_col16)","sum(_col3)"],keys:_col22, _col23, _col19, _col20
-                    Top N Key Operator [TNK_93] (rows=4290851668 width=807)
+                    Top N Key Operator [TNK_93] (rows=97302218301 width=807)
                       keys:_col22, _col23, _col19, _col20,sort order:++++,top n:100
-                      Merge Join Operator [MERGEJOIN_209] (rows=4290851668 width=807)
+                      Merge Join Operator [MERGEJOIN_209] (rows=97302218301 width=807)
                         Conds:RS_42._col6=RS_241._col0(Inner),Output:["_col3","_col10","_col16","_col19","_col20","_col22","_col23"]
                       <-Map 17 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_241]
@@ -156,7 +156,7 @@ Stage-0
                       <-Reducer 4 [SIMPLE_EDGE]
                         SHUFFLE [RS_42]
                           PartitionCols:_col6
-                          Merge Join Operator [MERGEJOIN_208] (rows=4290851668 width=527)
+                          Merge Join Operator [MERGEJOIN_208] (rows=97302218301 width=527)
                             Conds:RS_39._col8=RS_239._col0(Inner),Output:["_col3","_col6","_col10","_col16","_col19","_col20"]
                           <-Map 16 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_239]
@@ -168,17 +168,17 @@ Stage-0
                           <-Reducer 3 [SIMPLE_EDGE]
                             SHUFFLE [RS_39]
                               PartitionCols:_col8
-                              Merge Join Operator [MERGEJOIN_207] (rows=4290851668 width=343)
+                              Merge Join Operator [MERGEJOIN_207] (rows=97302218301 width=343)
                                 Conds:RS_36._col1, _col2=RS_37._col9, _col8(Inner),Output:["_col3","_col6","_col8","_col10","_col16"]
                               <-Reducer 11 [SIMPLE_EDGE]
                                 SHUFFLE [RS_37]
                                   PartitionCols:_col9, _col8
-                                  Merge Join Operator [MERGEJOIN_206] (rows=21091882 width=154)
+                                  Merge Join Operator [MERGEJOIN_206] (rows=478292911 width=234)
                                     Conds:RS_25._col2, _col1, _col4=RS_26._col2, _col1, _col3(Inner),Output:["_col1","_col3","_col5","_col8","_col9","_col11"]
                                   <-Reducer 10 [SIMPLE_EDGE]
                                     SHUFFLE [RS_25]
                                       PartitionCols:_col2, _col1, _col4
-                                      Merge Join Operator [MERGEJOIN_204] (rows=13737330 width=8)
+                                      Merge Join Operator [MERGEJOIN_204] (rows=501694138 width=122)
                                         Conds:RS_234._col0=RS_218._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
                                       <-Map 8 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_218]
@@ -212,7 +212,7 @@ Stage-0
                                   <-Reducer 13 [SIMPLE_EDGE]
                                     SHUFFLE [RS_26]
                                       PartitionCols:_col2, _col1, _col3
-                                      Merge Join Operator [MERGEJOIN_205] (rows=9402909 width=100)
+                                      Merge Join Operator [MERGEJOIN_205] (rows=53632139 width=119)
                                         Conds:RS_237._col0=RS_220._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                       <-Map 8 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_220]
@@ -234,7 +234,7 @@ Stage-0
                               <-Reducer 2 [SIMPLE_EDGE]
                                 SHUFFLE [RS_36]
                                   PartitionCols:_col1, _col2
-                                  Merge Join Operator [MERGEJOIN_203] (rows=54418158 width=119)
+                                  Merge Join Operator [MERGEJOIN_203] (rows=285117831 width=119)
                                     Conds:RS_229._col0=RS_216._col0(Inner),Output:["_col1","_col2","_col3"]
                                   <-Map 8 [SIMPLE_EDGE] vectorized
                                     PARTITION_ONLY_SHUFFLE [RS_216]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query26.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query26.q.out
index 2b0ba1c..441ac49 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query26.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query26.q.out
@@ -83,9 +83,9 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_27] (rows=462000 width=476)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["sum(_col4)","count(_col4)","sum(_col5)","count(_col5)","sum(_col7)","count(_col7)","sum(_col6)","count(_col6)"],keys:_col12
-                      Top N Key Operator [TNK_54] (rows=821787 width=100)
+                      Top N Key Operator [TNK_54] (rows=2317924 width=231)
                         keys:_col12,sort order:+,top n:100
-                        Merge Join Operator [MERGEJOIN_98] (rows=821787 width=100)
+                        Merge Join Operator [MERGEJOIN_98] (rows=2317924 width=231)
                           Conds:RS_23._col2=RS_117._col0(Inner),Output:["_col4","_col5","_col6","_col7","_col12"]
                         <-Map 12 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_117]
@@ -97,7 +97,7 @@ Stage-0
                         <-Reducer 4 [SIMPLE_EDGE]
                           SHUFFLE [RS_23]
                             PartitionCols:_col2
-                            Merge Join Operator [MERGEJOIN_97] (rows=821787 width=4)
+                            Merge Join Operator [MERGEJOIN_97] (rows=2317924 width=135)
                               Conds:RS_20._col3=RS_115._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col7"]
                             <-Map 11 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_115]
@@ -111,7 +111,7 @@ Stage-0
                             <-Reducer 3 [SIMPLE_EDGE]
                               SHUFFLE [RS_20]
                                 PartitionCols:_col3
-                                Merge Join Operator [MERGEJOIN_96] (rows=821787 width=4)
+                                Merge Join Operator [MERGEJOIN_96] (rows=2317924 width=137)
                                   Conds:RS_17._col0=RS_112._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6","_col7"]
                                 <-Map 10 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_112]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query27.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query27.q.out
index 2a3711f..0b38f7f 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query27.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query27.q.out
@@ -74,24 +74,24 @@ Stage-0
       File Output Operator [FS_124]
         Limit [LIM_123] (rows=100 width=538)
           Number of rows:100
-          Select Operator [SEL_122] (rows=4916010 width=538)
+          Select Operator [SEL_122] (rows=6526254 width=538)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_121]
-              Select Operator [SEL_120] (rows=4916010 width=538)
+              Select Operator [SEL_120] (rows=6526254 width=538)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                Group By Operator [GBY_119] (rows=4916010 width=570)
+                Group By Operator [GBY_119] (rows=6526254 width=570)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","sum(VALUE._col2)","count(VALUE._col3)","sum(VALUE._col4)","count(VALUE._col5)","sum(VALUE._col6)","count(VALUE._col7)"],keys:KEY._col0, KEY._col1, KEY._col2
                 <-Reducer 5 [SIMPLE_EDGE]
                   SHUFFLE [RS_29]
                     PartitionCols:_col0, _col1, _col2
-                    Group By Operator [GBY_28] (rows=4916010 width=570)
+                    Group By Operator [GBY_28] (rows=13907934 width=570)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"],aggregations:["sum(_col2)","count(_col2)","sum(_col3)","count(_col3)","sum(_col4)","count(_col4)","sum(_col5)","count(_col5)"],keys:_col0, _col1, 0L
-                      Top N Key Operator [TNK_55] (rows=1638670 width=186)
+                      Top N Key Operator [TNK_55] (rows=4635978 width=186)
                         keys:_col0, _col1, 0L,sort order:+++,top n:100
-                        Select Operator [SEL_26] (rows=1638670 width=186)
+                        Select Operator [SEL_26] (rows=4635978 width=186)
                           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                          Merge Join Operator [MERGEJOIN_99] (rows=1638670 width=186)
+                          Merge Join Operator [MERGEJOIN_99] (rows=4635978 width=186)
                             Conds:RS_23._col1=RS_118._col0(Inner),Output:["_col4","_col5","_col6","_col7","_col11","_col13"]
                           <-Map 12 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_118]
@@ -103,7 +103,7 @@ Stage-0
                           <-Reducer 4 [SIMPLE_EDGE]
                             SHUFFLE [RS_23]
                               PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_98] (rows=1638670 width=90)
+                              Merge Join Operator [MERGEJOIN_98] (rows=4635978 width=90)
                                 Conds:RS_20._col3=RS_116._col0(Inner),Output:["_col1","_col4","_col5","_col6","_col7","_col11"]
                               <-Map 11 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_116]
@@ -117,7 +117,7 @@ Stage-0
                               <-Reducer 3 [SIMPLE_EDGE]
                                 SHUFFLE [RS_20]
                                   PartitionCols:_col3
-                                  Merge Join Operator [MERGEJOIN_97] (rows=1655322 width=4)
+                                  Merge Join Operator [MERGEJOIN_97] (rows=4635978 width=4)
                                     Conds:RS_17._col0=RS_113._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7"]
                                   <-Map 10 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_113]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query29.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query29.q.out
index 3e1c9d6..f53b01c 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query29.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query29.q.out
@@ -129,20 +129,20 @@ Stage-0
       File Output Operator [FS_246]
         Limit [LIM_245] (rows=100 width=496)
           Number of rows:100
-          Select Operator [SEL_244] (rows=4290826997 width=496)
+          Select Operator [SEL_244] (rows=97302218301 width=496)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_243]
-              Group By Operator [GBY_242] (rows=4290826997 width=496)
+              Group By Operator [GBY_242] (rows=97302218301 width=496)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
               <-Reducer 5 [SIMPLE_EDGE]
                 SHUFFLE [RS_47]
                   PartitionCols:_col0, _col1, _col2, _col3
-                  Group By Operator [GBY_46] (rows=4290826997 width=496)
+                  Group By Operator [GBY_46] (rows=97302218301 width=496)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col10)","sum(_col16)","sum(_col3)"],keys:_col22, _col23, _col19, _col20
-                    Top N Key Operator [TNK_93] (rows=4290826997 width=483)
+                    Top N Key Operator [TNK_93] (rows=97302218301 width=483)
                       keys:_col22, _col23, _col19, _col20,sort order:++++,top n:100
-                      Merge Join Operator [MERGEJOIN_209] (rows=4290826997 width=483)
+                      Merge Join Operator [MERGEJOIN_209] (rows=97302218301 width=483)
                         Conds:RS_42._col6=RS_241._col0(Inner),Output:["_col3","_col10","_col16","_col19","_col20","_col22","_col23"]
                       <-Map 18 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_241]
@@ -154,7 +154,7 @@ Stage-0
                       <-Reducer 4 [SIMPLE_EDGE]
                         SHUFFLE [RS_42]
                           PartitionCols:_col6
-                          Merge Join Operator [MERGEJOIN_208] (rows=4290826997 width=203)
+                          Merge Join Operator [MERGEJOIN_208] (rows=97302218301 width=203)
                             Conds:RS_39._col8=RS_239._col0(Inner),Output:["_col3","_col6","_col10","_col16","_col19","_col20"]
                           <-Map 17 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_239]
@@ -166,17 +166,17 @@ Stage-0
                           <-Reducer 3 [SIMPLE_EDGE]
                             SHUFFLE [RS_39]
                               PartitionCols:_col8
-                              Merge Join Operator [MERGEJOIN_207] (rows=4290826997 width=19)
+                              Merge Join Operator [MERGEJOIN_207] (rows=97302218301 width=19)
                                 Conds:RS_36._col1, _col2=RS_37._col9, _col8(Inner),Output:["_col3","_col6","_col8","_col10","_col16"]
                               <-Reducer 12 [SIMPLE_EDGE]
                                 SHUFFLE [RS_37]
                                   PartitionCols:_col9, _col8
-                                  Merge Join Operator [MERGEJOIN_206] (rows=21091879 width=18)
+                                  Merge Join Operator [MERGEJOIN_206] (rows=478292911 width=23)
                                     Conds:RS_25._col2, _col1, _col4=RS_26._col2, _col1, _col3(Inner),Output:["_col1","_col3","_col5","_col8","_col9","_col11"]
                                   <-Reducer 11 [SIMPLE_EDGE]
                                     SHUFFLE [RS_25]
                                       PartitionCols:_col2, _col1, _col4
-                                      Merge Join Operator [MERGEJOIN_204] (rows=13737330 width=8)
+                                      Merge Join Operator [MERGEJOIN_204] (rows=501694138 width=19)
                                         Conds:RS_234._col0=RS_225._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
                                       <-Map 13 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_225]
@@ -210,7 +210,7 @@ Stage-0
                                   <-Reducer 15 [SIMPLE_EDGE]
                                     SHUFFLE [RS_26]
                                       PartitionCols:_col2, _col1, _col3
-                                      Merge Join Operator [MERGEJOIN_205] (rows=5384572 width=13)
+                                      Merge Join Operator [MERGEJOIN_205] (rows=53632139 width=15)
                                         Conds:RS_237._col0=RS_227._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                       <-Map 13 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_227]
@@ -232,7 +232,7 @@ Stage-0
                               <-Reducer 2 [SIMPLE_EDGE]
                                 SHUFFLE [RS_36]
                                   PartitionCols:_col1, _col2
-                                  Merge Join Operator [MERGEJOIN_203] (rows=7638375 width=10)
+                                  Merge Join Operator [MERGEJOIN_203] (rows=285117831 width=11)
                                     Conds:RS_220._col0=RS_212._col0(Inner),Output:["_col1","_col2","_col3"]
                                   <-Map 8 [SIMPLE_EDGE] vectorized
                                     PARTITION_ONLY_SHUFFLE [RS_212]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query3.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query3.q.out
index 880f70d..4c1f168 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query3.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query3.q.out
@@ -64,18 +64,18 @@ Stage-0
       File Output Operator [FS_72]
         Limit [LIM_71] (rows=100 width=220)
           Number of rows:100
-          Select Operator [SEL_70] (rows=274400 width=220)
+          Select Operator [SEL_70] (rows=7666836 width=219)
             Output:["_col0","_col1","_col2","_col3"]
           <-Reducer 4 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_69]
-              Group By Operator [GBY_68] (rows=274400 width=220)
+              Group By Operator [GBY_68] (rows=7666836 width=219)
                 Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
               <-Reducer 3 [SIMPLE_EDGE]
                 SHUFFLE [RS_17]
                   PartitionCols:_col0, _col1, _col2
-                  Group By Operator [GBY_16] (rows=274400 width=220)
+                  Group By Operator [GBY_16] (rows=7666836 width=219)
                     Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col2)"],keys:_col7, _col4, _col5
-                    Merge Join Operator [MERGEJOIN_53] (rows=589741 width=108)
+                    Merge Join Operator [MERGEJOIN_53] (rows=7666836 width=108)
                       Conds:RS_12._col0=RS_67._col0(Inner),Output:["_col2","_col4","_col5","_col7"]
                     <-Map 8 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_67]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query30.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query30.q.out
index f3b3229..ea16228 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query30.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query30.q.out
@@ -88,15 +88,15 @@ Stage-0
     Stage-1
       Reducer 4 vectorized
       File Output Operator [FS_215]
-        Limit [LIM_214] (rows=100 width=942)
+        Limit [LIM_214] (rows=100 width=949)
           Number of rows:100
-          Select Operator [SEL_213] (rows=601923 width=942)
+          Select Operator [SEL_213] (rows=4179738 width=949)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"]
           <-Reducer 3 [SIMPLE_EDGE]
             SHUFFLE [RS_66]
-              Select Operator [SEL_65] (rows=601923 width=942)
+              Select Operator [SEL_65] (rows=4179738 width=949)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"]
-                Merge Join Operator [MERGEJOIN_180] (rows=601923 width=942)
+                Merge Join Operator [MERGEJOIN_180] (rows=4179738 width=949)
                   Conds:RS_62._col0=RS_63._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col17"]
                 <-Reducer 2 [SIMPLE_EDGE]
                   SHUFFLE [RS_62]
@@ -124,31 +124,31 @@ Stage-0
                 <-Reducer 8 [SIMPLE_EDGE]
                   SHUFFLE [RS_63]
                     PartitionCols:_col0
-                    Select Operator [SEL_58] (rows=613960 width=227)
+                    Select Operator [SEL_58] (rows=4179738 width=227)
                       Output:["_col0","_col2"]
-                      Filter Operator [FIL_57] (rows=613960 width=227)
+                      Filter Operator [FIL_57] (rows=4179738 width=227)
                         predicate:(_col2 > _col3)
-                        Merge Join Operator [MERGEJOIN_179] (rows=1841880 width=227)
+                        Merge Join Operator [MERGEJOIN_179] (rows=12539214 width=227)
                           Conds:RS_206._col1=RS_212._col1(Inner),Output:["_col0","_col2","_col3"]
                         <-Reducer 10 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_212]
                             PartitionCols:_col1
-                            Select Operator [SEL_211] (rows=6 width=198)
+                            Select Operator [SEL_211] (rows=17 width=198)
                               Output:["_col0","_col1"]
-                              Filter Operator [FIL_210] (rows=6 width=206)
+                              Filter Operator [FIL_210] (rows=17 width=206)
                                 predicate:(_col1 is not null and _col2 is not null)
-                                Group By Operator [GBY_209] (rows=6 width=206)
+                                Group By Operator [GBY_209] (rows=17 width=206)
                                   Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col0
-                                  Select Operator [SEL_208] (rows=2537976 width=201)
+                                  Select Operator [SEL_208] (rows=13130761 width=201)
                                     Output:["_col0","_col2"]
-                                    Group By Operator [GBY_207] (rows=2537976 width=201)
+                                    Group By Operator [GBY_207] (rows=13130761 width=201)
                                       Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                                     <-Reducer 9 [SIMPLE_EDGE]
                                       SHUFFLE [RS_45]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_44] (rows=3923529 width=201)
+                                        Group By Operator [GBY_44] (rows=13130761 width=201)
                                           Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col6, _col1
-                                          Merge Join Operator [MERGEJOIN_178] (rows=3923529 width=184)
+                                          Merge Join Operator [MERGEJOIN_178] (rows=13130761 width=196)
                                             Conds:RS_40._col2=RS_192._col0(Inner),Output:["_col1","_col3","_col6"]
                                           <-Map 5 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_192]
@@ -161,7 +161,7 @@ Stage-0
                                           <-Reducer 13 [SIMPLE_EDGE]
                                             SHUFFLE [RS_40]
                                               PartitionCols:_col2
-                                              Merge Join Operator [MERGEJOIN_177] (rows=3923529 width=101)
+                                              Merge Join Operator [MERGEJOIN_177] (rows=13130761 width=114)
                                                 Conds:RS_198._col0=RS_202._col0(Inner),Output:["_col1","_col2","_col3"]
                                               <-Map 11 [SIMPLE_EDGE] vectorized
                                                 SHUFFLE [RS_198]
@@ -184,18 +184,18 @@ Stage-0
                         <-Reducer 7 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_206]
                             PartitionCols:_col1
-                            Filter Operator [FIL_205] (rows=1841880 width=201)
+                            Filter Operator [FIL_205] (rows=12539214 width=201)
                               predicate:_col2 is not null
-                              Select Operator [SEL_204] (rows=1841880 width=201)
+                              Select Operator [SEL_204] (rows=12539214 width=201)
                                 Output:["_col0","_col1","_col2"]
-                                Group By Operator [GBY_203] (rows=1841880 width=201)
+                                Group By Operator [GBY_203] (rows=12539214 width=201)
                                   Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                                 <-Reducer 6 [SIMPLE_EDGE]
                                   SHUFFLE [RS_23]
                                     PartitionCols:_col0, _col1
-                                    Group By Operator [GBY_22] (rows=3746772 width=201)
+                                    Group By Operator [GBY_22] (rows=12539214 width=201)
                                       Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col6, _col1
-                                      Merge Join Operator [MERGEJOIN_176] (rows=3746772 width=184)
+                                      Merge Join Operator [MERGEJOIN_176] (rows=12539214 width=196)
                                         Conds:RS_18._col2=RS_191._col0(Inner),Output:["_col1","_col3","_col6"]
                                       <-Map 5 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_191]
@@ -208,7 +208,7 @@ Stage-0
                                       <-Reducer 12 [SIMPLE_EDGE]
                                         SHUFFLE [RS_18]
                                           PartitionCols:_col2
-                                          Merge Join Operator [MERGEJOIN_175] (rows=3746772 width=101)
+                                          Merge Join Operator [MERGEJOIN_175] (rows=12539214 width=114)
                                             Conds:RS_197._col0=RS_201._col0(Inner),Output:["_col1","_col2","_col3"]
                                           <-Map 11 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_197]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query31.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query31.q.out
index 6364d87..ed4cee4 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query31.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query31.q.out
@@ -155,33 +155,33 @@ Stage-0
     Stage-1
       Reducer 7
       File Output Operator [FS_139]
-        Select Operator [SEL_138] (rows=110 width=550)
+        Select Operator [SEL_138] (rows=429 width=550)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-          Filter Operator [FIL_136] (rows=110 width=778)
+          Filter Operator [FIL_136] (rows=429 width=778)
             predicate:(CASE WHEN ((_col9 > 0)) THEN (CASE WHEN (_col7) THEN (((_col4 / _col6) > (_col13 / _col9))) ELSE (false) END) ELSE (false) END and CASE WHEN ((_col11 > 0)) THEN (CASE WHEN (_col2) THEN (((_col6 / _col1) > (_col9 / _col11))) ELSE (false) END) ELSE (false) END)
-            Merge Join Operator [MERGEJOIN_450] (rows=440 width=778)
+            Merge Join Operator [MERGEJOIN_450] (rows=1716 width=778)
               Conds:RS_133._col0=RS_134._col0(Inner),Output:["_col1","_col2","_col4","_col6","_col7","_col8","_col9","_col11","_col13"]
             <-Reducer 22 [SIMPLE_EDGE]
               SHUFFLE [RS_134]
                 PartitionCols:_col0
-                Merge Join Operator [MERGEJOIN_448] (rows=1605 width=434)
+                Merge Join Operator [MERGEJOIN_448] (rows=1716 width=434)
                   Conds:RS_123._col0=RS_538._col0(Inner),Output:["_col0","_col1","_col3","_col5"]
                 <-Reducer 21 [SIMPLE_EDGE]
                   SHUFFLE [RS_123]
                     PartitionCols:_col0
-                    Merge Join Operator [MERGEJOIN_447] (rows=1605 width=322)
+                    Merge Join Operator [MERGEJOIN_447] (rows=1716 width=322)
                       Conds:RS_524._col0=RS_531._col0(Inner),Output:["_col0","_col1","_col3"]
                     <-Reducer 20 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_524]
                         PartitionCols:_col0
-                        Group By Operator [GBY_523] (rows=1605 width=210)
+                        Group By Operator [GBY_523] (rows=1716 width=210)
                           Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                         <-Reducer 19 [SIMPLE_EDGE]
                           SHUFFLE [RS_77]
                             PartitionCols:_col0
-                            Group By Operator [GBY_76] (rows=33705 width=210)
+                            Group By Operator [GBY_76] (rows=722436 width=210)
                               Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col5
-                              Merge Join Operator [MERGEJOIN_441] (rows=37399561 width=139)
+                              Merge Join Operator [MERGEJOIN_441] (rows=525327191 width=204)
                                 Conds:RS_72._col1=RS_497._col0(Inner),Output:["_col2","_col5"]
                               <-Map 32 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_497]
@@ -195,7 +195,7 @@ Stage-0
                               <-Reducer 18 [SIMPLE_EDGE]
                                 SHUFFLE [RS_72]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_440] (rows=37399561 width=42)
+                                  Merge Join Operator [MERGEJOIN_440] (rows=525327191 width=110)
                                     Conds:RS_522._col0=RS_469._col0(Inner),Output:["_col1","_col2"]
                                   <-Map 8 [SIMPLE_EDGE] vectorized
                                     PARTITION_ONLY_SHUFFLE [RS_469]
@@ -229,14 +229,14 @@ Stage-0
                     <-Reducer 26 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_531]
                         PartitionCols:_col0
-                        Group By Operator [GBY_530] (rows=1605 width=210)
+                        Group By Operator [GBY_530] (rows=1716 width=210)
                           Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                         <-Reducer 25 [SIMPLE_EDGE]
                           SHUFFLE [RS_97]
                             PartitionCols:_col0
-                            Group By Operator [GBY_96] (rows=33705 width=210)
+                            Group By Operator [GBY_96] (rows=722436 width=210)
                               Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col5
-                              Merge Join Operator [MERGEJOIN_443] (rows=37399561 width=139)
+                              Merge Join Operator [MERGEJOIN_443] (rows=525327191 width=204)
                                 Conds:RS_92._col1=RS_498._col0(Inner),Output:["_col2","_col5"]
                               <-Map 32 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_498]
@@ -245,7 +245,7 @@ Stage-0
                               <-Reducer 24 [SIMPLE_EDGE]
                                 SHUFFLE [RS_92]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_442] (rows=37399561 width=42)
+                                  Merge Join Operator [MERGEJOIN_442] (rows=525327191 width=110)
                                     Conds:RS_529._col0=RS_471._col0(Inner),Output:["_col1","_col2"]
                                   <-Map 8 [SIMPLE_EDGE] vectorized
                                     PARTITION_ONLY_SHUFFLE [RS_471]
@@ -278,14 +278,14 @@ Stage-0
                 <-Reducer 30 [SIMPLE_EDGE] vectorized
                   SHUFFLE [RS_538]
                     PartitionCols:_col0
-                    Group By Operator [GBY_537] (rows=1605 width=210)
+                    Group By Operator [GBY_537] (rows=1716 width=210)
                       Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                     <-Reducer 29 [SIMPLE_EDGE]
                       SHUFFLE [RS_117]
                         PartitionCols:_col0
-                        Group By Operator [GBY_116] (rows=33705 width=210)
+                        Group By Operator [GBY_116] (rows=722436 width=210)
                           Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col5
-                          Merge Join Operator [MERGEJOIN_445] (rows=37399561 width=139)
+                          Merge Join Operator [MERGEJOIN_445] (rows=525327191 width=204)
                             Conds:RS_112._col1=RS_499._col0(Inner),Output:["_col2","_col5"]
                           <-Map 32 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_499]
@@ -294,7 +294,7 @@ Stage-0
                           <-Reducer 28 [SIMPLE_EDGE]
                             SHUFFLE [RS_112]
                               PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_444] (rows=37399561 width=42)
+                              Merge Join Operator [MERGEJOIN_444] (rows=525327191 width=110)
                                 Conds:RS_536._col0=RS_473._col0(Inner),Output:["_col1","_col2"]
                               <-Map 8 [SIMPLE_EDGE] vectorized
                                 PARTITION_ONLY_SHUFFLE [RS_473]
@@ -327,21 +327,21 @@ Stage-0
             <-Reducer 6 [SIMPLE_EDGE]
               SHUFFLE [RS_133]
                 PartitionCols:_col0
-                Merge Join Operator [MERGEJOIN_449] (rows=440 width=442)
+                Merge Join Operator [MERGEJOIN_449] (rows=1716 width=442)
                   Conds:RS_130._col0=RS_517._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col6","_col7"]
                 <-Reducer 16 [SIMPLE_EDGE] vectorized
                   SHUFFLE [RS_517]
                     PartitionCols:_col0
-                    Select Operator [SEL_516] (rows=440 width=214)
+                    Select Operator [SEL_516] (rows=1716 width=214)
                       Output:["_col0","_col1","_col2"]
-                      Group By Operator [GBY_515] (rows=440 width=210)
+                      Group By Operator [GBY_515] (rows=1716 width=210)
                         Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                       <-Reducer 15 [SIMPLE_EDGE]
                         SHUFFLE [RS_57]
                           PartitionCols:_col0
-                          Group By Operator [GBY_56] (rows=3960 width=210)
+                          Group By Operator [GBY_56] (rows=204204 width=210)
                             Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col5
-                            Merge Join Operator [MERGEJOIN_439] (rows=10246882 width=209)
+                            Merge Join Operator [MERGEJOIN_439] (rows=143931246 width=209)
                               Conds:RS_52._col1=RS_496._col0(Inner),Output:["_col2","_col5"]
                             <-Map 32 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_496]
@@ -350,7 +350,7 @@ Stage-0
                             <-Reducer 14 [SIMPLE_EDGE]
                               SHUFFLE [RS_52]
                                 PartitionCols:_col1
-                                Merge Join Operator [MERGEJOIN_438] (rows=10246882 width=115)
+                                Merge Join Operator [MERGEJOIN_438] (rows=143931246 width=115)
                                   Conds:RS_514._col0=RS_467._col0(Inner),Output:["_col1","_col2"]
                                 <-Map 8 [SIMPLE_EDGE] vectorized
                                   PARTITION_ONLY_SHUFFLE [RS_467]
@@ -383,19 +383,19 @@ Stage-0
                 <-Reducer 5 [SIMPLE_EDGE]
                   SHUFFLE [RS_130]
                     PartitionCols:_col0
-                    Merge Join Operator [MERGEJOIN_446] (rows=440 width=326)
+                    Merge Join Operator [MERGEJOIN_446] (rows=1716 width=326)
                       Conds:RS_502._col0=RS_509._col0(Inner),Output:["_col0","_col1","_col2","_col4"]
                     <-Reducer 12 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_509]
                         PartitionCols:_col0
-                        Group By Operator [GBY_508] (rows=440 width=210)
+                        Group By Operator [GBY_508] (rows=1716 width=210)
                           Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                         <-Reducer 11 [SIMPLE_EDGE]
                           SHUFFLE [RS_37]
                             PartitionCols:_col0
-                            Group By Operator [GBY_36] (rows=3960 width=210)
+                            Group By Operator [GBY_36] (rows=204204 width=210)
                               Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col5
-                              Merge Join Operator [MERGEJOIN_437] (rows=10246882 width=209)
+                              Merge Join Operator [MERGEJOIN_437] (rows=143931246 width=209)
                                 Conds:RS_32._col1=RS_495._col0(Inner),Output:["_col2","_col5"]
                               <-Map 32 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_495]
@@ -404,7 +404,7 @@ Stage-0
                               <-Reducer 10 [SIMPLE_EDGE]
                                 SHUFFLE [RS_32]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_436] (rows=10246882 width=115)
+                                  Merge Join Operator [MERGEJOIN_436] (rows=143931246 width=115)
                                     Conds:RS_507._col0=RS_465._col0(Inner),Output:["_col1","_col2"]
                                   <-Map 8 [SIMPLE_EDGE] vectorized
                                     PARTITION_ONLY_SHUFFLE [RS_465]
@@ -437,16 +437,16 @@ Stage-0
                     <-Reducer 4 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_502]
                         PartitionCols:_col0
-                        Select Operator [SEL_501] (rows=440 width=214)
+                        Select Operator [SEL_501] (rows=1716 width=214)
                           Output:["_col0","_col1","_col2"]
-                          Group By Operator [GBY_500] (rows=440 width=210)
+                          Group By Operator [GBY_500] (rows=1716 width=210)
                             Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                           <-Reducer 3 [SIMPLE_EDGE]
                             SHUFFLE [RS_17]
                               PartitionCols:_col0
-                              Group By Operator [GBY_16] (rows=3960 width=210)
+                              Group By Operator [GBY_16] (rows=204204 width=210)
                                 Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col5
-                                Merge Join Operator [MERGEJOIN_435] (rows=10246882 width=209)
+                                Merge Join Operator [MERGEJOIN_435] (rows=143931246 width=209)
                                   Conds:RS_12._col1=RS_494._col0(Inner),Output:["_col2","_col5"]
                                 <-Map 32 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_494]
@@ -455,7 +455,7 @@ Stage-0
                                 <-Reducer 2 [SIMPLE_EDGE]
                                   SHUFFLE [RS_12]
                                     PartitionCols:_col1
-                                    Merge Join Operator [MERGEJOIN_434] (rows=10246882 width=115)
+                                    Merge Join Operator [MERGEJOIN_434] (rows=143931246 width=115)
                                       Conds:RS_491._col0=RS_463._col0(Inner),Output:["_col1","_col2"]
                                     <-Map 8 [SIMPLE_EDGE] vectorized
                                       PARTITION_ONLY_SHUFFLE [RS_463]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query32.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query32.q.out
index 9290053..c93fad2 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query32.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query32.q.out
@@ -84,16 +84,16 @@ Stage-0
           PARTITION_ONLY_SHUFFLE [RS_36]
             Group By Operator [GBY_35] (rows=1 width=112)
               Output:["_col0"],aggregations:["sum(_col2)"]
-              Select Operator [SEL_34] (rows=32 width=115)
+              Select Operator [SEL_34] (rows=20854 width=112)
                 Output:["_col2"]
-                Filter Operator [FIL_33] (rows=32 width=115)
+                Filter Operator [FIL_33] (rows=20854 width=112)
                   predicate:(_col2 > _col6)
-                  Merge Join Operator [MERGEJOIN_103] (rows=97 width=113)
+                  Merge Join Operator [MERGEJOIN_103] (rows=62562 width=112)
                     Conds:RS_30._col4=RS_125._col0(Inner),Output:["_col2","_col6"]
                   <-Reducer 3 [SIMPLE_EDGE]
                     SHUFFLE [RS_30]
                       PartitionCols:_col4
-                      Merge Join Operator [MERGEJOIN_101] (rows=441513 width=4)
+                      Merge Join Operator [MERGEJOIN_101] (rows=3973887 width=75)
                         Conds:RS_27._col1=RS_106._col0(Inner),Output:["_col2","_col4"]
                       <-Map 9 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_106]
@@ -107,7 +107,7 @@ Stage-0
                       <-Reducer 2 [SIMPLE_EDGE]
                         SHUFFLE [RS_27]
                           PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_100] (rows=31677454 width=110)
+                          Merge Join Operator [MERGEJOIN_100] (rows=285116600 width=115)
                             Conds:RS_116._col0=RS_120._col0(Inner),Output:["_col1","_col2"]
                           <-Map 1 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_116]
@@ -141,18 +141,18 @@ Stage-0
                   <-Reducer 7 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_125]
                       PartitionCols:_col0
-                      Select Operator [SEL_124] (rows=6951 width=116)
+                      Select Operator [SEL_124] (rows=62562 width=116)
                         Output:["_col0","_col1"]
-                        Filter Operator [FIL_123] (rows=6951 width=124)
+                        Filter Operator [FIL_123] (rows=62562 width=124)
                           predicate:CAST( (1.3 * (_col1 / _col2)) AS decimal(14,7)) is not null
-                          Group By Operator [GBY_122] (rows=6951 width=124)
+                          Group By Operator [GBY_122] (rows=62562 width=124)
                             Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
                           <-Reducer 6 [SIMPLE_EDGE]
                             SHUFFLE [RS_20]
                               PartitionCols:_col0
-                              Group By Operator [GBY_19] (rows=97314 width=124)
+                              Group By Operator [GBY_19] (rows=8133060 width=124)
                                 Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
-                                Merge Join Operator [MERGEJOIN_102] (rows=31836679 width=110)
+                                Merge Join Operator [MERGEJOIN_102] (rows=286549727 width=115)
                                   Conds:RS_117._col0=RS_121._col0(Inner),Output:["_col1","_col2"]
                                 <-Map 1 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_117]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query33.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query33.q.out
index 67a0cea..f00567e 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query33.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query33.q.out
@@ -192,28 +192,28 @@ Stage-0
     Stage-1
       Reducer 7 vectorized
       File Output Operator [FS_357]
-        Limit [LIM_356] (rows=2 width=114)
+        Limit [LIM_356] (rows=7 width=116)
           Number of rows:100
-          Select Operator [SEL_355] (rows=2 width=114)
+          Select Operator [SEL_355] (rows=7 width=116)
             Output:["_col0","_col1"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_354]
-              Group By Operator [GBY_353] (rows=2 width=114)
+              Group By Operator [GBY_353] (rows=7 width=116)
                 Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
               <-Union 5 [SIMPLE_EDGE]
                 <-Reducer 11 [CONTAINS] vectorized
                   Reduce Output Operator [RS_373]
                     PartitionCols:_col0
-                    Group By Operator [GBY_372] (rows=2 width=114)
+                    Group By Operator [GBY_372] (rows=7 width=116)
                       Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
-                      Group By Operator [GBY_371] (rows=2 width=114)
+                      Group By Operator [GBY_371] (rows=7 width=116)
                         Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                       <-Reducer 10 [SIMPLE_EDGE]
                         SHUFFLE [RS_109]
                           PartitionCols:_col0
-                          Group By Operator [GBY_108] (rows=2 width=114)
+                          Group By Operator [GBY_108] (rows=8 width=116)
                             Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                            Merge Join Operator [MERGEJOIN_304] (rows=1134 width=0)
+                            Merge Join Operator [MERGEJOIN_304] (rows=206906 width=96)
                               Conds:RS_104._col0=RS_105._col2(Inner),Output:["_col1","_col7"]
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_104]
@@ -248,9 +248,9 @@ Stage-0
                             <-Reducer 23 [SIMPLE_EDGE]
                               SHUFFLE [RS_105]
                                 PartitionCols:_col2
-                                Select Operator [SEL_100] (rows=788222 width=110)
+                                Select Operator [SEL_100] (rows=143931246 width=115)
                                   Output:["_col2","_col4"]
-                                  Merge Join Operator [MERGEJOIN_301] (rows=788222 width=110)
+                                  Merge Join Operator [MERGEJOIN_301] (rows=143931246 width=115)
                                     Conds:RS_97._col2=RS_349._col0(Inner),Output:["_col1","_col3"]
                                   <-Map 25 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_349]
@@ -264,7 +264,7 @@ Stage-0
                                   <-Reducer 22 [SIMPLE_EDGE]
                                     SHUFFLE [RS_97]
                                       PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_300] (rows=3941109 width=118)
+                                      Merge Join Operator [MERGEJOIN_300] (rows=143931246 width=119)
                                         Conds:RS_370._col0=RS_332._col0(Inner),Output:["_col1","_col2","_col3"]
                                       <-Map 17 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_332]
@@ -298,16 +298,16 @@ Stage-0
                 <-Reducer 4 [CONTAINS] vectorized
                   Reduce Output Operator [RS_352]
                     PartitionCols:_col0
-                    Group By Operator [GBY_351] (rows=2 width=114)
+                    Group By Operator [GBY_351] (rows=7 width=116)
                       Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
-                      Group By Operator [GBY_350] (rows=2 width=116)
+                      Group By Operator [GBY_350] (rows=7 width=116)
                         Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                       <-Reducer 3 [SIMPLE_EDGE]
                         SHUFFLE [RS_34]
                           PartitionCols:_col0
-                          Group By Operator [GBY_33] (rows=2 width=116)
+                          Group By Operator [GBY_33] (rows=8 width=116)
                             Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                            Merge Join Operator [MERGEJOIN_302] (rows=4136 width=2)
+                            Merge Join Operator [MERGEJOIN_302] (rows=755172 width=3)
                               Conds:RS_29._col0=RS_30._col2(Inner),Output:["_col1","_col7"]
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_29]
@@ -316,9 +316,9 @@ Stage-0
                             <-Reducer 16 [SIMPLE_EDGE]
                               SHUFFLE [RS_30]
                                 PartitionCols:_col2
-                                Select Operator [SEL_25] (rows=2876890 width=4)
+                                Select Operator [SEL_25] (rows=525327191 width=110)
                                   Output:["_col2","_col4"]
-                                  Merge Join Operator [MERGEJOIN_295] (rows=2876890 width=4)
+                                  Merge Join Operator [MERGEJOIN_295] (rows=525327191 width=110)
                                     Conds:RS_22._col2=RS_347._col0(Inner),Output:["_col1","_col3"]
                                   <-Map 25 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_347]
@@ -327,7 +327,7 @@ Stage-0
                                   <-Reducer 15 [SIMPLE_EDGE]
                                     SHUFFLE [RS_22]
                                       PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_294] (rows=14384447 width=4)
+                                      Merge Join Operator [MERGEJOIN_294] (rows=525327191 width=114)
                                         Conds:RS_344._col0=RS_328._col0(Inner),Output:["_col1","_col2","_col3"]
                                       <-Map 17 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_328]
@@ -356,16 +356,16 @@ Stage-0
                 <-Reducer 9 [CONTAINS] vectorized
                   Reduce Output Operator [RS_365]
                     PartitionCols:_col0
-                    Group By Operator [GBY_364] (rows=2 width=114)
+                    Group By Operator [GBY_364] (rows=7 width=116)
                       Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
-                      Group By Operator [GBY_363] (rows=2 width=116)
+                      Group By Operator [GBY_363] (rows=7 width=116)
                         Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                       <-Reducer 8 [SIMPLE_EDGE]
                         SHUFFLE [RS_71]
                           PartitionCols:_col0
-                          Group By Operator [GBY_70] (rows=2 width=116)
+                          Group By Operator [GBY_70] (rows=8 width=116)
                             Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                            Merge Join Operator [MERGEJOIN_303] (rows=2229 width=1)
+                            Merge Join Operator [MERGEJOIN_303] (rows=409865 width=3)
                               Conds:RS_66._col0=RS_67._col3(Inner),Output:["_col1","_col7"]
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_66]
@@ -374,9 +374,9 @@ Stage-0
                             <-Reducer 20 [SIMPLE_EDGE]
                               SHUFFLE [RS_67]
                                 PartitionCols:_col3
-                                Select Operator [SEL_62] (rows=1550375 width=13)
+                                Select Operator [SEL_62] (rows=285117733 width=115)
                                   Output:["_col3","_col4"]
-                                  Merge Join Operator [MERGEJOIN_298] (rows=1550375 width=13)
+                                  Merge Join Operator [MERGEJOIN_298] (rows=285117733 width=115)
                                     Conds:RS_59._col1=RS_348._col0(Inner),Output:["_col2","_col3"]
                                   <-Map 25 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_348]
@@ -385,7 +385,7 @@ Stage-0
                                   <-Reducer 19 [SIMPLE_EDGE]
                                     SHUFFLE [RS_59]
                                       PartitionCols:_col1
-                                      Merge Join Operator [MERGEJOIN_297] (rows=7751872 width=98)
+                                      Merge Join Operator [MERGEJOIN_297] (rows=285117733 width=119)
                                         Conds:RS_362._col0=RS_330._col0(Inner),Output:["_col1","_col2","_col3"]
                                       <-Map 17 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_330]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query34.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query34.q.out
index f43a27a..9dab813 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query34.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query34.q.out
@@ -106,18 +106,18 @@ Stage-0
               <-Reducer 8 [SIMPLE_EDGE] vectorized
                 SHUFFLE [RS_122]
                   PartitionCols:_col1
-                  Filter Operator [FIL_121] (rows=6 width=12)
+                  Filter Operator [FIL_121] (rows=6 width=16)
                     predicate:_col2 BETWEEN 15L AND 20L
-                    Select Operator [SEL_120] (rows=13251253 width=12)
+                    Select Operator [SEL_120] (rows=479121995 width=15)
                       Output:["_col0","_col1","_col2"]
-                      Group By Operator [GBY_119] (rows=13251253 width=12)
+                      Group By Operator [GBY_119] (rows=479121995 width=15)
                         Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
                       <-Reducer 7 [SIMPLE_EDGE]
                         SHUFFLE [RS_25]
                           PartitionCols:_col0, _col1
-                          Group By Operator [GBY_24] (rows=13251253 width=12)
+                          Group By Operator [GBY_24] (rows=479121995 width=15)
                             Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col1, _col4
-                            Merge Join Operator [MERGEJOIN_98] (rows=13251253 width=4)
+                            Merge Join Operator [MERGEJOIN_98] (rows=479121995 width=7)
                               Conds:RS_20._col3=RS_118._col0(Inner),Output:["_col1","_col4"]
                             <-Map 12 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_118]
@@ -131,7 +131,7 @@ Stage-0
                             <-Reducer 6 [SIMPLE_EDGE]
                               SHUFFLE [RS_20]
                                 PartitionCols:_col3
-                                Merge Join Operator [MERGEJOIN_97] (rows=24979074 width=5)
+                                Merge Join Operator [MERGEJOIN_97] (rows=479121995 width=11)
                                   Conds:RS_17._col2=RS_115._col0(Inner),Output:["_col1","_col3","_col4"]
                                 <-Map 11 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_115]
@@ -145,7 +145,7 @@ Stage-0
                                 <-Reducer 5 [SIMPLE_EDGE]
                                   SHUFFLE [RS_17]
                                     PartitionCols:_col2
-                                    Merge Join Operator [MERGEJOIN_96] (rows=156119211 width=14)
+                                    Merge Join Operator [MERGEJOIN_96] (rows=479121995 width=15)
                                       Conds:RS_112._col0=RS_104._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                     <-Map 9 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_104]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query35.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query35.q.out
index 167a684..5672e73 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query35.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query35.q.out
@@ -129,20 +129,20 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 13 <- Reducer 16 (BROADCAST_EDGE)
-Map 21 <- Reducer 10 (BROADCAST_EDGE)
+Map 12 <- Reducer 15 (BROADCAST_EDGE)
+Map 21 <- Reducer 18 (BROADCAST_EDGE)
 Map 22 <- Reducer 9 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 14 <- Map 13 (SIMPLE_EDGE), Map 15 (SIMPLE_EDGE)
-Reducer 16 <- Map 15 (CUSTOM_SIMPLE_EDGE)
-Reducer 17 <- Map 15 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
-Reducer 18 <- Reducer 17 (SIMPLE_EDGE)
-Reducer 19 <- Map 15 (SIMPLE_EDGE), Map 22 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
+Reducer 13 <- Map 12 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE)
+Reducer 15 <- Map 14 (CUSTOM_SIMPLE_EDGE)
+Reducer 16 <- Map 14 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
+Reducer 17 <- Reducer 16 (SIMPLE_EDGE)
+Reducer 18 <- Map 14 (CUSTOM_SIMPLE_EDGE)
+Reducer 19 <- Map 14 (SIMPLE_EDGE), Map 22 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
 Reducer 20 <- Reducer 19 (SIMPLE_EDGE)
-Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Reducer 14 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 18 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 17 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
 Reducer 6 <- Reducer 20 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
@@ -153,74 +153,112 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 8 vectorized
-      File Output Operator [FS_223]
-        Limit [LIM_222] (rows=1 width=352)
+      File Output Operator [FS_226]
+        Limit [LIM_225] (rows=1 width=352)
           Number of rows:100
-          Select Operator [SEL_221] (rows=1 width=352)
+          Select Operator [SEL_224] (rows=1 width=352)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16"]
           <-Reducer 7 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_220]
-              Select Operator [SEL_219] (rows=1 width=352)
+            SHUFFLE [RS_223]
+              Select Operator [SEL_222] (rows=1 width=352)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col9","_col10","_col11","_col12","_col14","_col15","_col16","_col17"]
-                Group By Operator [GBY_218] (rows=1 width=336)
+                Group By Operator [GBY_221] (rows=1 width=336)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","count(VALUE._col2)","max(VALUE._col3)","sum(VALUE._col4)","count(VALUE._col5)","max(VALUE._col6)","sum(VALUE._col7)","count(VALUE._col8)","max(VALUE._col9)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5
                 <-Reducer 6 [SIMPLE_EDGE]
                   SHUFFLE [RS_66]
                     PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
-                    Group By Operator [GBY_65] (rows=1 width=336)
+                    Group By Operator [GBY_65] (rows=2 width=336)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15"],aggregations:["count()","sum(_col8)","count(_col8)","max(_col8)","sum(_col9)","count(_col9)","max(_col9)","sum(_col10)","count(_col10)","max(_col10)"],keys:_col4, _col6, _col7, _col8, _col9, _col10
-                      Top N Key Operator [TNK_103] (rows=1635 width=276)
+                      Top N Key Operator [TNK_103] (rows=1401496 width=276)
                         keys:_col4, _col6, _col7, _col8, _col9, _col10,sort order:++++++,top n:100
-                        Select Operator [SEL_64] (rows=1635 width=276)
+                        Select Operator [SEL_64] (rows=1401496 width=276)
                           Output:["_col4","_col6","_col7","_col8","_col9","_col10"]
-                          Filter Operator [FIL_63] (rows=1635 width=276)
+                          Filter Operator [FIL_63] (rows=1401496 width=276)
                             predicate:(_col11 is not null or _col13 is not null)
-                            Merge Join Operator [MERGEJOIN_181] (rows=1635 width=276)
-                              Conds:RS_60._col0=RS_217._col1(Left Outer),Output:["_col4","_col6","_col7","_col8","_col9","_col10","_col11","_col13"]
+                            Merge Join Operator [MERGEJOIN_181] (rows=1401496 width=276)
+                              Conds:RS_60._col0=RS_220._col1(Left Outer),Output:["_col4","_col6","_col7","_col8","_col9","_col10","_col11","_col13"]
                             <-Reducer 5 [SIMPLE_EDGE]
                               PARTITION_ONLY_SHUFFLE [RS_60]
                                 PartitionCols:_col0
-                                Merge Join Operator [MERGEJOIN_180] (rows=1663 width=276)
-                                  Conds:RS_57._col0=RS_209._col1(Left Outer),Output:["_col0","_col4","_col6","_col7","_col8","_col9","_col10","_col11"]
+                                Merge Join Operator [MERGEJOIN_180] (rows=1414922 width=276)
+                                  Conds:RS_57._col0=RS_212._col1(Left Outer),Output:["_col0","_col4","_col6","_col7","_col8","_col9","_col10","_col11"]
+                                <-Reducer 17 [SIMPLE_EDGE] vectorized
+                                  SHUFFLE [RS_212]
+                                    PartitionCols:_col1
+                                    Select Operator [SEL_211] (rows=1414922 width=7)
+                                      Output:["_col0","_col1"]
+                                      Group By Operator [GBY_210] (rows=1414922 width=3)
+                                        Output:["_col0"],keys:KEY._col0
+                                      <-Reducer 16 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_35]
+                                          PartitionCols:_col0
+                                          Group By Operator [GBY_34] (rows=143930993 width=3)
+                                            Output:["_col0"],keys:_col1
+                                            Merge Join Operator [MERGEJOIN_177] (rows=143930993 width=3)
+                                              Conds:RS_209._col0=RS_193._col0(Inner),Output:["_col1"]
+                                            <-Map 14 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_193]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_190] (rows=652 width=4)
+                                                  Output:["_col0"]
+                                                  Filter Operator [FIL_189] (rows=652 width=12)
+                                                    predicate:((d_year = 1999) and (d_qoy < 4))
+                                                    TableScan [TS_17] (rows=73049 width=12)
+                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_qoy"]
+                                            <-Map 21 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_209]
+                                                PartitionCols:_col0
+                                                Select Operator [SEL_208] (rows=143930993 width=7)
+                                                  Output:["_col0","_col1"]
+                                                  Filter Operator [FIL_207] (rows=143930993 width=7)
+                                                    predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_sold_date_sk BETWEEN DynamicValue(RS_31_date_dim_d_date_sk_min) AND DynamicValue(RS_31_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_31_date_dim_d_date_sk_bloom_filter)))
+                                                    TableScan [TS_24] (rows=144002668 width=7)
+                                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_bill_customer_sk"]
+                                                    <-Reducer 18 [BROADCAST_EDGE] vectorized
+                                                      BROADCAST [RS_206]
+                                                        Group By Operator [GBY_205] (rows=1 width=12)
+                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                        <-Map 14 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                          SHUFFLE [RS_199]
+                                                            Group By Operator [GBY_197] (rows=1 width=12)
+                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                              Select Operator [SEL_194] (rows=652 width=4)
+                                                                Output:["_col0"]
+                                                                 Please refer to the previous Select Operator [SEL_190]
                                 <-Reducer 4 [SIMPLE_EDGE]
                                   SHUFFLE [RS_57]
                                     PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_179] (rows=1334622 width=272)
+                                    Merge Join Operator [MERGEJOIN_179] (rows=525327388 width=272)
                                       Conds:RS_54._col0=RS_55._col0(Left Semi),Output:["_col0","_col4","_col6","_col7","_col8","_col9","_col10"]
-                                    <-Reducer 14 [SIMPLE_EDGE]
+                                    <-Reducer 13 [SIMPLE_EDGE]
                                       SHUFFLE [RS_55]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_53] (rows=1383003 width=3)
+                                        Group By Operator [GBY_53] (rows=525327388 width=3)
                                           Output:["_col0"],keys:_col0
-                                          Select Operator [SEL_23] (rows=187573258 width=3)
+                                          Select Operator [SEL_23] (rows=525327388 width=3)
                                             Output:["_col0"]
-                                            Merge Join Operator [MERGEJOIN_176] (rows=187573258 width=3)
-                                              Conds:RS_201._col0=RS_191._col0(Inner),Output:["_col1"]
-                                            <-Map 15 [SIMPLE_EDGE] vectorized
+                                            Merge Join Operator [MERGEJOIN_176] (rows=525327388 width=3)
+                                              Conds:RS_204._col0=RS_191._col0(Inner),Output:["_col1"]
+                                            <-Map 14 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_191]
                                                 PartitionCols:_col0
-                                                Select Operator [SEL_190] (rows=652 width=4)
-                                                  Output:["_col0"]
-                                                  Filter Operator [FIL_189] (rows=652 width=12)
-                                                    predicate:((d_year = 1999) and (d_qoy < 4))
-                                                    TableScan [TS_17] (rows=73049 width=12)
-                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_qoy"]
-                                            <-Map 13 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_201]
+                                                 Please refer to the previous Select Operator [SEL_190]
+                                            <-Map 12 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_204]
                                                 PartitionCols:_col0
-                                                Select Operator [SEL_200] (rows=525327388 width=7)
+                                                Select Operator [SEL_203] (rows=525327388 width=7)
                                                   Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_199] (rows=525327388 width=7)
+                                                  Filter Operator [FIL_202] (rows=525327388 width=7)
                                                     predicate:(ss_sold_date_sk is not null and ss_customer_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_21_date_dim_d_date_sk_min) AND DynamicValue(RS_21_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_21_date_dim_d_date_sk_bloom_filter)))
                                                     TableScan [TS_14] (rows=575995635 width=7)
                                                       default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_customer_sk"]
-                                                    <-Reducer 16 [BROADCAST_EDGE] vectorized
-                                                      BROADCAST [RS_198]
-                                                        Group By Operator [GBY_197] (rows=1 width=12)
+                                                    <-Reducer 15 [BROADCAST_EDGE] vectorized
+                                                      BROADCAST [RS_201]
+                                                        Group By Operator [GBY_200] (rows=1 width=12)
                                                           Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                        <-Map 15 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                          SHUFFLE [RS_196]
-                                                            Group By Operator [GBY_195] (rows=1 width=12)
+                                                        <-Map 14 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                          SHUFFLE [RS_198]
+                                                            Group By Operator [GBY_196] (rows=1 width=12)
                                                               Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
                                                               Select Operator [SEL_192] (rows=652 width=4)
                                                                 Output:["_col0"]
@@ -232,7 +270,7 @@ Stage-0
                                           Output:["_col0","_col4","_col6","_col7","_col8","_col9","_col10"]
                                           Merge Join Operator [MERGEJOIN_175] (rows=78293105 width=272)
                                             Conds:RS_10._col2=RS_188._col0(Inner),Output:["_col0","_col4","_col5","_col6","_col7","_col8","_col10"]
-                                          <-Map 12 [SIMPLE_EDGE] vectorized
+                                          <-Map 11 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_188]
                                               PartitionCols:_col0
                                               Select Operator [SEL_187] (rows=40000000 width=90)
@@ -253,87 +291,49 @@ Stage-0
                                                       predicate:(c_current_cdemo_sk is not null and c_current_addr_sk is not null)
                                                       TableScan [TS_0] (rows=80000000 width=11)
                                                         default@customer,c,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_current_cdemo_sk","c_current_addr_sk"]
-                                              <-Map 11 [SIMPLE_EDGE] vectorized
+                                              <-Map 10 [SIMPLE_EDGE] vectorized
                                                 SHUFFLE [RS_186]
                                                   PartitionCols:_col0
                                                   Select Operator [SEL_185] (rows=1861800 width=186)
                                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                                                     TableScan [TS_3] (rows=1861800 width=186)
                                                       default@customer_demographics,customer_demographics,Tbl:COMPLETE,Col:COMPLETE,Output:["cd_demo_sk","cd_gender","cd_marital_status","cd_dep_count","cd_dep_employed_count","cd_dep_college_count"]
-                                <-Reducer 18 [SIMPLE_EDGE] vectorized
-                                  SHUFFLE [RS_209]
-                                    PartitionCols:_col1
-                                    Select Operator [SEL_208] (rows=505213 width=7)
-                                      Output:["_col0","_col1"]
-                                      Group By Operator [GBY_207] (rows=505213 width=3)
-                                        Output:["_col0"],keys:KEY._col0
-                                      <-Reducer 17 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_35]
-                                          PartitionCols:_col0
-                                          Group By Operator [GBY_34] (rows=505213 width=3)
-                                            Output:["_col0"],keys:_col1
-                                            Merge Join Operator [MERGEJOIN_177] (rows=51391963 width=3)
-                                              Conds:RS_206._col0=RS_193._col0(Inner),Output:["_col1"]
-                                            <-Map 15 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_193]
-                                                PartitionCols:_col0
-                                                 Please refer to the previous Select Operator [SEL_190]
-                                            <-Map 21 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_206]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_205] (rows=143930993 width=7)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_204] (rows=143930993 width=7)
-                                                    predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_bill_customer_sk BETWEEN DynamicValue(RS_57_c_c_customer_sk_min) AND DynamicValue(RS_57_c_c_customer_sk_max) and in_bloom_filter(ws_bill_customer_sk, DynamicValue(RS_57_c_c_customer_sk_bloom_filter)))
-                                                    TableScan [TS_24] (rows=144002668 width=7)
-                                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_bill_customer_sk"]
-                                                    <-Reducer 10 [BROADCAST_EDGE] vectorized
-                                                      BROADCAST [RS_203]
-                                                        Group By Operator [GBY_202] (rows=1 width=12)
-                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                        <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-                                                          SHUFFLE [RS_151]
-                                                            Group By Operator [GBY_150] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                              Select Operator [SEL_149] (rows=1334622 width=4)
-                                                                Output:["_col0"]
-                                                                 Please refer to the previous Merge Join Operator [MERGEJOIN_179]
                             <-Reducer 20 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_217]
+                              SHUFFLE [RS_220]
                                 PartitionCols:_col1
-                                Select Operator [SEL_216] (rows=496881 width=7)
+                                Select Operator [SEL_219] (rows=1401496 width=7)
                                   Output:["_col0","_col1"]
-                                  Group By Operator [GBY_215] (rows=496881 width=3)
+                                  Group By Operator [GBY_218] (rows=1401496 width=3)
                                     Output:["_col0"],keys:KEY._col0
                                   <-Reducer 19 [SIMPLE_EDGE]
                                     SHUFFLE [RS_49]
                                       PartitionCols:_col0
-                                      Group By Operator [GBY_48] (rows=993762 width=3)
+                                      Group By Operator [GBY_48] (rows=285115246 width=3)
                                         Output:["_col0"],keys:_col1
-                                        Merge Join Operator [MERGEJOIN_178] (rows=101083527 width=3)
-                                          Conds:RS_214._col0=RS_194._col0(Inner),Output:["_col1"]
-                                        <-Map 15 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_194]
+                                        Merge Join Operator [MERGEJOIN_178] (rows=285115246 width=3)
+                                          Conds:RS_217._col0=RS_195._col0(Inner),Output:["_col1"]
+                                        <-Map 14 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_195]
                                             PartitionCols:_col0
                                              Please refer to the previous Select Operator [SEL_190]
                                         <-Map 22 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_214]
+                                          SHUFFLE [RS_217]
                                             PartitionCols:_col0
-                                            Select Operator [SEL_213] (rows=285115246 width=7)
+                                            Select Operator [SEL_216] (rows=285115246 width=7)
                                               Output:["_col0","_col1"]
-                                              Filter Operator [FIL_212] (rows=285115246 width=7)
+                                              Filter Operator [FIL_215] (rows=285115246 width=7)
                                                 predicate:(cs_ship_customer_sk is not null and cs_sold_date_sk is not null and cs_ship_customer_sk BETWEEN DynamicValue(RS_60_c_c_customer_sk_min) AND DynamicValue(RS_60_c_c_customer_sk_max) and in_bloom_filter(cs_ship_customer_sk, DynamicValue(RS_60_c_c_customer_sk_bloom_filter)))
                                                 TableScan [TS_38] (rows=287989836 width=7)
                                                   default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_ship_customer_sk"]
                                                 <-Reducer 9 [BROADCAST_EDGE] vectorized
-                                                  BROADCAST [RS_211]
-                                                    Group By Operator [GBY_210] (rows=1 width=12)
+                                                  BROADCAST [RS_214]
+                                                    Group By Operator [GBY_213] (rows=1 width=12)
                                                       Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
                                                     <-Reducer 5 [CUSTOM_SIMPLE_EDGE]
                                                       PARTITION_ONLY_SHUFFLE [RS_166]
                                                         Group By Operator [GBY_165] (rows=1 width=12)
                                                           Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                          Select Operator [SEL_164] (rows=1663 width=4)
+                                                          Select Operator [SEL_164] (rows=1414922 width=4)
                                                             Output:["_col0"]
                                                              Please refer to the previous Merge Join Operator [MERGEJOIN_180]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query36.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query36.q.out
index bf2b52f..f7bccf9 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query36.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query36.q.out
@@ -106,11 +106,11 @@ Stage-0
                         <-Reducer 4 [SIMPLE_EDGE]
                           SHUFFLE [RS_23]
                             PartitionCols:_col0, _col1, _col2
-                            Group By Operator [GBY_22] (rows=85680 width=414)
+                            Group By Operator [GBY_22] (rows=2487780 width=414)
                               Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col2)","sum(_col3)"],keys:_col0, _col1, 0L
-                              Select Operator [SEL_20] (rows=30601888 width=232)
+                              Select Operator [SEL_20] (rows=525329897 width=395)
                                 Output:["_col0","_col1","_col2","_col3"]
-                                Merge Join Operator [MERGEJOIN_81] (rows=30601888 width=232)
+                                Merge Join Operator [MERGEJOIN_81] (rows=525329897 width=395)
                                   Conds:RS_17._col1=RS_97._col0(Inner),Output:["_col3","_col4","_col8","_col9"]
                                 <-Map 11 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_97]
@@ -122,7 +122,7 @@ Stage-0
                                 <-Reducer 3 [SIMPLE_EDGE]
                                   SHUFFLE [RS_17]
                                     PartitionCols:_col1
-                                    Merge Join Operator [MERGEJOIN_80] (rows=30601888 width=54)
+                                    Merge Join Operator [MERGEJOIN_80] (rows=525329897 width=217)
                                       Conds:RS_14._col2=RS_95._col0(Inner),Output:["_col1","_col3","_col4"]
                                     <-Map 10 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_95]
@@ -136,7 +136,7 @@ Stage-0
                                     <-Reducer 2 [SIMPLE_EDGE]
                                       SHUFFLE [RS_14]
                                         PartitionCols:_col2
-                                        Merge Join Operator [MERGEJOIN_79] (rows=187574154 width=203)
+                                        Merge Join Operator [MERGEJOIN_79] (rows=525329897 width=221)
                                           Conds:RS_92._col0=RS_84._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                         <-Map 8 [SIMPLE_EDGE] vectorized
                                           PARTITION_ONLY_SHUFFLE [RS_84]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query38.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query38.q.out
index a5a8df3..b9641ab 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query38.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query38.q.out
@@ -90,28 +90,28 @@ Stage-0
               Select Operator [SEL_226] (rows=1 width=8)
                 Filter Operator [FIL_225] (rows=1 width=8)
                   predicate:(_col3 = 3L)
-                  Select Operator [SEL_224] (rows=165330890 width=8)
+                  Select Operator [SEL_224] (rows=954376212 width=8)
                     Output:["_col3"]
-                    Group By Operator [GBY_223] (rows=165330890 width=282)
+                    Group By Operator [GBY_223] (rows=954376212 width=282)
                       Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                     <-Union 5 [SIMPLE_EDGE]
                       <-Reducer 12 [CONTAINS] vectorized
                         Reduce Output Operator [RS_240]
                           PartitionCols:_col0, _col1, _col2
-                          Group By Operator [GBY_239] (rows=165330890 width=282)
+                          Group By Operator [GBY_239] (rows=954376212 width=282)
                             Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                            Group By Operator [GBY_238] (rows=49146883 width=282)
+                            Group By Operator [GBY_238] (rows=285117831 width=282)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
-                              Select Operator [SEL_237] (rows=49146883 width=274)
+                              Select Operator [SEL_237] (rows=285117831 width=274)
                                 Output:["_col0","_col1","_col2"]
-                                Group By Operator [GBY_236] (rows=49146883 width=274)
+                                Group By Operator [GBY_236] (rows=285117831 width=274)
                                   Output:["_col0","_col1","_col2"],keys:KEY._col0, KEY._col1, KEY._col2
                                 <-Reducer 11 [SIMPLE_EDGE]
                                   SHUFFLE [RS_40]
                                     PartitionCols:_col0, _col1, _col2
-                                    Group By Operator [GBY_39] (rows=49146883 width=274)
+                                    Group By Operator [GBY_39] (rows=285117831 width=274)
                                       Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
-                                      Merge Join Operator [MERGEJOIN_174] (rows=49146883 width=274)
+                                      Merge Join Operator [MERGEJOIN_174] (rows=285117831 width=274)
                                         Conds:RS_35._col1=RS_216._col0(Inner),Output:["_col3","_col5","_col6"]
                                       <-Map 18 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_216]
@@ -123,7 +123,7 @@ Stage-0
                                       <-Reducer 10 [SIMPLE_EDGE]
                                         SHUFFLE [RS_35]
                                           PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_173] (rows=49146883 width=97)
+                                          Merge Join Operator [MERGEJOIN_173] (rows=285117831 width=97)
                                             Conds:RS_235._col0=RS_199._col0(Inner),Output:["_col1","_col3"]
                                           <-Map 8 [SIMPLE_EDGE] vectorized
                                             PARTITION_ONLY_SHUFFLE [RS_199]
@@ -157,20 +157,20 @@ Stage-0
                       <-Reducer 16 [CONTAINS] vectorized
                         Reduce Output Operator [RS_250]
                           PartitionCols:_col0, _col1, _col2
-                          Group By Operator [GBY_249] (rows=165330890 width=282)
+                          Group By Operator [GBY_249] (rows=954376212 width=282)
                             Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                            Group By Operator [GBY_248] (rows=24986582 width=282)
+                            Group By Operator [GBY_248] (rows=143930993 width=282)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
-                              Select Operator [SEL_247] (rows=24986582 width=274)
+                              Select Operator [SEL_247] (rows=143930993 width=274)
                                 Output:["_col0","_col1","_col2"]
-                                Group By Operator [GBY_246] (rows=24986582 width=274)
+                                Group By Operator [GBY_246] (rows=143930993 width=274)
                                   Output:["_col0","_col1","_col2"],keys:KEY._col0, KEY._col1, KEY._col2
                                 <-Reducer 15 [SIMPLE_EDGE]
                                   SHUFFLE [RS_65]
                                     PartitionCols:_col0, _col1, _col2
-                                    Group By Operator [GBY_64] (rows=24986582 width=274)
+                                    Group By Operator [GBY_64] (rows=143930993 width=274)
                                       Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
-                                      Merge Join Operator [MERGEJOIN_176] (rows=24986582 width=274)
+                                      Merge Join Operator [MERGEJOIN_176] (rows=143930993 width=274)
                                         Conds:RS_60._col1=RS_217._col0(Inner),Output:["_col3","_col5","_col6"]
                                       <-Map 18 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_217]
@@ -179,7 +179,7 @@ Stage-0
                                       <-Reducer 14 [SIMPLE_EDGE]
                                         SHUFFLE [RS_60]
                                           PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_175] (rows=24986582 width=97)
+                                          Merge Join Operator [MERGEJOIN_175] (rows=143930993 width=97)
                                             Conds:RS_245._col0=RS_201._col0(Inner),Output:["_col1","_col3"]
                                           <-Map 8 [SIMPLE_EDGE] vectorized
                                             PARTITION_ONLY_SHUFFLE [RS_201]
@@ -208,20 +208,20 @@ Stage-0
                       <-Reducer 4 [CONTAINS] vectorized
                         Reduce Output Operator [RS_222]
                           PartitionCols:_col0, _col1, _col2
-                          Group By Operator [GBY_221] (rows=165330890 width=282)
+                          Group By Operator [GBY_221] (rows=954376212 width=282)
                             Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                            Group By Operator [GBY_220] (rows=91197425 width=282)
+                            Group By Operator [GBY_220] (rows=525327388 width=282)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
-                              Select Operator [SEL_219] (rows=91197425 width=274)
+                              Select Operator [SEL_219] (rows=525327388 width=274)
                                 Output:["_col0","_col1","_col2"]
-                                Group By Operator [GBY_218] (rows=91197425 width=274)
+                                Group By Operator [GBY_218] (rows=525327388 width=274)
                                   Output:["_col0","_col1","_col2"],keys:KEY._col0, KEY._col1, KEY._col2
                                 <-Reducer 3 [SIMPLE_EDGE]
                                   SHUFFLE [RS_16]
                                     PartitionCols:_col0, _col1, _col2
-                                    Group By Operator [GBY_15] (rows=91197425 width=274)
+                                    Group By Operator [GBY_15] (rows=525327388 width=274)
                                       Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
-                                      Merge Join Operator [MERGEJOIN_172] (rows=91197425 width=274)
+                                      Merge Join Operator [MERGEJOIN_172] (rows=525327388 width=274)
                                         Conds:RS_11._col1=RS_215._col0(Inner),Output:["_col3","_col5","_col6"]
                                       <-Map 18 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_215]
@@ -230,7 +230,7 @@ Stage-0
                                       <-Reducer 2 [SIMPLE_EDGE]
                                         SHUFFLE [RS_11]
                                           PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_171] (rows=91197425 width=96)
+                                          Merge Join Operator [MERGEJOIN_171] (rows=525327388 width=97)
                                             Conds:RS_213._col0=RS_197._col0(Inner),Output:["_col1","_col3"]
                                           <-Map 8 [SIMPLE_EDGE] vectorized
                                             PARTITION_ONLY_SHUFFLE [RS_197]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query4.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query4.q.out
index 6915f53..68796c2 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query4.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query4.q.out
@@ -274,15 +274,15 @@ Stage-0
       File Output Operator [FS_562]
         Limit [LIM_561] (rows=100 width=85)
           Number of rows:100
-          Select Operator [SEL_560] (rows=7323197 width=85)
+          Select Operator [SEL_560] (rows=11399756 width=85)
             Output:["_col0"]
           <-Reducer 9 [SIMPLE_EDGE]
             SHUFFLE [RS_135]
-              Select Operator [SEL_134] (rows=7323197 width=85)
+              Select Operator [SEL_134] (rows=11399756 width=85)
                 Output:["_col0"]
-                Filter Operator [FIL_133] (rows=7323197 width=537)
+                Filter Operator [FIL_133] (rows=11399756 width=537)
                   predicate:CASE WHEN (_col4 is not null) THEN (CASE WHEN (_col7) THEN (((_col9 / _col6) > (_col14 / _col4))) ELSE (false) END) ELSE (false) END
-                  Merge Join Operator [MERGEJOIN_466] (rows=14646395 width=537)
+                  Merge Join Operator [MERGEJOIN_466] (rows=22799512 width=537)
                     Conds:RS_130._col3=RS_559._col0(Inner),Output:["_col4","_col6","_col7","_col9","_col13","_col14"]
                   <-Reducer 30 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_559]
@@ -296,7 +296,7 @@ Stage-0
                             PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
                             Group By Operator [GBY_113] (rows=80000000 width=764)
                               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                              Merge Join Operator [MERGEJOIN_461] (rows=187573258 width=764)
+                              Merge Join Operator [MERGEJOIN_461] (rows=525327388 width=764)
                                 Conds:RS_109._col1=RS_505._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                               <-Map 38 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_505]
@@ -308,7 +308,7 @@ Stage-0
                               <-Reducer 28 [SIMPLE_EDGE]
                                 SHUFFLE [RS_109]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_460] (rows=187573258 width=115)
+                                  Merge Join Operator [MERGEJOIN_460] (rows=525327388 width=115)
                                     Conds:RS_556._col0=RS_475._col0(Inner),Output:["_col1","_col2"]
                                   <-Map 31 [SIMPLE_EDGE] vectorized
                                     PARTITION_ONLY_SHUFFLE [RS_475]
@@ -342,23 +342,23 @@ Stage-0
                   <-Reducer 8 [SIMPLE_EDGE]
                     SHUFFLE [RS_130]
                       PartitionCols:_col3
-                      Filter Operator [FIL_129] (rows=12248093 width=668)
+                      Filter Operator [FIL_129] (rows=19066162 width=668)
                         predicate:CASE WHEN (_col2) THEN (CASE WHEN (_col7) THEN (((_col9 / _col6) > (_col11 / _col1))) ELSE (false) END) ELSE (false) END
-                        Merge Join Operator [MERGEJOIN_465] (rows=24496187 width=668)
+                        Merge Join Operator [MERGEJOIN_465] (rows=38132324 width=668)
                           Conds:RS_126._col3=RS_551._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col9","_col11"]
                         <-Reducer 26 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_551]
                             PartitionCols:_col0
-                            Select Operator [SEL_550] (rows=51391963 width=212)
+                            Select Operator [SEL_550] (rows=80000000 width=212)
                               Output:["_col0","_col1"]
-                              Group By Operator [GBY_549] (rows=51391963 width=764)
+                              Group By Operator [GBY_549] (rows=80000000 width=764)
                                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6
                               <-Reducer 25 [SIMPLE_EDGE]
                                 SHUFFLE [RS_95]
                                   PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
-                                  Group By Operator [GBY_94] (rows=51391963 width=764)
+                                  Group By Operator [GBY_94] (rows=80000000 width=764)
                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                                    Merge Join Operator [MERGEJOIN_459] (rows=51391963 width=764)
+                                    Merge Join Operator [MERGEJOIN_459] (rows=143930993 width=764)
                                       Conds:RS_90._col1=RS_506._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                     <-Map 38 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_506]
@@ -367,7 +367,7 @@ Stage-0
                                     <-Reducer 24 [SIMPLE_EDGE]
                                       SHUFFLE [RS_90]
                                         PartitionCols:_col1
-                                        Merge Join Operator [MERGEJOIN_458] (rows=51391963 width=115)
+                                        Merge Join Operator [MERGEJOIN_458] (rows=143930993 width=115)
                                           Conds:RS_548._col0=RS_477._col0(Inner),Output:["_col1","_col2"]
                                         <-Map 31 [SIMPLE_EDGE] vectorized
                                           PARTITION_ONLY_SHUFFLE [RS_477]
@@ -396,7 +396,7 @@ Stage-0
                         <-Reducer 7 [SIMPLE_EDGE]
                           SHUFFLE [RS_126]
                             PartitionCols:_col3
-                            Merge Join Operator [MERGEJOIN_464] (rows=20485012 width=556)
+                            Merge Join Operator [MERGEJOIN_464] (rows=31888273 width=556)
                               Conds:RS_123._col3=RS_543._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7","_col9"]
                             <-Reducer 22 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_543]
@@ -410,7 +410,7 @@ Stage-0
                                       PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
                                       Group By Operator [GBY_75] (rows=80000000 width=764)
                                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                                        Merge Join Operator [MERGEJOIN_457] (rows=101084444 width=764)
+                                        Merge Join Operator [MERGEJOIN_457] (rows=285117831 width=764)
                                           Conds:RS_71._col1=RS_507._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                         <-Map 38 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_507]
@@ -419,7 +419,7 @@ Stage-0
                                         <-Reducer 20 [SIMPLE_EDGE]
                                           SHUFFLE [RS_71]
                                             PartitionCols:_col1
-                                            Merge Join Operator [MERGEJOIN_456] (rows=101084444 width=115)
+                                            Merge Join Operator [MERGEJOIN_456] (rows=285117831 width=115)
                                               Conds:RS_540._col0=RS_479._col0(Inner),Output:["_col1","_col2"]
                                             <-Map 31 [SIMPLE_EDGE] vectorized
                                               PARTITION_ONLY_SHUFFLE [RS_479]
@@ -448,7 +448,7 @@ Stage-0
                             <-Reducer 6 [SIMPLE_EDGE]
                               SHUFFLE [RS_123]
                                 PartitionCols:_col3
-                                Merge Join Operator [MERGEJOIN_463] (rows=17130654 width=444)
+                                Merge Join Operator [MERGEJOIN_463] (rows=26666666 width=444)
                                   Conds:RS_120._col3=RS_535._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6","_col7"]
                                 <-Reducer 18 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_535]
@@ -466,7 +466,7 @@ Stage-0
                                               PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
                                               Group By Operator [GBY_55] (rows=80000000 width=764)
                                                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                                                Merge Join Operator [MERGEJOIN_455] (rows=101084444 width=764)
+                                                Merge Join Operator [MERGEJOIN_455] (rows=285117831 width=764)
                                                   Conds:RS_51._col1=RS_510._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                                 <-Map 38 [SIMPLE_EDGE] vectorized
                                                   SHUFFLE [RS_510]
@@ -475,7 +475,7 @@ Stage-0
                                                 <-Reducer 16 [SIMPLE_EDGE]
                                                   SHUFFLE [RS_51]
                                                     PartitionCols:_col1
-                                                    Merge Join Operator [MERGEJOIN_454] (rows=101084444 width=115)
+                                                    Merge Join Operator [MERGEJOIN_454] (rows=285117831 width=115)
                                                       Conds:RS_530._col0=RS_485._col0(Inner),Output:["_col1","_col2"]
                                                     <-Map 31 [SIMPLE_EDGE] vectorized
                                                       PARTITION_ONLY_SHUFFLE [RS_485]
@@ -508,7 +508,7 @@ Stage-0
                                 <-Reducer 5 [SIMPLE_EDGE]
                                   SHUFFLE [RS_120]
                                     PartitionCols:_col3
-                                    Merge Join Operator [MERGEJOIN_462] (rows=17130654 width=328)
+                                    Merge Join Operator [MERGEJOIN_462] (rows=26666666 width=328)
                                       Conds:RS_515._col0=RS_525._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                     <-Reducer 14 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_525]
@@ -526,7 +526,7 @@ Stage-0
                                                   PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
                                                   Group By Operator [GBY_35] (rows=80000000 width=764)
                                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                                                    Merge Join Operator [MERGEJOIN_453] (rows=187573258 width=764)
+                                                    Merge Join Operator [MERGEJOIN_453] (rows=525327388 width=764)
                                                       Conds:RS_31._col1=RS_509._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                                     <-Map 38 [SIMPLE_EDGE] vectorized
                                                       SHUFFLE [RS_509]
@@ -535,7 +535,7 @@ Stage-0
                                                     <-Reducer 12 [SIMPLE_EDGE]
                                                       SHUFFLE [RS_31]
                                                         PartitionCols:_col1
-                                                        Merge Join Operator [MERGEJOIN_452] (rows=187573258 width=115)
+                                                        Merge Join Operator [MERGEJOIN_452] (rows=525327388 width=115)
                                                           Conds:RS_520._col0=RS_483._col0(Inner),Output:["_col1","_col2"]
                                                         <-Map 31 [SIMPLE_EDGE] vectorized
                                                           PARTITION_ONLY_SHUFFLE [RS_483]
@@ -568,20 +568,20 @@ Stage-0
                                     <-Reducer 4 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_515]
                                         PartitionCols:_col0
-                                        Select Operator [SEL_514] (rows=17130654 width=216)
+                                        Select Operator [SEL_514] (rows=26666666 width=216)
                                           Output:["_col0","_col1","_col2"]
-                                          Filter Operator [FIL_513] (rows=17130654 width=212)
+                                          Filter Operator [FIL_513] (rows=26666666 width=212)
                                             predicate:(_col7 > 0)
-                                            Select Operator [SEL_512] (rows=51391963 width=212)
+                                            Select Operator [SEL_512] (rows=80000000 width=212)
                                               Output:["_col0","_col7"]
-                                              Group By Operator [GBY_511] (rows=51391963 width=764)
+                                              Group By Operator [GBY_511] (rows=80000000 width=764)
                                                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6
                                               <-Reducer 3 [SIMPLE_EDGE]
                                                 SHUFFLE [RS_16]
                                                   PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6
-                                                  Group By Operator [GBY_15] (rows=51391963 width=764)
+                                                  Group By Operator [GBY_15] (rows=80000000 width=764)
                                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"],aggregations:["sum(_col2)"],keys:_col5, _col6, _col7, _col8, _col9, _col10, _col11
-                                                    Merge Join Operator [MERGEJOIN_451] (rows=51391963 width=764)
+                                                    Merge Join Operator [MERGEJOIN_451] (rows=143930993 width=764)
                                                       Conds:RS_11._col1=RS_508._col0(Inner),Output:["_col2","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                                     <-Map 38 [SIMPLE_EDGE] vectorized
                                                       SHUFFLE [RS_508]
@@ -590,7 +590,7 @@ Stage-0
                                                     <-Reducer 2 [SIMPLE_EDGE]
                                                       SHUFFLE [RS_11]
                                                         PartitionCols:_col1
-                                                        Merge Join Operator [MERGEJOIN_450] (rows=51391963 width=115)
+                                                        Merge Join Operator [MERGEJOIN_450] (rows=143930993 width=115)
                                                           Conds:RS_503._col0=RS_481._col0(Inner),Output:["_col1","_col2"]
                                                         <-Map 31 [SIMPLE_EDGE] vectorized
                                                           PARTITION_ONLY_SHUFFLE [RS_481]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query40.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query40.q.out
index e7249e2..dd337c7 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query40.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query40.q.out
@@ -93,13 +93,13 @@ Stage-0
               <-Reducer 5 [SIMPLE_EDGE]
                 SHUFFLE [RS_28]
                   PartitionCols:_col0, _col1
-                  Group By Operator [GBY_27] (rows=5757278 width=410)
+                  Group By Operator [GBY_27] (rows=51819042 width=410)
                     Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col2)","sum(_col3)"],keys:_col0, _col1
-                    Top N Key Operator [TNK_53] (rows=5757278 width=278)
+                    Top N Key Operator [TNK_53] (rows=51819042 width=302)
                       keys:_col0, _col1,sort order:++,top n:100
-                      Select Operator [SEL_25] (rows=5757278 width=278)
+                      Select Operator [SEL_25] (rows=51819042 width=302)
                         Output:["_col0","_col1","_col2","_col3"]
-                        Merge Join Operator [MERGEJOIN_99] (rows=5757278 width=278)
+                        Merge Join Operator [MERGEJOIN_99] (rows=51819042 width=302)
                           Conds:RS_22._col1=RS_117._col0(Inner),Output:["_col4","_col7","_col9","_col10","_col12","_col14"]
                         <-Map 12 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_117]
@@ -111,7 +111,7 @@ Stage-0
                         <-Reducer 4 [SIMPLE_EDGE]
                           SHUFFLE [RS_22]
                             PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_98] (rows=5757278 width=195)
+                            Merge Join Operator [MERGEJOIN_98] (rows=51819042 width=220)
                               Conds:RS_19._col2=RS_102._col0(Inner),Output:["_col1","_col4","_col7","_col9","_col10","_col12"]
                             <-Map 10 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_102]
@@ -125,7 +125,7 @@ Stage-0
                             <-Reducer 3 [SIMPLE_EDGE]
                               SHUFFLE [RS_19]
                                 PartitionCols:_col2
-                                Merge Join Operator [MERGEJOIN_97] (rows=51815831 width=124)
+                                Merge Join Operator [MERGEJOIN_97] (rows=466374405 width=171)
                                   Conds:RS_16._col0=RS_115._col0(Inner),Output:["_col1","_col2","_col4","_col7","_col9","_col10"]
                                 <-Map 9 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_115]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query42.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query42.q.out
index fc0d04f..b2dc7b3 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query42.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query42.q.out
@@ -77,9 +77,9 @@ Stage-0
                 <-Reducer 3 [SIMPLE_EDGE]
                   SHUFFLE [RS_17]
                     PartitionCols:_col0, _col1
-                    Group By Operator [GBY_16] (rows=120 width=206)
+                    Group By Operator [GBY_16] (rows=6840 width=206)
                       Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col5, _col6
-                      Merge Join Operator [MERGEJOIN_54] (rows=2301098 width=94)
+                      Merge Join Operator [MERGEJOIN_54] (rows=84037218 width=173)
                         Conds:RS_12._col1=RS_68._col0(Inner),Output:["_col2","_col5","_col6"]
                       <-Map 8 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_68]
@@ -93,7 +93,7 @@ Stage-0
                       <-Reducer 2 [SIMPLE_EDGE]
                         SHUFFLE [RS_12]
                           PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_53] (rows=15062131 width=4)
+                          Merge Join Operator [MERGEJOIN_53] (rows=550076554 width=110)
                             Conds:RS_65._col0=RS_57._col0(Inner),Output:["_col1","_col2"]
                           <-Map 6 [SIMPLE_EDGE] vectorized
                             PARTITION_ONLY_SHUFFLE [RS_57]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query43.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query43.q.out
index 123520c..4cc3edd 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query43.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query43.q.out
@@ -69,13 +69,13 @@ Stage-0
               <-Reducer 3 [SIMPLE_EDGE]
                 SHUFFLE [RS_18]
                   PartitionCols:_col0, _col1
-                  Group By Operator [GBY_17] (rows=142538 width=972)
+                  Group By Operator [GBY_17] (rows=2486913 width=972)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["sum(_col2)","sum(_col3)","sum(_col4)","sum(_col5)","sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col0, _col1
-                    Top N Key Operator [TNK_33] (rows=37536846 width=257)
+                    Top N Key Operator [TNK_33] (rows=525329897 width=322)
                       keys:_col0, _col1,sort order:++,top n:100
-                      Select Operator [SEL_15] (rows=37536846 width=257)
+                      Select Operator [SEL_15] (rows=525329897 width=322)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
-                        Merge Join Operator [MERGEJOIN_55] (rows=37536846 width=257)
+                        Merge Join Operator [MERGEJOIN_55] (rows=525329897 width=322)
                           Conds:RS_12._col1=RS_69._col0(Inner),Output:["_col2","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col12","_col13"]
                         <-Map 8 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_69]
@@ -89,7 +89,7 @@ Stage-0
                         <-Reducer 2 [SIMPLE_EDGE]
                           SHUFFLE [RS_12]
                             PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_54] (rows=187574154 width=129)
+                            Merge Join Operator [MERGEJOIN_54] (rows=525329897 width=138)
                               Conds:RS_66._col0=RS_58._col0(Inner),Output:["_col1","_col2","_col4","_col5","_col6","_col7","_col8","_col9","_col10"]
                             <-Map 6 [SIMPLE_EDGE] vectorized
                               PARTITION_ONLY_SHUFFLE [RS_58]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query45.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query45.q.out
index 8d74f20..21bb318 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query45.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query45.q.out
@@ -70,26 +70,26 @@ Stage-0
       File Output Operator [FS_149]
         Limit [LIM_148] (rows=100 width=299)
           Number of rows:100
-          Select Operator [SEL_147] (rows=285780 width=299)
+          Select Operator [SEL_147] (rows=17401956 width=299)
             Output:["_col0","_col1","_col2"]
           <-Reducer 4 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_146]
-              Group By Operator [GBY_145] (rows=285780 width=299)
+              Group By Operator [GBY_145] (rows=17401956 width=299)
                 Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
               <-Reducer 3 [SIMPLE_EDGE]
                 SHUFFLE [RS_41]
                   PartitionCols:_col0, _col1
-                  Group By Operator [GBY_40] (rows=3715140 width=299)
+                  Group By Operator [GBY_40] (rows=143930993 width=299)
                     Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col8, _col7
-                    Top N Key Operator [TNK_69] (rows=10246864 width=302)
+                    Top N Key Operator [TNK_69] (rows=143930993 width=302)
                       keys:_col8, _col7,sort order:++,top n:100
-                      Select Operator [SEL_39] (rows=10246864 width=302)
+                      Select Operator [SEL_39] (rows=143930993 width=302)
                         Output:["_col3","_col7","_col8"]
-                        Filter Operator [FIL_38] (rows=10246864 width=302)
+                        Filter Operator [FIL_38] (rows=143930993 width=302)
                           predicate:(_col15 is not null or (substr(_col8, 1, 5)) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792'))
-                          Select Operator [SEL_37] (rows=10246864 width=302)
+                          Select Operator [SEL_37] (rows=143930993 width=302)
                             Output:["_col3","_col7","_col8","_col15"]
-                            Merge Join Operator [MERGEJOIN_119] (rows=10246864 width=302)
+                            Merge Join Operator [MERGEJOIN_119] (rows=143930993 width=302)
                               Conds:RS_34._col0=RS_35._col6(Inner),Output:["_col3","_col7","_col8","_col12"]
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_34]
@@ -123,12 +123,12 @@ Stage-0
                             <-Reducer 9 [SIMPLE_EDGE]
                               SHUFFLE [RS_35]
                                 PartitionCols:_col6
-                                Merge Join Operator [MERGEJOIN_118] (rows=10246864 width=302)
+                                Merge Join Operator [MERGEJOIN_118] (rows=143930993 width=302)
                                   Conds:RS_27._col0=RS_28._col2(Inner),Output:["_col3","_col4","_col6","_col8"]
                                 <-Reducer 12 [SIMPLE_EDGE]
                                   SHUFFLE [RS_28]
                                     PartitionCols:_col2
-                                    Merge Join Operator [MERGEJOIN_117] (rows=10246864 width=119)
+                                    Merge Join Operator [MERGEJOIN_117] (rows=143930993 width=119)
                                       Conds:RS_144._col0=RS_136._col0(Inner),Output:["_col1","_col2","_col3"]
                                     <-Map 13 [SIMPLE_EDGE] vectorized
                                       PARTITION_ONLY_SHUFFLE [RS_136]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query46.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query46.q.out
index d5c1418..804013f 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query46.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query46.q.out
@@ -102,15 +102,15 @@ Stage-0
       File Output Operator [FS_172]
         Limit [LIM_171] (rows=100 width=594)
           Number of rows:100
-          Select Operator [SEL_170] (rows=8380115 width=594)
+          Select Operator [SEL_170] (rows=457565061 width=594)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
           <-Reducer 3 [SIMPLE_EDGE]
             SHUFFLE [RS_44]
-              Select Operator [SEL_43] (rows=8380115 width=594)
+              Select Operator [SEL_43] (rows=457565061 width=594)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                Filter Operator [FIL_42] (rows=8380115 width=594)
+                Filter Operator [FIL_42] (rows=457565061 width=594)
                   predicate:(_col5 <> _col8)
-                  Merge Join Operator [MERGEJOIN_143] (rows=8380115 width=594)
+                  Merge Join Operator [MERGEJOIN_143] (rows=457565061 width=594)
                     Conds:RS_39._col0=RS_169._col1(Inner),Output:["_col2","_col3","_col5","_col6","_col8","_col9","_col10"]
                   <-Reducer 2 [SIMPLE_EDGE]
                     SHUFFLE [RS_39]
@@ -136,16 +136,16 @@ Stage-0
                   <-Reducer 7 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_169]
                       PartitionCols:_col1
-                      Select Operator [SEL_168] (rows=8380115 width=321)
+                      Select Operator [SEL_168] (rows=457565061 width=324)
                         Output:["_col0","_col1","_col2","_col3","_col4"]
-                        Group By Operator [GBY_167] (rows=8380115 width=321)
+                        Group By Operator [GBY_167] (rows=457565061 width=328)
                           Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
                         <-Reducer 6 [SIMPLE_EDGE]
                           SHUFFLE [RS_33]
                             PartitionCols:_col0, _col1, _col2, _col3
-                            Group By Operator [GBY_32] (rows=8380115 width=321)
+                            Group By Operator [GBY_32] (rows=457565061 width=328)
                               Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col6)","sum(_col7)"],keys:_col1, _col12, _col3, _col5
-                              Merge Join Operator [MERGEJOIN_142] (rows=8380115 width=97)
+                              Merge Join Operator [MERGEJOIN_142] (rows=457565061 width=318)
                                 Conds:RS_28._col3=RS_149._col0(Inner),Output:["_col1","_col3","_col5","_col6","_col7","_col12"]
                               <-Map 5 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_149]
@@ -154,7 +154,7 @@ Stage-0
                               <-Reducer 11 [SIMPLE_EDGE]
                                 SHUFFLE [RS_28]
                                   PartitionCols:_col3
-                                  Merge Join Operator [MERGEJOIN_141] (rows=8380115 width=4)
+                                  Merge Join Operator [MERGEJOIN_141] (rows=457565061 width=225)
                                     Conds:RS_25._col2=RS_166._col0(Inner),Output:["_col1","_col3","_col5","_col6","_col7"]
                                   <-Map 15 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_166]
@@ -168,7 +168,7 @@ Stage-0
                                   <-Reducer 10 [SIMPLE_EDGE]
                                     SHUFFLE [RS_25]
                                       PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_140] (rows=32526589 width=90)
+                                      Merge Join Operator [MERGEJOIN_140] (rows=457565061 width=229)
                                         Conds:RS_22._col4=RS_163._col0(Inner),Output:["_col1","_col2","_col3","_col5","_col6","_col7"]
                                       <-Map 14 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_163]
@@ -182,7 +182,7 @@ Stage-0
                                       <-Reducer 9 [SIMPLE_EDGE]
                                         SHUFFLE [RS_22]
                                           PartitionCols:_col4
-                                          Merge Join Operator [MERGEJOIN_139] (rows=196204013 width=218)
+                                          Merge Join Operator [MERGEJOIN_139] (rows=457565061 width=233)
                                             Conds:RS_160._col0=RS_152._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
                                           <-Map 12 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_152]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query47.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query47.q.out
index d3e36d0..701861a 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query47.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query47.q.out
@@ -159,9 +159,9 @@ Stage-0
                               <-Reducer 4 [SIMPLE_EDGE]
                                 SHUFFLE [RS_93]
                                   PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
-                                  Group By Operator [GBY_92] (rows=162257387 width=489)
+                                  Group By Operator [GBY_92] (rows=525329897 width=489)
                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col3)"],keys:_col5, _col6, _col8, _col9, _col11, _col12
-                                    Merge Join Operator [MERGEJOIN_276] (rows=162257387 width=472)
+                                    Merge Join Operator [MERGEJOIN_276] (rows=525329897 width=483)
                                       Conds:RS_88._col1=RS_295._col0(Inner),Output:["_col3","_col5","_col6","_col8","_col9","_col11","_col12"]
                                     <-Map 15 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_295]
@@ -175,7 +175,7 @@ Stage-0
                                     <-Reducer 3 [SIMPLE_EDGE]
                                       SHUFFLE [RS_88]
                                         PartitionCols:_col1
-                                        Merge Join Operator [MERGEJOIN_275] (rows=162257387 width=286)
+                                        Merge Join Operator [MERGEJOIN_275] (rows=525329897 width=297)
                                           Conds:RS_85._col2=RS_292._col0(Inner),Output:["_col1","_col3","_col5","_col6","_col8","_col9"]
                                         <-Map 14 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_292]
@@ -189,7 +189,7 @@ Stage-0
                                         <-Reducer 2 [SIMPLE_EDGE]
                                           SHUFFLE [RS_85]
                                             PartitionCols:_col2
-                                            Merge Join Operator [MERGEJOIN_274] (rows=162257387 width=111)
+                                            Merge Join Operator [MERGEJOIN_274] (rows=525329897 width=122)
                                               Conds:RS_289._col0=RS_281._col0(Inner),Output:["_col1","_col2","_col3","_col5","_col6"]
                                             <-Map 12 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_281]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query48.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query48.q.out
index ceba5a4..7cff382 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query48.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query48.q.out
@@ -162,11 +162,11 @@ Stage-0
           PARTITION_ONLY_SHUFFLE [RS_24]
             Group By Operator [GBY_23] (rows=1 width=8)
               Output:["_col0"],aggregations:["sum(_col3)"]
-              Select Operator [SEL_22] (rows=613581 width=24)
+              Select Operator [SEL_22] (rows=6953964 width=24)
                 Output:["_col3"]
-                Filter Operator [FIL_21] (rows=613581 width=24)
+                Filter Operator [FIL_21] (rows=6953964 width=24)
                   predicate:((_col10 and _col4) or (_col11 and _col5) or (_col12 and _col6))
-                  Merge Join Operator [MERGEJOIN_73] (rows=818111 width=24)
+                  Merge Join Operator [MERGEJOIN_73] (rows=9271954 width=24)
                     Conds:RS_18._col2=RS_90._col0(Inner),Output:["_col3","_col4","_col5","_col6","_col10","_col11","_col12"]
                   <-Map 9 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_90]
@@ -180,7 +180,7 @@ Stage-0
                   <-Reducer 3 [SIMPLE_EDGE]
                     SHUFFLE [RS_18]
                       PartitionCols:_col2
-                      Merge Join Operator [MERGEJOIN_72] (rows=9271916 width=13)
+                      Merge Join Operator [MERGEJOIN_72] (rows=9271954 width=13)
                         Conds:RS_15._col1=RS_87._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
                       <-Map 8 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_87]
@@ -194,7 +194,7 @@ Stage-0
                       <-Reducer 2 [SIMPLE_EDGE]
                         SHUFFLE [RS_15]
                           PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_71] (rows=57024544 width=22)
+                          Merge Join Operator [MERGEJOIN_71] (rows=159705894 width=23)
                             Conds:RS_84._col0=RS_76._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
                           <-Map 6 [SIMPLE_EDGE] vectorized
                             PARTITION_ONLY_SHUFFLE [RS_76]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query49.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query49.q.out
index 9d1118a..9f305bf 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query49.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query49.q.out
@@ -302,52 +302,52 @@ Stage-0
       File Output Operator [FS_310]
         Limit [LIM_309] (rows=100 width=215)
           Number of rows:100
-          Select Operator [SEL_308] (rows=3418 width=215)
+          Select Operator [SEL_308] (rows=40436 width=215)
             Output:["_col0","_col1","_col2","_col3","_col4"]
           <-Reducer 10 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_307]
-              Select Operator [SEL_306] (rows=3418 width=215)
+              Select Operator [SEL_306] (rows=40436 width=215)
                 Output:["_col0","_col1","_col2","_col3","_col4"]
-                Group By Operator [GBY_305] (rows=3418 width=215)
+                Group By Operator [GBY_305] (rows=40436 width=215)
                   Output:["_col0","_col1","_col2","_col3","_col4"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
                 <-Union 9 [SIMPLE_EDGE]
                   <-Reducer 24 [CONTAINS] vectorized
                     Reduce Output Operator [RS_351]
                       PartitionCols:_col0, _col1, _col2, _col3, _col4
-                      Group By Operator [GBY_350] (rows=3418 width=215)
+                      Group By Operator [GBY_350] (rows=40436 width=215)
                         Output:["_col0","_col1","_col2","_col3","_col4"],keys:_col0, _col3, _col4, _col1, _col2
-                        Top N Key Operator [TNK_349] (rows=3418 width=214)
+                        Top N Key Operator [TNK_349] (rows=40436 width=214)
                           keys:_col0, _col3, _col4, _col1, _col2,sort order:+++++,top n:100
-                          Select Operator [SEL_348] (rows=1142 width=213)
+                          Select Operator [SEL_348] (rows=14232 width=213)
                             Output:["_col0","_col1","_col2","_col3","_col4"]
-                            Filter Operator [FIL_347] (rows=1142 width=248)
+                            Filter Operator [FIL_347] (rows=14232 width=248)
                               predicate:((_col0 <= 10) or (rank_window_1 <= 10))
-                              PTF Operator [PTF_346] (rows=1714 width=248)
+                              PTF Operator [PTF_346] (rows=21349 width=248)
                                 Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col4 AS decimal(15,4)) / CAST( _col5 AS decimal(15,4))) ASC NULLS LAST","partition by:":"0"}]
-                                Select Operator [SEL_345] (rows=1714 width=248)
+                                Select Operator [SEL_345] (rows=21349 width=248)
                                   Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                                 <-Reducer 23 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_344]
                                     PartitionCols:0
-                                    Select Operator [SEL_343] (rows=1714 width=244)
+                                    Select Operator [SEL_343] (rows=21349 width=244)
                                       Output:["rank_window_0","_col0","_col1","_col2","_col3","_col4"]
-                                      PTF Operator [PTF_342] (rows=1714 width=244)
+                                      PTF Operator [PTF_342] (rows=21349 width=244)
                                         Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col1 AS decimal(15,4)) / CAST( _col2 AS decimal(15,4))) ASC NULLS LAST","partition by:":"0"}]
-                                        Select Operator [SEL_341] (rows=1714 width=244)
+                                        Select Operator [SEL_341] (rows=21349 width=244)
                                           Output:["_col0","_col1","_col2","_col3","_col4"]
                                         <-Reducer 22 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_340]
                                             PartitionCols:0
-                                            Group By Operator [GBY_339] (rows=1714 width=244)
+                                            Group By Operator [GBY_339] (rows=21349 width=244)
                                               Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0
                                             <-Reducer 21 [SIMPLE_EDGE]
                                               SHUFFLE [RS_89]
                                                 PartitionCols:_col0
-                                                Group By Operator [GBY_88] (rows=1714 width=244)
+                                                Group By Operator [GBY_88] (rows=384282 width=244)
                                                   Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col1)","sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0
-                                                  Select Operator [SEL_86] (rows=1673571 width=73)
+                                                  Select Operator [SEL_86] (rows=20856667 width=216)
                                                     Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                    Merge Join Operator [MERGEJOIN_237] (rows=1673571 width=73)
+                                                    Merge Join Operator [MERGEJOIN_237] (rows=20856667 width=216)
                                                       Conds:RS_83._col1, _col2=RS_338._col0, _col1(Inner),Output:["_col1","_col3","_col4","_col9","_col10"]
                                                     <-Map 30 [SIMPLE_EDGE] vectorized
                                                       SHUFFLE [RS_338]
@@ -361,7 +361,7 @@ Stage-0
                                                     <-Reducer 20 [SIMPLE_EDGE]
                                                       SHUFFLE [RS_83]
                                                         PartitionCols:_col1, _col2
-                                                        Merge Join Operator [MERGEJOIN_236] (rows=1673571 width=8)
+                                                        Merge Join Operator [MERGEJOIN_236] (rows=61119617 width=118)
                                                           Conds:RS_335._col0=RS_272._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                                         <-Map 12 [SIMPLE_EDGE] vectorized
                                                           PARTITION_ONLY_SHUFFLE [RS_272]
@@ -395,50 +395,50 @@ Stage-0
                   <-Reducer 8 [CONTAINS] vectorized
                     Reduce Output Operator [RS_304]
                       PartitionCols:_col0, _col1, _col2, _col3, _col4
-                      Group By Operator [GBY_303] (rows=3418 width=215)
+                      Group By Operator [GBY_303] (rows=40436 width=215)
                         Output:["_col0","_col1","_col2","_col3","_col4"],keys:_col0, _col3, _col4, _col1, _col2
-                        Top N Key Operator [TNK_302] (rows=3418 width=214)
+                        Top N Key Operator [TNK_302] (rows=40436 width=214)
                           keys:_col0, _col3, _col4, _col1, _col2,sort order:+++++,top n:100
-                          Select Operator [SEL_301] (rows=2276 width=215)
+                          Select Operator [SEL_301] (rows=26204 width=215)
                             Output:["_col0","_col1","_col2","_col3","_col4"]
-                            Group By Operator [GBY_300] (rows=2276 width=215)
+                            Group By Operator [GBY_300] (rows=26204 width=215)
                               Output:["_col0","_col1","_col2","_col3","_col4"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
                             <-Union 7 [SIMPLE_EDGE]
                               <-Reducer 18 [CONTAINS] vectorized
                                 Reduce Output Operator [RS_330]
                                   PartitionCols:_col0, _col1, _col2, _col3, _col4
-                                  Group By Operator [GBY_329] (rows=2276 width=215)
+                                  Group By Operator [GBY_329] (rows=26204 width=215)
                                     Output:["_col0","_col1","_col2","_col3","_col4"],keys:_col0, _col3, _col4, _col1, _col2
-                                    Select Operator [SEL_328] (rows=1134 width=215)
+                                    Select Operator [SEL_328] (rows=12574 width=215)
                                       Output:["_col0","_col1","_col2","_col3","_col4"]
-                                      Filter Operator [FIL_327] (rows=1134 width=248)
+                                      Filter Operator [FIL_327] (rows=12574 width=248)
                                         predicate:((_col0 <= 10) or (rank_window_1 <= 10))
-                                        PTF Operator [PTF_326] (rows=1701 width=248)
+                                        PTF Operator [PTF_326] (rows=18863 width=248)
                                           Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col4 AS decimal(15,4)) / CAST( _col5 AS decimal(15,4))) ASC NULLS LAST","partition by:":"0"}]
-                                          Select Operator [SEL_325] (rows=1701 width=248)
+                                          Select Operator [SEL_325] (rows=18863 width=248)
                                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                                           <-Reducer 17 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_324]
                                               PartitionCols:0
-                                              Select Operator [SEL_323] (rows=1701 width=244)
+                                              Select Operator [SEL_323] (rows=18863 width=244)
                                                 Output:["rank_window_0","_col0","_col1","_col2","_col3","_col4"]
-                                                PTF Operator [PTF_322] (rows=1701 width=244)
+                                                PTF Operator [PTF_322] (rows=18863 width=244)
                                                   Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col1 AS decimal(15,4)) / CAST( _col2 AS decimal(15,4))) ASC NULLS LAST","partition by:":"0"}]
-                                                  Select Operator [SEL_321] (rows=1701 width=244)
+                                                  Select Operator [SEL_321] (rows=18863 width=244)
                                                     Output:["_col0","_col1","_col2","_col3","_col4"]
                                                   <-Reducer 16 [SIMPLE_EDGE] vectorized
                                                     SHUFFLE [RS_320]
                                                       PartitionCols:0
-                                                      Group By Operator [GBY_319] (rows=1701 width=244)
+                                                      Group By Operator [GBY_319] (rows=18863 width=244)
                                                         Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0
                                                       <-Reducer 15 [SIMPLE_EDGE]
                                                         SHUFFLE [RS_50]
                                                           PartitionCols:_col0
-                                                          Group By Operator [GBY_49] (rows=1701 width=244)
+                                                          Group By Operator [GBY_49] (rows=169767 width=244)
                                                             Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col1)","sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0
-                                                            Select Operator [SEL_47] (rows=865646 width=188)
+                                                            Select Operator [SEL_47] (rows=9599627 width=231)
                                                               Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                              Merge Join Operator [MERGEJOIN_235] (rows=865646 width=188)
+                                                              Merge Join Operator [MERGEJOIN_235] (rows=9599627 width=231)
                                                                 Conds:RS_44._col1, _col2=RS_318._col0, _col1(Inner),Output:["_col1","_col3","_col4","_col9","_col10"]
                                                               <-Map 28 [SIMPLE_EDGE] vectorized
                                                                 SHUFFLE [RS_318]
@@ -452,7 +452,7 @@ Stage-0
                                                               <-Reducer 14 [SIMPLE_EDGE]
                                                                 SHUFFLE [RS_44]
                                                                   PartitionCols:_col1, _col2
-                                                                  Merge Join Operator [MERGEJOIN_234] (rows=865646 width=102)
+                                                                  Merge Join Operator [MERGEJOIN_234] (rows=31838858 width=123)
                                                                     Conds:RS_315._col0=RS_270._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                                                   <-Map 12 [SIMPLE_EDGE] vectorized
                                                                     PARTITION_ONLY_SHUFFLE [RS_270]
@@ -481,38 +481,38 @@ Stage-0
                               <-Reducer 6 [CONTAINS] vectorized
                                 Reduce Output Operator [RS_299]
                                   PartitionCols:_col0, _col1, _col2, _col3, _col4
-                                  Group By Operator [GBY_298] (rows=2276 width=215)
+                                  Group By Operator [GBY_298] (rows=26204 width=215)
                                     Output:["_col0","_col1","_col2","_col3","_col4"],keys:_col0, _col3, _col4, _col1, _col2
-                                    Select Operator [SEL_297] (rows=1142 width=211)
+                                    Select Operator [SEL_297] (rows=13630 width=211)
                                       Output:["_col0","_col1","_col2","_col3","_col4"]
-                                      Filter Operator [FIL_296] (rows=1142 width=248)
+                                      Filter Operator [FIL_296] (rows=13630 width=248)
                                         predicate:((_col0 <= 10) or (rank_window_1 <= 10))
-                                        PTF Operator [PTF_295] (rows=1714 width=248)
+                                        PTF Operator [PTF_295] (rows=20445 width=248)
                                           Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col4 AS decimal(15,4)) / CAST( _col5 AS decimal(15,4))) ASC NULLS LAST","partition by:":"0"}]
-                                          Select Operator [SEL_294] (rows=1714 width=248)
+                                          Select Operator [SEL_294] (rows=20445 width=248)
                                             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                                           <-Reducer 5 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_293]
                                               PartitionCols:0
-                                              Select Operator [SEL_292] (rows=1714 width=244)
+                                              Select Operator [SEL_292] (rows=20445 width=244)
                                                 Output:["rank_window_0","_col0","_col1","_col2","_col3","_col4"]
-                                                PTF Operator [PTF_291] (rows=1714 width=244)
+                                                PTF Operator [PTF_291] (rows=20445 width=244)
                                                   Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(CAST( _col1 AS decimal(15,4)) / CAST( _col2 AS decimal(15,4))) ASC NULLS LAST","partition by:":"0"}]
-                                                  Select Operator [SEL_290] (rows=1714 width=244)
+                                                  Select Operator [SEL_290] (rows=20445 width=244)
                                                     Output:["_col0","_col1","_col2","_col3","_col4"]
                                                   <-Reducer 4 [SIMPLE_EDGE] vectorized
                                                     SHUFFLE [RS_289]
                                                       PartitionCols:0
-                                                      Group By Operator [GBY_288] (rows=1714 width=244)
+                                                      Group By Operator [GBY_288] (rows=20445 width=244)
                                                         Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0
                                                       <-Reducer 3 [SIMPLE_EDGE]
                                                         SHUFFLE [RS_18]
                                                           PartitionCols:_col0
-                                                          Group By Operator [GBY_17] (rows=1714 width=244)
+                                                          Group By Operator [GBY_17] (rows=102225 width=244)
                                                             Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col1)","sum(_col2)","sum(_col3)","sum(_col4)"],keys:_col0
-                                                            Select Operator [SEL_15] (rows=438010 width=177)
+                                                            Select Operator [SEL_15] (rows=5227456 width=231)
                                                               Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                              Merge Join Operator [MERGEJOIN_233] (rows=438010 width=177)
+                                                              Merge Join Operator [MERGEJOIN_233] (rows=5227456 width=231)
                                                                 Conds:RS_12._col1, _col2=RS_287._col0, _col1(Inner),Output:["_col1","_col3","_col4","_col9","_col10"]
                                                               <-Map 26 [SIMPLE_EDGE] vectorized
                                                                 SHUFFLE [RS_287]
@@ -526,7 +526,7 @@ Stage-0
                                                               <-Reducer 2 [SIMPLE_EDGE]
                                                                 SHUFFLE [RS_12]
                                                                   PartitionCols:_col1, _col2
-                                                                  Merge Join Operator [MERGEJOIN_232] (rows=438010 width=122)
+                                                                  Merge Join Operator [MERGEJOIN_232] (rows=15996318 width=123)
                                                                     Conds:RS_284._col0=RS_268._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                                                   <-Map 12 [SIMPLE_EDGE] vectorized
                                                                     PARTITION_ONLY_SHUFFLE [RS_268]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query5.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query5.q.out
index 1a8d6b0..3d0c064 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query5.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query5.q.out
@@ -329,9 +329,9 @@ Stage-0
                             <-Reducer 13 [SIMPLE_EDGE]
                               SHUFFLE [RS_45]
                                 PartitionCols:_col0
-                                Group By Operator [GBY_44] (rows=2835758 width=548)
+                                Group By Operator [GBY_44] (rows=26026820 width=548)
                                   Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col2)","sum(_col4)","sum(_col3)","sum(_col5)"],keys:_col8
-                                  Merge Join Operator [MERGEJOIN_219] (rows=34813117 width=535)
+                                  Merge Join Operator [MERGEJOIN_219] (rows=313339499 width=546)
                                     Conds:RS_40._col0=RS_305._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col8"]
                                   <-Map 24 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_305]
@@ -343,7 +343,7 @@ Stage-0
                                   <-Reducer 12 [SIMPLE_EDGE]
                                     SHUFFLE [RS_40]
                                       PartitionCols:_col0
-                                      Merge Join Operator [MERGEJOIN_218] (rows=34813117 width=438)
+                                      Merge Join Operator [MERGEJOIN_218] (rows=313339499 width=450)
                                         Conds:Union 22._col1=RS_273._col0(Inner),Output:["_col0","_col2","_col3","_col4","_col5"]
                                       <-Map 10 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_273]
@@ -398,9 +398,9 @@ Stage-0
                             <-Reducer 17 [SIMPLE_EDGE]
                               SHUFFLE [RS_77]
                                 PartitionCols:_col0
-                                Group By Operator [GBY_76] (rows=3498 width=548)
+                                Group By Operator [GBY_76] (rows=31641 width=548)
                                   Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col2)","sum(_col4)","sum(_col3)","sum(_col5)"],keys:_col8
-                                  Merge Join Operator [MERGEJOIN_221] (rows=30966059 width=543)
+                                  Merge Join Operator [MERGEJOIN_221] (rows=278713608 width=547)
                                     Conds:RS_72._col0=RS_312._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col8"]
                                   <-Map 30 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_312]
@@ -412,7 +412,7 @@ Stage-0
                                   <-Reducer 16 [SIMPLE_EDGE]
                                     SHUFFLE [RS_72]
                                       PartitionCols:_col0
-                                      Merge Join Operator [MERGEJOIN_220] (rows=30966059 width=447)
+                                      Merge Join Operator [MERGEJOIN_220] (rows=278713608 width=451)
                                         Conds:Union 26._col1=RS_275._col0(Inner),Output:["_col0","_col2","_col3","_col4","_col5"]
                                       <-Map 10 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_275]
@@ -478,9 +478,9 @@ Stage-0
                             <-Reducer 4 [SIMPLE_EDGE]
                               SHUFFLE [RS_21]
                                 PartitionCols:_col0
-                                Group By Operator [GBY_20] (rows=78090 width=548)
+                                Group By Operator [GBY_20] (rows=983934 width=548)
                                   Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col2)","sum(_col4)","sum(_col3)","sum(_col5)"],keys:_col8
-                                  Merge Join Operator [MERGEJOIN_217] (rows=64325014 width=376)
+                                  Merge Join Operator [MERGEJOIN_217] (rows=578964757 width=528)
                                     Conds:RS_16._col0=RS_289._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col8"]
                                   <-Map 20 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_289]
@@ -492,7 +492,7 @@ Stage-0
                                   <-Reducer 3 [SIMPLE_EDGE]
                                     SHUFFLE [RS_16]
                                       PartitionCols:_col0
-                                      Merge Join Operator [MERGEJOIN_216] (rows=64325014 width=277)
+                                      Merge Join Operator [MERGEJOIN_216] (rows=578964757 width=432)
                                         Conds:Union 2._col1=RS_271._col0(Inner),Output:["_col0","_col2","_col3","_col4","_col5"]
                                       <-Map 10 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_271]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query50.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query50.q.out
index a45226c..c138e23 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query50.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query50.q.out
@@ -127,55 +127,62 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 9 <- Reducer 7 (BROADCAST_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
-Reducer 3 <- Map 9 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 10 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
 Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
-Reducer 7 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:100
     Stage-1
       Reducer 6 vectorized
-      File Output Operator [FS_114]
-        Limit [LIM_113] (rows=100 width=858)
+      File Output Operator [FS_112]
+        Limit [LIM_111] (rows=100 width=858)
           Number of rows:100
-          Select Operator [SEL_112] (rows=11945216 width=857)
+          Select Operator [SEL_110] (rows=478292911 width=857)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"]
           <-Reducer 5 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_111]
-              Group By Operator [GBY_110] (rows=11945216 width=857)
+            SHUFFLE [RS_109]
+              Group By Operator [GBY_108] (rows=478292911 width=857)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9
               <-Reducer 4 [SIMPLE_EDGE]
                 SHUFFLE [RS_23]
                   PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                  Group By Operator [GBY_22] (rows=11945216 width=857)
+                  Group By Operator [GBY_22] (rows=478292911 width=857)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
-                    Top N Key Operator [TNK_43] (rows=11945216 width=821)
+                    Top N Key Operator [TNK_43] (rows=478292911 width=825)
                       keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9,sort order:++++++++++,top n:100
-                      Select Operator [SEL_20] (rows=11945216 width=821)
+                      Select Operator [SEL_20] (rows=478292911 width=825)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"]
-                        Merge Join Operator [MERGEJOIN_96] (rows=11945216 width=821)
-                          Conds:RS_17._col8=RS_109._col0(Inner),Output:["_col0","_col5","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20"]
-                        <-Map 10 [SIMPLE_EDGE] vectorized
-                          SHUFFLE [RS_109]
+                        Merge Join Operator [MERGEJOIN_96] (rows=478292911 width=825)
+                          Conds:RS_17._col8=RS_107._col0(Inner),Output:["_col0","_col5","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20"]
+                        <-Map 9 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_107]
                             PartitionCols:_col0
-                            Select Operator [SEL_108] (rows=1704 width=821)
+                            Select Operator [SEL_106] (rows=1704 width=821)
                               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"]
                               TableScan [TS_9] (rows=1704 width=821)
                                 default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_store_sk","s_store_name","s_company_id","s_street_number","s_street_name","s_street_type","s_suite_number","s_city","s_county","s_state","s_zip"]
                         <-Reducer 3 [SIMPLE_EDGE]
                           SHUFFLE [RS_17]
                             PartitionCols:_col8
-                            Merge Join Operator [MERGEJOIN_95] (rows=11945216 width=3)
-                              Conds:RS_14._col1, _col2, _col3=RS_107._col1, _col2, _col4(Inner),Output:["_col0","_col5","_col8"]
+                            Merge Join Operator [MERGEJOIN_95] (rows=478292911 width=11)
+                              Conds:RS_14._col1, _col2, _col3=RS_105._col1, _col2, _col4(Inner),Output:["_col0","_col5","_col8"]
+                            <-Map 8 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_105]
+                                PartitionCols:_col1, _col2, _col4
+                                Select Operator [SEL_104] (rows=501694138 width=19)
+                                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                                  Filter Operator [FIL_103] (rows=501694138 width=19)
+                                    predicate:(ss_sold_date_sk is not null and ss_customer_sk is not null and ss_store_sk is not null)
+                                    TableScan [TS_6] (rows=575995635 width=19)
+                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_store_sk","ss_ticket_number"]
                             <-Reducer 2 [SIMPLE_EDGE]
-                              PARTITION_ONLY_SHUFFLE [RS_14]
+                              SHUFFLE [RS_14]
                                 PartitionCols:_col1, _col2, _col3
-                                Merge Join Operator [MERGEJOIN_94] (rows=1339446 width=8)
+                                Merge Join Operator [MERGEJOIN_94] (rows=53632139 width=15)
                                   Conds:RS_99._col0=RS_102._col0(Inner),Output:["_col0","_col1","_col2","_col3"]
                                 <-Map 1 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_99]
@@ -186,7 +193,7 @@ Stage-0
                                         predicate:(sr_customer_sk is not null and sr_returned_date_sk is not null)
                                         TableScan [TS_0] (rows=57591150 width=15)
                                           default@store_returns,store_returns,Tbl:COMPLETE,Col:COMPLETE,Output:["sr_returned_date_sk","sr_item_sk","sr_customer_sk","sr_ticket_number"]
-                                <-Map 8 [SIMPLE_EDGE] vectorized
+                                <-Map 7 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_102]
                                     PartitionCols:_col0
                                     Select Operator [SEL_101] (rows=50 width=4)
@@ -195,24 +202,4 @@ Stage-0
                                         predicate:((d_year = 2000) and (d_moy = 9))
                                         TableScan [TS_3] (rows=73049 width=12)
                                           default@date_dim,d2,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
-                            <-Map 9 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_107]
-                                PartitionCols:_col1, _col2, _col4
-                                Select Operator [SEL_106] (rows=501694138 width=19)
-                                  Output:["_col0","_col1","_col2","_col3","_col4"]
-                                  Filter Operator [FIL_105] (rows=501694138 width=19)
-                                    predicate:(ss_sold_date_sk is not null and ss_customer_sk is not null and ss_store_sk is not null and ss_ticket_number BETWEEN DynamicValue(RS_14_store_returns_sr_ticket_number_min) AND DynamicValue(RS_14_store_returns_sr_ticket_number_max) and in_bloom_filter(ss_ticket_number, DynamicValue(RS_14_store_returns_sr_ticket_number_bloom_filter)))
-                                    TableScan [TS_6] (rows=575995635 width=19)
-                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_store_sk","ss_ticket_number"]
-                                    <-Reducer 7 [BROADCAST_EDGE] vectorized
-                                      BROADCAST [RS_104]
-                                        Group By Operator [GBY_103] (rows=1 width=12)
-                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                        <-Reducer 2 [CUSTOM_SIMPLE_EDGE]
-                                          PARTITION_ONLY_SHUFFLE [RS_81]
-                                            Group By Operator [GBY_80] (rows=1 width=12)
-                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                              Select Operator [SEL_79] (rows=1339446 width=8)
-                                                Output:["_col0"]
-                                                 Please refer to the previous Merge Join Operator [MERGEJOIN_94]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query51.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query51.q.out
index f00e230..4255d5c 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query51.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query51.q.out
@@ -117,38 +117,38 @@ Stage-0
       File Output Operator [FS_117]
         Limit [LIM_116] (rows=100 width=636)
           Number of rows:100
-          Select Operator [SEL_115] (rows=363803676 width=636)
+          Select Operator [SEL_115] (rows=2095597606 width=636)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 5 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_114]
-              Select Operator [SEL_113] (rows=363803676 width=636)
+              Select Operator [SEL_113] (rows=2095597606 width=636)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                Filter Operator [FIL_112] (rows=363803676 width=420)
+                Filter Operator [FIL_112] (rows=2095597606 width=420)
                   predicate:(max_window_0 > max_window_1)
-                  PTF Operator [PTF_111] (rows=1091411029 width=420)
+                  PTF Operator [PTF_111] (rows=6286792818 width=420)
                     Function definitions:[{},{"name:":"windowingtablefunction","order by:":"CASE WHEN (_col4 is not null) THEN (_col4) ELSE (_col1) END ASC NULLS LAST","partition by:":"CASE WHEN (_col3 is not null) THEN (_col3) ELSE (_col0) END"}]
-                    Select Operator [SEL_110] (rows=1091411029 width=420)
+                    Select Operator [SEL_110] (rows=6286792818 width=420)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                     <-Reducer 4 [SIMPLE_EDGE]
                       SHUFFLE [RS_43]
                         PartitionCols:CASE WHEN (_col3 is not null) THEN (_col3) ELSE (_col0) END
-                        Merge Join Operator [MERGEJOIN_87] (rows=1091411029 width=420)
+                        Merge Join Operator [MERGEJOIN_87] (rows=6286792818 width=420)
                           Conds:RS_40._col0, _col1=RS_41._col0, _col1(Outer),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                         <-Reducer 10 [SIMPLE_EDGE]
                           SHUFFLE [RS_41]
                             PartitionCols:_col0, _col1
-                            Select Operator [SEL_37] (rows=3442937 width=210)
+                            Select Operator [SEL_37] (rows=19832154 width=210)
                               Output:["_col0","_col1","_col2"]
-                              PTF Operator [PTF_36] (rows=3442937 width=210)
+                              PTF Operator [PTF_36] (rows=19832154 width=210)
                                 Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS LAST","partition by:":"_col0"}]
-                                Group By Operator [GBY_32] (rows=3442937 width=210)
+                                Group By Operator [GBY_32] (rows=19832154 width=210)
                                   Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                                 <-Reducer 9 [SIMPLE_EDGE]
                                   SHUFFLE [RS_31]
                                     PartitionCols:_col0
-                                    Group By Operator [GBY_30] (rows=24992810 width=210)
+                                    Group By Operator [GBY_30] (rows=143966864 width=210)
                                       Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col1, _col4
-                                      Merge Join Operator [MERGEJOIN_86] (rows=24992810 width=209)
+                                      Merge Join Operator [MERGEJOIN_86] (rows=143966864 width=209)
                                         Conds:RS_108._col0=RS_92._col0(Inner),Output:["_col1","_col2","_col4"]
                                       <-Map 7 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_92]
@@ -182,18 +182,18 @@ Stage-0
                         <-Reducer 3 [SIMPLE_EDGE]
                           SHUFFLE [RS_40]
                             PartitionCols:_col0, _col1
-                            Select Operator [SEL_17] (rows=3442937 width=210)
+                            Select Operator [SEL_17] (rows=19832154 width=210)
                               Output:["_col0","_col1","_col2"]
-                              PTF Operator [PTF_16] (rows=3442937 width=210)
+                              PTF Operator [PTF_16] (rows=19832154 width=210)
                                 Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS LAST","partition by:":"_col0"}]
-                                Group By Operator [GBY_12] (rows=3442937 width=210)
+                                Group By Operator [GBY_12] (rows=19832154 width=210)
                                   Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                                 <-Reducer 2 [SIMPLE_EDGE]
                                   SHUFFLE [RS_11]
                                     PartitionCols:_col0
-                                    Group By Operator [GBY_10] (rows=95493908 width=210)
+                                    Group By Operator [GBY_10] (rows=550076554 width=210)
                                       Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col1, _col4
-                                      Merge Join Operator [MERGEJOIN_85] (rows=95493908 width=180)
+                                      Merge Join Operator [MERGEJOIN_85] (rows=550076554 width=204)
                                         Conds:RS_102._col0=RS_90._col0(Inner),Output:["_col1","_col2","_col4"]
                                       <-Map 7 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_90]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query52.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query52.q.out
index 1582a9d..56d73b8 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query52.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query52.q.out
@@ -79,7 +79,7 @@ Stage-0
                     PartitionCols:_col0, _col1
                     Group By Operator [GBY_16] (rows=7333 width=216)
                       Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col5, _col6
-                      Merge Join Operator [MERGEJOIN_54] (rows=2301098 width=104)
+                      Merge Join Operator [MERGEJOIN_54] (rows=84037218 width=183)
                         Conds:RS_12._col1=RS_68._col0(Inner),Output:["_col2","_col5","_col6"]
                       <-Map 8 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_68]
@@ -93,7 +93,7 @@ Stage-0
                       <-Reducer 2 [SIMPLE_EDGE]
                         SHUFFLE [RS_12]
                           PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_53] (rows=15062131 width=4)
+                          Merge Join Operator [MERGEJOIN_53] (rows=550076554 width=110)
                             Conds:RS_65._col0=RS_57._col0(Inner),Output:["_col1","_col2"]
                           <-Map 6 [SIMPLE_EDGE] vectorized
                             PARTITION_ONLY_SHUFFLE [RS_57]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query53.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query53.q.out
index d59ffba..f8f495d 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query53.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query53.q.out
@@ -78,30 +78,30 @@ Stage-0
     Stage-1
       Reducer 5 vectorized
       File Output Operator [FS_81]
-        Limit [LIM_80] (rows=25 width=228)
+        Limit [LIM_80] (rows=100 width=228)
           Number of rows:100
-          Select Operator [SEL_79] (rows=25 width=228)
+          Select Operator [SEL_79] (rows=130 width=228)
             Output:["_col0","_col1","_col2"]
           <-Reducer 4 [SIMPLE_EDGE]
             SHUFFLE [RS_27]
-              Select Operator [SEL_24] (rows=25 width=228)
+              Select Operator [SEL_24] (rows=130 width=228)
                 Output:["_col0","_col1","_col2"]
-                Filter Operator [FIL_36] (rows=25 width=228)
+                Filter Operator [FIL_36] (rows=130 width=228)
                   predicate:CASE WHEN ((avg_window_0 > 0)) THEN (((abs((_col2 - avg_window_0)) / avg_window_0) > 0.1)) ELSE (false) END
-                  Select Operator [SEL_23] (rows=50 width=116)
+                  Select Operator [SEL_23] (rows=260 width=116)
                     Output:["avg_window_0","_col0","_col2"]
-                    PTF Operator [PTF_22] (rows=50 width=116)
+                    PTF Operator [PTF_22] (rows=260 width=116)
                       Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col0 ASC NULLS FIRST","partition by:":"_col0"}]
-                      Select Operator [SEL_19] (rows=50 width=116)
+                      Select Operator [SEL_19] (rows=260 width=116)
                         Output:["_col0","_col2"]
-                        Group By Operator [GBY_18] (rows=50 width=120)
+                        Group By Operator [GBY_18] (rows=260 width=120)
                           Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                         <-Reducer 3 [SIMPLE_EDGE]
                           SHUFFLE [RS_17]
                             PartitionCols:_col0
-                            Group By Operator [GBY_16] (rows=50 width=120)
+                            Group By Operator [GBY_16] (rows=260 width=120)
                               Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col4, _col6
-                              Merge Join Operator [MERGEJOIN_62] (rows=98800 width=8)
+                              Merge Join Operator [MERGEJOIN_62] (rows=569118 width=8)
                                 Conds:RS_12._col0=RS_76._col0(Inner),Output:["_col2","_col4","_col6"]
                               <-Map 8 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_76]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query54.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query54.q.out
index d14697d..d85597a 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query54.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query54.q.out
@@ -1,7 +1,7 @@
-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
+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
 PREHOOK: query: explain
 with my_customers as (
  select distinct c_customer_sk
@@ -133,28 +133,26 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 17 (BROADCAST_EDGE)
-Map 23 <- Reducer 31 (BROADCAST_EDGE), Union 24 (CONTAINS)
-Map 29 <- Reducer 31 (BROADCAST_EDGE), Union 24 (CONTAINS)
+Map 22 <- Reducer 30 (BROADCAST_EDGE), Union 23 (CONTAINS)
+Map 28 <- Reducer 30 (BROADCAST_EDGE), Union 23 (CONTAINS)
 Reducer 10 <- Map 9 (SIMPLE_EDGE)
 Reducer 11 <- Reducer 10 (CUSTOM_SIMPLE_EDGE)
 Reducer 12 <- Map 9 (SIMPLE_EDGE)
 Reducer 13 <- Reducer 12 (CUSTOM_SIMPLE_EDGE)
 Reducer 14 <- Map 9 (SIMPLE_EDGE)
-Reducer 15 <- Map 34 (CUSTOM_SIMPLE_EDGE), Reducer 14 (CUSTOM_SIMPLE_EDGE)
-Reducer 16 <- Reducer 15 (CUSTOM_SIMPLE_EDGE), Reducer 18 (CUSTOM_SIMPLE_EDGE)
-Reducer 17 <- Reducer 16 (CUSTOM_SIMPLE_EDGE)
-Reducer 18 <- Map 9 (SIMPLE_EDGE)
+Reducer 15 <- Map 33 (CUSTOM_SIMPLE_EDGE), Reducer 14 (CUSTOM_SIMPLE_EDGE)
+Reducer 16 <- Reducer 15 (CUSTOM_SIMPLE_EDGE), Reducer 17 (CUSTOM_SIMPLE_EDGE)
+Reducer 17 <- Map 9 (SIMPLE_EDGE)
+Reducer 19 <- Map 18 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
 Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 11 (CUSTOM_SIMPLE_EDGE)
-Reducer 20 <- Map 19 (SIMPLE_EDGE), Map 22 (SIMPLE_EDGE)
-Reducer 21 <- Reducer 20 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE)
-Reducer 25 <- Map 30 (SIMPLE_EDGE), Union 24 (SIMPLE_EDGE)
+Reducer 20 <- Reducer 19 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE)
+Reducer 24 <- Map 29 (SIMPLE_EDGE), Union 23 (SIMPLE_EDGE)
+Reducer 25 <- Map 31 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
 Reducer 26 <- Map 32 (SIMPLE_EDGE), Reducer 25 (SIMPLE_EDGE)
-Reducer 27 <- Map 33 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE)
-Reducer 28 <- Reducer 27 (SIMPLE_EDGE)
+Reducer 27 <- Reducer 26 (SIMPLE_EDGE)
 Reducer 3 <- Reducer 13 (CUSTOM_SIMPLE_EDGE), Reducer 2 (CUSTOM_SIMPLE_EDGE)
-Reducer 31 <- Map 30 (CUSTOM_SIMPLE_EDGE)
-Reducer 4 <- Reducer 21 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 30 <- Map 29 (CUSTOM_SIMPLE_EDGE)
+Reducer 4 <- Reducer 20 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 Reducer 5 <- Reducer 16 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
 Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
@@ -165,231 +163,231 @@ Stage-0
     limit:100
     Stage-1
       Reducer 8 vectorized
-      File Output Operator [FS_369]
-        Limit [LIM_368] (rows=1 width=16)
+      File Output Operator [FS_367]
+        Limit [LIM_366] (rows=100 width=16)
           Number of rows:100
-          Select Operator [SEL_367] (rows=1 width=16)
+          Select Operator [SEL_365] (rows=338 width=16)
             Output:["_col0","_col1","_col2"]
           <-Reducer 7 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_366]
-              Select Operator [SEL_365] (rows=1 width=16)
+            SHUFFLE [RS_364]
+              Select Operator [SEL_363] (rows=338 width=16)
                 Output:["_col0","_col1","_col2"]
-                Group By Operator [GBY_364] (rows=1 width=12)
+                Group By Operator [GBY_362] (rows=338 width=12)
                   Output:["_col0","_col1"],aggregations:["count(VALUE._col0)"],keys:KEY._col0
                 <-Reducer 6 [SIMPLE_EDGE] vectorized
-                  SHUFFLE [RS_363]
+                  SHUFFLE [RS_361]
                     PartitionCols:_col0
-                    Group By Operator [GBY_362] (rows=1 width=12)
+                    Group By Operator [GBY_360] (rows=338 width=12)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                      Select Operator [SEL_361] (rows=1 width=116)
+                      Select Operator [SEL_359] (rows=676 width=116)
                         Output:["_col0"]
-                        Group By Operator [GBY_360] (rows=1 width=116)
+                        Group By Operator [GBY_358] (rows=676 width=116)
                           Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                         <-Reducer 5 [SIMPLE_EDGE]
                           SHUFFLE [RS_119]
                             PartitionCols:_col0
-                            Group By Operator [GBY_118] (rows=1 width=116)
+                            Group By Operator [GBY_118] (rows=154128 width=116)
                               Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col5
-                              Merge Join Operator [MERGEJOIN_285] (rows=998805 width=4)
+                              Merge Join Operator [MERGEJOIN_285] (rows=525327388 width=110)
                                 Conds:RS_114._col0=RS_115._col0(Inner),Output:["_col2","_col5"]
                               <-Reducer 16 [SIMPLE_EDGE]
                                 SHUFFLE [RS_115]
                                   PartitionCols:_col0
-                                  Select Operator [SEL_104] (rows=32466 width=12)
+                                  Select Operator [SEL_104] (rows=5072841 width=12)
                                     Output:["_col0"]
-                                    Filter Operator [FIL_103] (rows=32466 width=12)
+                                    Filter Operator [FIL_103] (rows=5072841 width=12)
                                       predicate:(_col1 <= _col3)
-                                      Merge Join Operator [MERGEJOIN_283] (rows=97398 width=12)
+                                      Merge Join Operator [MERGEJOIN_283] (rows=15218525 width=12)
                                         Conds:(Inner),Output:["_col0","_col1","_col3"]
                                       <-Reducer 15 [CUSTOM_SIMPLE_EDGE]
                                         PARTITION_ONLY_SHUFFLE [RS_100]
-                                          Filter Operator [FIL_99] (rows=48699 width=12)
+                                          Filter Operator [FIL_99] (rows=608741 width=12)
                                             predicate:(_col2 <= _col1)
-                                            Merge Join Operator [MERGEJOIN_280] (rows=146098 width=12)
+                                            Merge Join Operator [MERGEJOIN_280] (rows=1826225 width=12)
                                               Conds:(Inner),Output:["_col0","_col1","_col2"]
-                                            <-Map 34 [CUSTOM_SIMPLE_EDGE] vectorized
-                                              PARTITION_ONLY_SHUFFLE [RS_315]
-                                                Select Operator [SEL_314] (rows=73049 width=8)
+                                            <-Map 33 [CUSTOM_SIMPLE_EDGE] vectorized
+                                              PARTITION_ONLY_SHUFFLE [RS_355]
+                                                Select Operator [SEL_354] (rows=73049 width=8)
                                                   Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_313] (rows=73049 width=8)
+                                                  Filter Operator [FIL_353] (rows=73049 width=8)
                                                     predicate:d_month_seq is not null
                                                     TableScan [TS_77] (rows=73049 width=8)
                                                       default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_month_seq"]
                                             <-Reducer 14 [CUSTOM_SIMPLE_EDGE] vectorized
-                                              PARTITION_ONLY_SHUFFLE [RS_312]
-                                                Group By Operator [GBY_311] (rows=2 width=4)
+                                              PARTITION_ONLY_SHUFFLE [RS_352]
+                                                Group By Operator [GBY_351] (rows=25 width=4)
                                                   Output:["_col0"],keys:KEY._col0
                                                 <-Map 9 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_309]
+                                                  SHUFFLE [RS_312]
                                                     PartitionCols:_col0
-                                                    Group By Operator [GBY_305] (rows=2 width=4)
+                                                    Group By Operator [GBY_308] (rows=25 width=4)
                                                       Output:["_col0"],keys:_col0
-                                                      Select Operator [SEL_301] (rows=50 width=12)
+                                                      Select Operator [SEL_304] (rows=50 width=12)
                                                         Output:["_col0"]
-                                                        Filter Operator [FIL_297] (rows=50 width=12)
+                                                        Filter Operator [FIL_300] (rows=50 width=12)
                                                           predicate:((d_year = 1999) and (d_moy = 3) and d_month_seq is not null)
                                                           TableScan [TS_3] (rows=73049 width=12)
                                                             default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_month_seq","d_year","d_moy"]
-                                      <-Reducer 18 [CUSTOM_SIMPLE_EDGE] vectorized
-                                        PARTITION_ONLY_SHUFFLE [RS_317]
-                                          Group By Operator [GBY_316] (rows=2 width=4)
+                                      <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
+                                        PARTITION_ONLY_SHUFFLE [RS_357]
+                                          Group By Operator [GBY_356] (rows=25 width=4)
                                             Output:["_col0"],keys:KEY._col0
                                           <-Map 9 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_310]
+                                            SHUFFLE [RS_313]
                                               PartitionCols:_col0
-                                              Group By Operator [GBY_306] (rows=2 width=4)
+                                              Group By Operator [GBY_309] (rows=25 width=4)
                                                 Output:["_col0"],keys:_col0
-                                                Select Operator [SEL_302] (rows=50 width=12)
+                                                Select Operator [SEL_305] (rows=50 width=12)
                                                   Output:["_col0"]
-                                                  Filter Operator [FIL_298] (rows=50 width=12)
+                                                  Filter Operator [FIL_301] (rows=50 width=12)
                                                     predicate:((d_year = 1999) and (d_moy = 3) and d_month_seq is not null)
                                                      Please refer to the previous TableScan [TS_3]
                               <-Reducer 4 [SIMPLE_EDGE]
                                 SHUFFLE [RS_114]
                                   PartitionCols:_col0
-                                  Merge Join Operator [MERGEJOIN_284] (rows=8989304 width=4)
+                                  Merge Join Operator [MERGEJOIN_284] (rows=525327388 width=114)
                                     Conds:RS_111._col1=RS_112._col0(Inner),Output:["_col0","_col2","_col5"]
-                                  <-Reducer 21 [SIMPLE_EDGE]
+                                  <-Reducer 20 [SIMPLE_EDGE]
                                     SHUFFLE [RS_112]
                                       PartitionCols:_col0
-                                      Select Operator [SEL_76] (rows=55046 width=4)
+                                      Select Operator [SEL_76] (rows=4049224 width=4)
                                         Output:["_col0"]
-                                        Merge Join Operator [MERGEJOIN_282] (rows=55046 width=4)
-                                          Conds:RS_73._col0=RS_359._col1(Inner),Output:["_col5"]
-                                        <-Reducer 20 [SIMPLE_EDGE]
+                                        Merge Join Operator [MERGEJOIN_282] (rows=4049224 width=4)
+                                          Conds:RS_73._col0=RS_350._col1(Inner),Output:["_col5"]
+                                        <-Reducer 19 [SIMPLE_EDGE]
                                           SHUFFLE [RS_73]
                                             PartitionCols:_col0
                                             Merge Join Operator [MERGEJOIN_276] (rows=39720279 width=4)
-                                              Conds:RS_341._col1, _col2=RS_344._col0, _col1(Inner),Output:["_col0"]
-                                            <-Map 19 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_341]
+                                              Conds:RS_332._col1, _col2=RS_335._col0, _col1(Inner),Output:["_col0"]
+                                            <-Map 18 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_332]
                                                 PartitionCols:_col1, _col2
-                                                Select Operator [SEL_340] (rows=40000000 width=188)
+                                                Select Operator [SEL_331] (rows=40000000 width=188)
                                                   Output:["_col0","_col1","_col2"]
-                                                  Filter Operator [FIL_339] (rows=40000000 width=188)
+                                                  Filter Operator [FIL_330] (rows=40000000 width=188)
                                                     predicate:(ca_county is not null and ca_state is not null)
                                                     TableScan [TS_33] (rows=40000000 width=188)
                                                       default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_county","ca_state"]
-                                            <-Map 22 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_344]
+                                            <-Map 21 [SIMPLE_EDGE] vectorized
+                                              SHUFFLE [RS_335]
                                                 PartitionCols:_col0, _col1
-                                                Select Operator [SEL_343] (rows=1704 width=184)
+                                                Select Operator [SEL_334] (rows=1704 width=184)
                                                   Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_342] (rows=1704 width=184)
+                                                  Filter Operator [FIL_333] (rows=1704 width=184)
                                                     predicate:(s_county is not null and s_state is not null)
                                                     TableScan [TS_36] (rows=1704 width=184)
                                                       default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_county","s_state"]
-                                        <-Reducer 28 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_359]
+                                        <-Reducer 27 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_350]
                                             PartitionCols:_col1
-                                            Select Operator [SEL_358] (rows=55046 width=8)
+                                            Select Operator [SEL_349] (rows=4049224 width=8)
                                               Output:["_col0","_col1"]
-                                              Group By Operator [GBY_357] (rows=55046 width=8)
+                                              Group By Operator [GBY_348] (rows=4049224 width=8)
                                                 Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                              <-Reducer 27 [SIMPLE_EDGE]
+                                              <-Reducer 26 [SIMPLE_EDGE]
                                                 SHUFFLE [RS_67]
                                                   PartitionCols:_col0, _col1
-                                                  Group By Operator [GBY_66] (rows=55046 width=8)
+                                                  Group By Operator [GBY_66] (rows=4049224 width=8)
                                                     Output:["_col0","_col1"],keys:_col6, _col5
-                                                    Merge Join Operator [MERGEJOIN_279] (rows=110092 width=8)
-                                                      Conds:RS_62._col1=RS_356._col0(Inner),Output:["_col5","_col6"]
-                                                    <-Map 33 [SIMPLE_EDGE] vectorized
-                                                      SHUFFLE [RS_356]
+                                                    Merge Join Operator [MERGEJOIN_279] (rows=4049224 width=8)
+                                                      Conds:RS_62._col1=RS_347._col0(Inner),Output:["_col5","_col6"]
+                                                    <-Map 32 [SIMPLE_EDGE] vectorized
+                                                      SHUFFLE [RS_347]
                                                         PartitionCols:_col0
-                                                        Select Operator [SEL_355] (rows=80000000 width=8)
+                                                        Select Operator [SEL_346] (rows=80000000 width=8)
                                                           Output:["_col0","_col1"]
-                                                          Filter Operator [FIL_354] (rows=80000000 width=8)
+                                                          Filter Operator [FIL_345] (rows=80000000 width=8)
                                                             predicate:c_current_addr_sk is not null
                                                             TableScan [TS_53] (rows=80000000 width=8)
                                                               default@customer,customer,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_current_addr_sk"]
-                                                    <-Reducer 26 [SIMPLE_EDGE]
+                                                    <-Reducer 25 [SIMPLE_EDGE]
                                                       SHUFFLE [RS_62]
                                                         PartitionCols:_col1
-                                                        Merge Join Operator [MERGEJOIN_278] (rows=110092 width=0)
-                                                          Conds:RS_59._col2=RS_353._col0(Inner),Output:["_col1"]
-                                                        <-Map 32 [SIMPLE_EDGE] vectorized
-                                                          SHUFFLE [RS_353]
+                                                        Merge Join Operator [MERGEJOIN_278] (rows=4049224 width=1)
+                                                          Conds:RS_59._col2=RS_344._col0(Inner),Output:["_col1"]
+                                                        <-Map 31 [SIMPLE_EDGE] vectorized
+                                                          SHUFFLE [RS_344]
                                                             PartitionCols:_col0
-                                                            Select Operator [SEL_352] (rows=453 width=4)
+                                                            Select Operator [SEL_343] (rows=453 width=4)
                                                               Output:["_col0"]
-                                                              Filter Operator [FIL_351] (rows=453 width=186)
+                                                              Filter Operator [FIL_342] (rows=453 width=186)
                                                                 predicate:((i_class = 'consignment') and (i_category = 'Jewelry'))
                                                                 TableScan [TS_50] (rows=462000 width=186)
                                                                   default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_class","i_category"]
-                                                        <-Reducer 25 [SIMPLE_EDGE]
+                                                        <-Reducer 24 [SIMPLE_EDGE]
                                                           SHUFFLE [RS_59]
                                                             PartitionCols:_col2
-                                                            Merge Join Operator [MERGEJOIN_277] (rows=11665117 width=7)
-                                                              Conds:Union 24._col0=RS_347._col0(Inner),Output:["_col1","_col2"]
-                                                            <-Map 30 [SIMPLE_EDGE] vectorized
-                                                              PARTITION_ONLY_SHUFFLE [RS_347]
+                                                            Merge Join Operator [MERGEJOIN_277] (rows=429048824 width=7)
+                                                              Conds:Union 23._col0=RS_338._col0(Inner),Output:["_col1","_col2"]
+                                                            <-Map 29 [SIMPLE_EDGE] vectorized
+                                                              PARTITION_ONLY_SHUFFLE [RS_338]
                                                                 PartitionCols:_col0
-                                                                Select Operator [SEL_346] (rows=50 width=4)
+                                                                Select Operator [SEL_337] (rows=50 width=4)
                                                                   Output:["_col0"]
-                                                                  Filter Operator [FIL_345] (rows=50 width=12)
+                                                                  Filter Operator [FIL_336] (rows=50 width=12)
                                                                     predicate:((d_year = 1999) and (d_moy = 3))
                                                                     TableScan [TS_47] (rows=73049 width=12)
                                                                       default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
-                                                            <-Union 24 [SIMPLE_EDGE]
-                                                              <-Map 23 [CONTAINS] vectorized
-                                                                Reduce Output Operator [RS_375]
+                                                            <-Union 23 [SIMPLE_EDGE]
+                                                              <-Map 22 [CONTAINS] vectorized
+                                                                Reduce Output Operator [RS_373]
                                                                   PartitionCols:_col0
-                                                                  Select Operator [SEL_374] (rows=285117831 width=11)
+                                                                  Select Operator [SEL_372] (rows=285117831 width=11)
                                                                     Output:["_col0","_col1","_col2"]
-                                                                    Filter Operator [FIL_373] (rows=285117831 width=11)
+                                                                    Filter Operator [FIL_371] (rows=285117831 width=11)
                                                                       predicate:(cs_sold_date_sk is not null and cs_bill_customer_sk is not null and cs_sold_date_sk BETWEEN DynamicValue(RS_57_date_dim_d_date_sk_min) AND DynamicValue(RS_57_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_57_date_dim_d_date_sk_bloom_filter)))
                                                                       TableScan [TS_286] (rows=287989836 width=11)
                                                                         Output:["cs_sold_date_sk","cs_bill_customer_sk","cs_item_sk"]
-                                                                      <-Reducer 31 [BROADCAST_EDGE] vectorized
-                                                                        BROADCAST [RS_371]
-                                                                          Group By Operator [GBY_370] (rows=1 width=12)
+                                                                      <-Reducer 30 [BROADCAST_EDGE] vectorized
+                                                                        BROADCAST [RS_369]
+                                                                          Group By Operator [GBY_368] (rows=1 width=12)
                                                                             Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                          <-Map 30 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                            PARTITION_ONLY_SHUFFLE [RS_350]
-                                                                              Group By Operator [GBY_349] (rows=1 width=12)
+                                                                          <-Map 29 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                            PARTITION_ONLY_SHUFFLE [RS_341]
+                                                                              Group By Operator [GBY_340] (rows=1 width=12)
                                                                                 Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                                Select Operator [SEL_348] (rows=50 width=4)
+                                                                                Select Operator [SEL_339] (rows=50 width=4)
                                                                                   Output:["_col0"]
-                                                                                   Please refer to the previous Select Operator [SEL_346]
-                                                              <-Map 29 [CONTAINS] vectorized
-                                                                Reduce Output Operator [RS_378]
+                                                                                   Please refer to the previous Select Operator [SEL_337]
+                                                              <-Map 28 [CONTAINS] vectorized
+                                                                Reduce Output Operator [RS_376]
                                                                   PartitionCols:_col0
-                                                                  Select Operator [SEL_377] (rows=143930993 width=11)
+                                                                  Select Operator [SEL_375] (rows=143930993 width=11)
                                                                     Output:["_col0","_col1","_col2"]
-                                                                    Filter Operator [FIL_376] (rows=143930993 width=11)
+                                                                    Filter Operator [FIL_374] (rows=143930993 width=11)
                                                                       predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_sold_date_sk BETWEEN DynamicValue(RS_57_date_dim_d_date_sk_min) AND DynamicValue(RS_57_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_57_date_dim_d_date_sk_bloom_filter)))
                                                                       TableScan [TS_291] (rows=144002668 width=11)
                                                                         Output:["ws_sold_date_sk","ws_item_sk","ws_bill_customer_sk"]
-                                                                      <-Reducer 31 [BROADCAST_EDGE] vectorized
-                                                                        BROADCAST [RS_372]
-                                                                           Please refer to the previous Group By Operator [GBY_370]
+                                                                      <-Reducer 30 [BROADCAST_EDGE] vectorized
+                                                                        BROADCAST [RS_370]
+                                                                           Please refer to the previous Group By Operator [GBY_368]
                                   <-Reducer 3 [SIMPLE_EDGE]
                                     SHUFFLE [RS_111]
                                       PartitionCols:_col1
                                       Merge Join Operator [MERGEJOIN_281] (rows=525327388 width=114)
                                         Conds:(Inner),Output:["_col0","_col1","_col2"]
                                       <-Reducer 13 [CUSTOM_SIMPLE_EDGE] vectorized
-                                        PARTITION_ONLY_SHUFFLE [RS_338]
-                                          Select Operator [SEL_337] (rows=1 width=8)
-                                            Filter Operator [FIL_336] (rows=1 width=8)
+                                        PARTITION_ONLY_SHUFFLE [RS_329]
+                                          Select Operator [SEL_328] (rows=1 width=8)
+                                            Filter Operator [FIL_327] (rows=1 width=8)
                                               predicate:(sq_count_check(_col0) <= 1)
-                                              Group By Operator [GBY_335] (rows=1 width=8)
+                                              Group By Operator [GBY_326] (rows=1 width=8)
                                                 Output:["_col0"],aggregations:["count(VALUE._col0)"]
                                               <-Reducer 12 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                PARTITION_ONLY_SHUFFLE [RS_334]
-                                                  Group By Operator [GBY_333] (rows=1 width=8)
+                                                PARTITION_ONLY_SHUFFLE [RS_325]
+                                                  Group By Operator [GBY_324] (rows=1 width=8)
                                                     Output:["_col0"],aggregations:["count()"]
-                                                    Select Operator [SEL_332] (rows=25 width=4)
-                                                      Group By Operator [GBY_331] (rows=25 width=4)
+                                                    Select Operator [SEL_323] (rows=25 width=4)
+                                                      Group By Operator [GBY_322] (rows=25 width=4)
                                                         Output:["_col0"],keys:KEY._col0
                                                       <-Map 9 [SIMPLE_EDGE] vectorized
-                                                        SHUFFLE [RS_308]
+                                                        SHUFFLE [RS_311]
                                                           PartitionCols:_col0
-                                                          Group By Operator [GBY_304] (rows=25 width=4)
+                                                          Group By Operator [GBY_307] (rows=25 width=4)
                                                             Output:["_col0"],keys:_col0
-                                                            Select Operator [SEL_300] (rows=50 width=12)
+                                                            Select Operator [SEL_303] (rows=50 width=12)
                                                               Output:["_col0"]
-                                                              Filter Operator [FIL_296] (rows=50 width=12)
+                                                              Filter Operator [FIL_299] (rows=50 width=12)
                                                                 predicate:((d_year = 1999) and (d_moy = 3))
                                                                  Please refer to the previous TableScan [TS_3]
                                       <-Reducer 2 [CUSTOM_SIMPLE_EDGE]
@@ -397,44 +395,33 @@ Stage-0
                                           Merge Join Operator [MERGEJOIN_275] (rows=525327388 width=114)
                                             Conds:(Inner),Output:["_col0","_col1","_col2"]
                                           <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
-                                            PARTITION_ONLY_SHUFFLE [RS_322]
-                                              Select Operator [SEL_321] (rows=525327388 width=114)
+                                            PARTITION_ONLY_SHUFFLE [RS_298]
+                                              Select Operator [SEL_297] (rows=525327388 width=114)
                                                 Output:["_col0","_col1","_col2"]
-                                                Filter Operator [FIL_320] (rows=525327388 width=114)
-                                                  predicate:(ss_sold_date_sk is not null and ss_customer_sk is not null and ss_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(ss_sold_date_sk, DynamicValue(RS_115_date_dim_d_date_sk_bloom_filter)))
+                                                Filter Operator [FIL_296] (rows=525327388 width=114)
+                                                  predicate:(ss_sold_date_sk is not null and ss_customer_sk is not null)
                                                   TableScan [TS_0] (rows=575995635 width=114)
                                                     default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_customer_sk","ss_ext_sales_price"]
-                                                  <-Reducer 17 [BROADCAST_EDGE] vectorized
-                                                    BROADCAST [RS_319]
-                                                      Group By Operator [GBY_318] (rows=1 width=12)
-                                                        Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                      <-Reducer 16 [CUSTOM_SIMPLE_EDGE]
-                                                        SHUFFLE [RS_205]
-                                                          Group By Operator [GBY_204] (rows=1 width=12)
-                                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                            Select Operator [SEL_203] (rows=32466 width=4)
-                                                              Output:["_col0"]
-                                                               Please refer to the previous Select Operator [SEL_104]
                                           <-Reducer 11 [CUSTOM_SIMPLE_EDGE] vectorized
-                                            PARTITION_ONLY_SHUFFLE [RS_330]
-                                              Select Operator [SEL_329] (rows=1 width=8)
-                                                Filter Operator [FIL_328] (rows=1 width=8)
+                                            PARTITION_ONLY_SHUFFLE [RS_321]
+                                              Select Operator [SEL_320] (rows=1 width=8)
+                                                Filter Operator [FIL_319] (rows=1 width=8)
                                                   predicate:(sq_count_check(_col0) <= 1)
-                                                  Group By Operator [GBY_327] (rows=1 width=8)
+                                                  Group By Operator [GBY_318] (rows=1 width=8)
                                                     Output:["_col0"],aggregations:["count(VALUE._col0)"]
                                                   <-Reducer 10 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                    PARTITION_ONLY_SHUFFLE [RS_326]
-                                                      Group By Operator [GBY_325] (rows=1 width=8)
+                                                    PARTITION_ONLY_SHUFFLE [RS_317]
+                                                      Group By Operator [GBY_316] (rows=1 width=8)
                                                         Output:["_col0"],aggregations:["count()"]
-                                                        Select Operator [SEL_324] (rows=25 width=4)
-                                                          Group By Operator [GBY_323] (rows=25 width=4)
+                                                        Select Operator [SEL_315] (rows=25 width=4)
+                                                          Group By Operator [GBY_314] (rows=25 width=4)
                                                             Output:["_col0"],keys:KEY._col0
                                                           <-Map 9 [SIMPLE_EDGE] vectorized
-                                                            SHUFFLE [RS_307]
+                                                            SHUFFLE [RS_310]
                                                               PartitionCols:_col0
-                                                              Group By Operator [GBY_303] (rows=25 width=4)
+                                                              Group By Operator [GBY_306] (rows=25 width=4)
                                                                 Output:["_col0"],keys:_col0
-                                                                Select Operator [SEL_299] (rows=50 width=12)
+                                                                Select Operator [SEL_302] (rows=50 width=12)
                                                                   Output:["_col0"]
-                                                                   Please refer to the previous Filter Operator [FIL_296]
+                                                                   Please refer to the previous Filter Operator [FIL_299]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query55.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query55.q.out
index 3548ce7..fb52de2 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query55.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query55.q.out
@@ -63,7 +63,7 @@ Stage-0
                     PartitionCols:_col0, _col1
                     Group By Operator [GBY_16] (rows=7333 width=216)
                       Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col5, _col6
-                      Merge Join Operator [MERGEJOIN_54] (rows=2301098 width=104)
+                      Merge Join Operator [MERGEJOIN_54] (rows=84037218 width=183)
                         Conds:RS_12._col1=RS_68._col0(Inner),Output:["_col2","_col5","_col6"]
                       <-Map 8 [SIMPLE_EDGE] vectorized
                         SHUFFLE [RS_68]
@@ -77,7 +77,7 @@ Stage-0
                       <-Reducer 2 [SIMPLE_EDGE]
                         SHUFFLE [RS_12]
                           PartitionCols:_col1
-                          Merge Join Operator [MERGEJOIN_53] (rows=15062131 width=4)
+                          Merge Join Operator [MERGEJOIN_53] (rows=550076554 width=110)
                             Conds:RS_65._col0=RS_57._col0(Inner),Output:["_col1","_col2"]
                           <-Map 6 [SIMPLE_EDGE] vectorized
                             PARTITION_ONLY_SHUFFLE [RS_57]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query56.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query56.q.out
index 69f21bd..cd438d4 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query56.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query56.q.out
@@ -197,9 +197,9 @@ Stage-0
                       <-Reducer 10 [SIMPLE_EDGE]
                         SHUFFLE [RS_106]
                           PartitionCols:_col0
-                          Group By Operator [GBY_105] (rows=355 width=212)
+                          Group By Operator [GBY_105] (rows=9585 width=212)
                             Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                            Merge Join Operator [MERGEJOIN_301] (rows=172427 width=188)
+                            Merge Join Operator [MERGEJOIN_301] (rows=31485548 width=211)
                               Conds:RS_101._col0=RS_102._col2(Inner),Output:["_col1","_col7"]
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_101]
@@ -232,9 +232,9 @@ Stage-0
                             <-Reducer 23 [SIMPLE_EDGE]
                               SHUFFLE [RS_102]
                                 PartitionCols:_col2
-                                Select Operator [SEL_97] (rows=788222 width=110)
+                                Select Operator [SEL_97] (rows=143931246 width=115)
                                   Output:["_col2","_col4"]
-                                  Merge Join Operator [MERGEJOIN_298] (rows=788222 width=110)
+                                  Merge Join Operator [MERGEJOIN_298] (rows=143931246 width=115)
                                     Conds:RS_94._col2=RS_345._col0(Inner),Output:["_col1","_col3"]
                                   <-Map 25 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_345]
@@ -248,7 +248,7 @@ Stage-0
                                   <-Reducer 22 [SIMPLE_EDGE]
                                     SHUFFLE [RS_94]
                                       PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_297] (rows=3941109 width=118)
+                                      Merge Join Operator [MERGEJOIN_297] (rows=143931246 width=119)
                                         Conds:RS_366._col0=RS_328._col0(Inner),Output:["_col1","_col2","_col3"]
                                       <-Map 17 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_328]
@@ -289,9 +289,9 @@ Stage-0
                       <-Reducer 3 [SIMPLE_EDGE]
                         SHUFFLE [RS_33]
                           PartitionCols:_col0
-                          Group By Operator [GBY_32] (rows=355 width=212)
+                          Group By Operator [GBY_32] (rows=30175 width=212)
                             Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                            Merge Join Operator [MERGEJOIN_299] (rows=629332 width=100)
+                            Merge Join Operator [MERGEJOIN_299] (rows=114917468 width=188)
                               Conds:RS_28._col0=RS_29._col2(Inner),Output:["_col1","_col7"]
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_28]
@@ -300,9 +300,9 @@ Stage-0
                             <-Reducer 16 [SIMPLE_EDGE]
                               SHUFFLE [RS_29]
                                 PartitionCols:_col2
-                                Select Operator [SEL_24] (rows=2876890 width=4)
+                                Select Operator [SEL_24] (rows=525327191 width=110)
                                   Output:["_col2","_col4"]
-                                  Merge Join Operator [MERGEJOIN_292] (rows=2876890 width=4)
+                                  Merge Join Operator [MERGEJOIN_292] (rows=525327191 width=110)
                                     Conds:RS_21._col2=RS_343._col0(Inner),Output:["_col1","_col3"]
                                   <-Map 25 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_343]
@@ -311,7 +311,7 @@ Stage-0
                                   <-Reducer 15 [SIMPLE_EDGE]
                                     SHUFFLE [RS_21]
                                       PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_291] (rows=14384447 width=4)
+                                      Merge Join Operator [MERGEJOIN_291] (rows=525327191 width=114)
                                         Conds:RS_340._col0=RS_324._col0(Inner),Output:["_col1","_col2","_col3"]
                                       <-Map 17 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_324]
@@ -347,9 +347,9 @@ Stage-0
                       <-Reducer 8 [SIMPLE_EDGE]
                         SHUFFLE [RS_69]
                           PartitionCols:_col0
-                          Group By Operator [GBY_68] (rows=355 width=212)
+                          Group By Operator [GBY_68] (rows=18460 width=212)
                             Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                            Merge Join Operator [MERGEJOIN_300] (rows=339151 width=100)
+                            Merge Join Operator [MERGEJOIN_300] (rows=62370668 width=209)
                               Conds:RS_64._col0=RS_65._col3(Inner),Output:["_col1","_col7"]
                             <-Reducer 2 [SIMPLE_EDGE]
                               SHUFFLE [RS_64]
@@ -358,9 +358,9 @@ Stage-0
                             <-Reducer 20 [SIMPLE_EDGE]
                               SHUFFLE [RS_65]
                                 PartitionCols:_col3
-                                Select Operator [SEL_60] (rows=1550375 width=13)
+                                Select Operator [SEL_60] (rows=285117733 width=115)
                                   Output:["_col3","_col4"]
-                                  Merge Join Operator [MERGEJOIN_295] (rows=1550375 width=13)
+                                  Merge Join Operator [MERGEJOIN_295] (rows=285117733 width=115)
                                     Conds:RS_57._col1=RS_344._col0(Inner),Output:["_col2","_col3"]
                                   <-Map 25 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_344]
@@ -369,7 +369,7 @@ Stage-0
                                   <-Reducer 19 [SIMPLE_EDGE]
                                     SHUFFLE [RS_57]
                                       PartitionCols:_col1
-                                      Merge Join Operator [MERGEJOIN_294] (rows=7751872 width=98)
+                                      Merge Join Operator [MERGEJOIN_294] (rows=285117733 width=119)
                                         Conds:RS_358._col0=RS_326._col0(Inner),Output:["_col1","_col2","_col3"]
                                       <-Map 17 [SIMPLE_EDGE] vectorized
                                         PARTITION_ONLY_SHUFFLE [RS_326]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query57.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query57.q.out
index 6b1a27e..7896f97 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query57.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query57.q.out
@@ -153,9 +153,9 @@ Stage-0
                               <-Reducer 4 [SIMPLE_EDGE]
                                 SHUFFLE [RS_93]
                                   PartitionCols:_col0, _col1, _col2, _col3, _col4
-                                  Group By Operator [GBY_92] (rows=87441185 width=408)
+                                  Group By Operator [GBY_92] (rows=285117980 width=408)
                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col3)"],keys:_col5, _col6, _col8, _col10, _col11
-                                    Merge Join Operator [MERGEJOIN_276] (rows=87441185 width=406)
+                                    Merge Join Operator [MERGEJOIN_276] (rows=285117980 width=407)
                                       Conds:RS_88._col2=RS_295._col0(Inner),Output:["_col3","_col5","_col6","_col8","_col10","_col11"]
                                     <-Map 15 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_295]
@@ -169,7 +169,7 @@ Stage-0
                                     <-Reducer 3 [SIMPLE_EDGE]
                                       SHUFFLE [RS_88]
                                         PartitionCols:_col2
-                                        Merge Join Operator [MERGEJOIN_275] (rows=87441185 width=220)
+                                        Merge Join Operator [MERGEJOIN_275] (rows=285117980 width=221)
                                           Conds:RS_85._col1=RS_292._col0(Inner),Output:["_col2","_col3","_col5","_col6","_col8"]
                                         <-Map 14 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_292]
@@ -183,7 +183,7 @@ Stage-0
                                         <-Reducer 2 [SIMPLE_EDGE]
                                           SHUFFLE [RS_85]
                                             PartitionCols:_col1
-                                            Merge Join Operator [MERGEJOIN_274] (rows=87441185 width=126)
+                                            Merge Join Operator [MERGEJOIN_274] (rows=285117980 width=127)
                                               Conds:RS_289._col0=RS_281._col0(Inner),Output:["_col1","_col2","_col3","_col5","_col6"]
                                             <-Map 12 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_281]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query58.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query58.q.out
index 0193e7d..5fa126b 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query58.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query58.q.out
@@ -182,21 +182,21 @@ Stage-0
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
                 Filter Operator [FIL_152] (rows=1 width=660)
                   predicate:(_col5 BETWEEN (0.9 * _col1) AND (1.1 * _col1) and _col5 BETWEEN (0.9 * _col3) AND (1.1 * _col3) and _col1 BETWEEN _col6 AND _col7 and _col3 BETWEEN _col6 AND _col7)
-                  Merge Join Operator [MERGEJOIN_422] (rows=384 width=660)
+                  Merge Join Operator [MERGEJOIN_422] (rows=3836 width=660)
                     Conds:RS_149._col0=RS_467._col0(Inner),Output:["_col0","_col1","_col3","_col5","_col6","_col7"]
                   <-Reducer 16 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_467]
                       PartitionCols:_col0
-                      Select Operator [SEL_466] (rows=15768 width=436)
+                      Select Operator [SEL_466] (rows=310774 width=436)
                         Output:["_col0","_col1","_col2","_col3"]
-                        Group By Operator [GBY_465] (rows=15768 width=212)
+                        Group By Operator [GBY_465] (rows=310774 width=212)
                           Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                         <-Reducer 15 [SIMPLE_EDGE]
                           SHUFFLE [RS_141]
                             PartitionCols:_col0
-                            Group By Operator [GBY_140] (rows=15768 width=212)
+                            Group By Operator [GBY_140] (rows=37292880 width=212)
                               Output:["_col0","_col1"],aggregations:["sum(_col5)"],keys:_col7
-                              Merge Join Operator [MERGEJOIN_420] (rows=31537 width=100)
+                              Merge Join Operator [MERGEJOIN_420] (rows=143966864 width=211)
                                 Conds:RS_136._col4=RS_450._col0(Inner),Output:["_col5","_col7"]
                               <-Map 25 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_450]
@@ -208,7 +208,7 @@ Stage-0
                               <-Reducer 14 [SIMPLE_EDGE]
                                 SHUFFLE [RS_136]
                                   PartitionCols:_col4
-                                  Merge Join Operator [MERGEJOIN_419] (rows=31537 width=4)
+                                  Merge Join Operator [MERGEJOIN_419] (rows=143966864 width=115)
                                     Conds:RS_133._col0=RS_464._col0(Inner),Output:["_col4","_col5"]
                                   <-Reducer 2 [SIMPLE_EDGE]
                                     PARTITION_ONLY_SHUFFLE [RS_133]
@@ -296,21 +296,21 @@ Stage-0
                   <-Reducer 6 [SIMPLE_EDGE]
                     SHUFFLE [RS_149]
                       PartitionCols:_col0
-                      Filter Operator [FIL_147] (rows=384 width=324)
+                      Filter Operator [FIL_147] (rows=3836 width=324)
                         predicate:(_col1 BETWEEN (0.9 * _col3) AND (1.1 * _col3) and _col3 BETWEEN (0.9 * _col1) AND (1.1 * _col1))
-                        Merge Join Operator [MERGEJOIN_421] (rows=31163 width=324)
+                        Merge Join Operator [MERGEJOIN_421] (rows=310774 width=324)
                           Conds:RS_452._col0=RS_459._col0(Inner),Output:["_col0","_col1","_col3"]
                         <-Reducer 12 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_459]
                             PartitionCols:_col0
-                            Group By Operator [GBY_458] (rows=60249 width=212)
+                            Group By Operator [GBY_458] (rows=310774 width=212)
                               Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                             <-Reducer 11 [SIMPLE_EDGE]
                               SHUFFLE [RS_93]
                                 PartitionCols:_col0
-                                Group By Operator [GBY_92] (rows=60249 width=212)
+                                Group By Operator [GBY_92] (rows=138294430 width=212)
                                   Output:["_col0","_col1"],aggregations:["sum(_col5)"],keys:_col7
-                                  Merge Join Operator [MERGEJOIN_415] (rows=120498 width=100)
+                                  Merge Join Operator [MERGEJOIN_415] (rows=550076554 width=206)
                                     Conds:RS_88._col4=RS_449._col0(Inner),Output:["_col5","_col7"]
                                   <-Map 25 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_449]
@@ -319,7 +319,7 @@ Stage-0
                                   <-Reducer 10 [SIMPLE_EDGE]
                                     SHUFFLE [RS_88]
                                       PartitionCols:_col4
-                                      Merge Join Operator [MERGEJOIN_414] (rows=120498 width=4)
+                                      Merge Join Operator [MERGEJOIN_414] (rows=550076554 width=110)
                                         Conds:RS_85._col0=RS_457._col0(Inner),Output:["_col4","_col5"]
                                       <-Reducer 2 [SIMPLE_EDGE]
                                         PARTITION_ONLY_SHUFFLE [RS_85]
@@ -348,14 +348,14 @@ Stage-0
                         <-Reducer 5 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_452]
                             PartitionCols:_col0
-                            Group By Operator [GBY_451] (rows=31163 width=212)
+                            Group By Operator [GBY_451] (rows=310774 width=212)
                               Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
                             <-Reducer 4 [SIMPLE_EDGE]
                               SHUFFLE [RS_45]
                                 PartitionCols:_col0
-                                Group By Operator [GBY_44] (rows=31163 width=212)
+                                Group By Operator [GBY_44] (rows=73653438 width=212)
                                   Output:["_col0","_col1"],aggregations:["sum(_col5)"],keys:_col7
-                                  Merge Join Operator [MERGEJOIN_410] (rows=62327 width=100)
+                                  Merge Join Operator [MERGEJOIN_410] (rows=286549727 width=211)
                                     Conds:RS_40._col4=RS_448._col0(Inner),Output:["_col5","_col7"]
                                   <-Map 25 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_448]
@@ -364,7 +364,7 @@ Stage-0
                                   <-Reducer 3 [SIMPLE_EDGE]
                                     SHUFFLE [RS_40]
                                       PartitionCols:_col4
-                                      Merge Join Operator [MERGEJOIN_409] (rows=62327 width=4)
+                                      Merge Join Operator [MERGEJOIN_409] (rows=286549727 width=115)
                                         Conds:RS_37._col0=RS_446._col0(Inner),Output:["_col4","_col5"]
                                       <-Reducer 2 [SIMPLE_EDGE]
                                         PARTITION_ONLY_SHUFFLE [RS_37]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query6.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query6.q.out
index e783704..d603115 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query6.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query6.q.out
@@ -99,22 +99,22 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_68] (rows=1 width=94)
                       Output:["_col0","_col1"],aggregations:["count()"],keys:_col4
-                      Map Join Operator [MAPJOIN_175] (rows=40 width=86)
+                      Map Join Operator [MAPJOIN_175] (rows=36482 width=86)
                         Conds:MAPJOIN_174._col5=RS_220._col0(Inner),Output:["_col4"]
                       <-Map 15 [BROADCAST_EDGE] vectorized
                         BROADCAST [RS_220]
                           PartitionCols:_col0
-                          Map Join Operator [MAPJOIN_219] (rows=52 width=4)
+                          Map Join Operator [MAPJOIN_219] (rows=660 width=4)
                             Conds:SEL_218._col1=RS_216._col0(Inner),Output:["_col0"]
                           <-Reducer 8 [BROADCAST_EDGE] vectorized
                             BROADCAST [RS_216]
                               PartitionCols:_col0
-                              Group By Operator [GBY_215] (rows=2 width=4)
+                              Group By Operator [GBY_215] (rows=25 width=4)
                                 Output:["_col0"],keys:KEY._col0
                               <-Map 5 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_188]
                                   PartitionCols:_col0
-                                  Group By Operator [GBY_186] (rows=2 width=4)
+                                  Group By Operator [GBY_186] (rows=25 width=4)
                                     Output:["_col0"],keys:d_month_seq
                                     Select Operator [SEL_184] (rows=50 width=12)
                                       Output:["d_month_seq"]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query60.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query60.q.out
index ba967b5..372d25a 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query60.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query60.q.out
@@ -219,9 +219,9 @@ Stage-0
                         <-Reducer 10 [SIMPLE_EDGE]
                           SHUFFLE [RS_106]
                             PartitionCols:_col0
-                            Group By Operator [GBY_105] (rows=1717 width=212)
+                            Group By Operator [GBY_105] (rows=99586 width=212)
                               Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                              Merge Join Operator [MERGEJOIN_302] (rows=379339 width=201)
+                              Merge Join Operator [MERGEJOIN_302] (rows=69268204 width=211)
                                 Conds:RS_101._col0=RS_102._col2(Inner),Output:["_col1","_col7"]
                               <-Reducer 2 [SIMPLE_EDGE]
                                 SHUFFLE [RS_101]
@@ -254,9 +254,9 @@ Stage-0
                               <-Reducer 23 [SIMPLE_EDGE]
                                 SHUFFLE [RS_102]
                                   PartitionCols:_col2
-                                  Select Operator [SEL_97] (rows=788222 width=110)
+                                  Select Operator [SEL_97] (rows=143931246 width=115)
                                     Output:["_col2","_col4"]
-                                    Merge Join Operator [MERGEJOIN_299] (rows=788222 width=110)
+                                    Merge Join Operator [MERGEJOIN_299] (rows=143931246 width=115)
                                       Conds:RS_94._col2=RS_349._col0(Inner),Output:["_col1","_col3"]
                                     <-Map 25 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_349]
@@ -270,7 +270,7 @@ Stage-0
                                     <-Reducer 22 [SIMPLE_EDGE]
                                       SHUFFLE [RS_94]
                                         PartitionCols:_col2
-                                        Merge Join Operator [MERGEJOIN_298] (rows=3941109 width=118)
+                                        Merge Join Operator [MERGEJOIN_298] (rows=143931246 width=119)
                                           Conds:RS_372._col0=RS_332._col0(Inner),Output:["_col1","_col2","_col3"]
                                         <-Map 17 [SIMPLE_EDGE] vectorized
                                           PARTITION_ONLY_SHUFFLE [RS_332]
@@ -313,9 +313,9 @@ Stage-0
                         <-Reducer 3 [SIMPLE_EDGE]
                           SHUFFLE [RS_33]
                             PartitionCols:_col0
-                            Group By Operator [GBY_32] (rows=1717 width=212)
+                            Group By Operator [GBY_32] (rows=343400 width=212)
                               Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                              Merge Join Operator [MERGEJOIN_300] (rows=1384530 width=100)
+                              Merge Join Operator [MERGEJOIN_300] (rows=252818424 width=201)
                                 Conds:RS_28._col0=RS_29._col2(Inner),Output:["_col1","_col7"]
                               <-Reducer 2 [SIMPLE_EDGE]
                                 SHUFFLE [RS_28]
@@ -324,9 +324,9 @@ Stage-0
                               <-Reducer 16 [SIMPLE_EDGE]
                                 SHUFFLE [RS_29]
                                   PartitionCols:_col2
-                                  Select Operator [SEL_24] (rows=2876890 width=4)
+                                  Select Operator [SEL_24] (rows=525327191 width=110)
                                     Output:["_col2","_col4"]
-                                    Merge Join Operator [MERGEJOIN_293] (rows=2876890 width=4)
+                                    Merge Join Operator [MERGEJOIN_293] (rows=525327191 width=110)
                                       Conds:RS_21._col2=RS_347._col0(Inner),Output:["_col1","_col3"]
                                     <-Map 25 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_347]
@@ -335,7 +335,7 @@ Stage-0
                                     <-Reducer 15 [SIMPLE_EDGE]
                                       SHUFFLE [RS_21]
                                         PartitionCols:_col2
-                                        Merge Join Operator [MERGEJOIN_292] (rows=14384447 width=4)
+                                        Merge Join Operator [MERGEJOIN_292] (rows=525327191 width=114)
                                           Conds:RS_344._col0=RS_328._col0(Inner),Output:["_col1","_col2","_col3"]
                                         <-Map 17 [SIMPLE_EDGE] vectorized
                                           PARTITION_ONLY_SHUFFLE [RS_328]
@@ -373,9 +373,9 @@ Stage-0
                         <-Reducer 8 [SIMPLE_EDGE]
                           SHUFFLE [RS_69]
                             PartitionCols:_col0
-                            Group By Operator [GBY_68] (rows=1717 width=212)
+                            Group By Operator [GBY_68] (rows=195738 width=212)
                               Output:["_col0","_col1"],aggregations:["sum(_col7)"],keys:_col1
-                              Merge Join Operator [MERGEJOIN_301] (rows=746132 width=100)
+                              Merge Join Operator [MERGEJOIN_301] (rows=137215467 width=210)
                                 Conds:RS_64._col0=RS_65._col3(Inner),Output:["_col1","_col7"]
                               <-Reducer 2 [SIMPLE_EDGE]
                                 SHUFFLE [RS_64]
@@ -384,9 +384,9 @@ Stage-0
                               <-Reducer 20 [SIMPLE_EDGE]
                                 SHUFFLE [RS_65]
                                   PartitionCols:_col3
-                                  Select Operator [SEL_60] (rows=1550375 width=13)
+                                  Select Operator [SEL_60] (rows=285117733 width=115)
                                     Output:["_col3","_col4"]
-                                    Merge Join Operator [MERGEJOIN_296] (rows=1550375 width=13)
+                                    Merge Join Operator [MERGEJOIN_296] (rows=285117733 width=115)
                                       Conds:RS_57._col1=RS_348._col0(Inner),Output:["_col2","_col3"]
                                     <-Map 25 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_348]
@@ -395,7 +395,7 @@ Stage-0
                                     <-Reducer 19 [SIMPLE_EDGE]
                                       SHUFFLE [RS_57]
                                         PartitionCols:_col1
-                                        Merge Join Operator [MERGEJOIN_295] (rows=7751872 width=98)
+                                        Merge Join Operator [MERGEJOIN_295] (rows=285117733 width=119)
                                           Conds:RS_363._col0=RS_330._col0(Inner),Output:["_col1","_col2","_col3"]
                                         <-Map 17 [SIMPLE_EDGE] vectorized
                                           PARTITION_ONLY_SHUFFLE [RS_330]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query61.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query61.q.out
index b7dbd43..6da93b1 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query61.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query61.q.out
@@ -140,7 +140,7 @@ Stage-0
                 PARTITION_ONLY_SHUFFLE [RS_42]
                   Group By Operator [GBY_41] (rows=1 width=112)
                     Output:["_col0"],aggregations:["sum(_col8)"]
-                    Merge Join Operator [MERGEJOIN_261] (rows=505397 width=0)
+                    Merge Join Operator [MERGEJOIN_261] (rows=461162530 width=106)
                       Conds:RS_37._col0=RS_38._col2(Inner),Output:["_col8"]
                     <-Reducer 2 [SIMPLE_EDGE]
                       SHUFFLE [RS_37]
@@ -168,7 +168,7 @@ Stage-0
                     <-Reducer 13 [SIMPLE_EDGE]
                       SHUFFLE [RS_38]
                         PartitionCols:_col2
-                        Merge Join Operator [MERGEJOIN_256] (rows=2526982 width=0)
+                        Merge Join Operator [MERGEJOIN_256] (rows=461162530 width=110)
                           Conds:RS_30._col4=RS_295._col0(Inner),Output:["_col2","_col5"]
                         <-Map 22 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_295]
@@ -182,7 +182,7 @@ Stage-0
                         <-Reducer 12 [SIMPLE_EDGE]
                           SHUFFLE [RS_30]
                             PartitionCols:_col4
-                            Merge Join Operator [MERGEJOIN_255] (rows=2526982 width=0)
+                            Merge Join Operator [MERGEJOIN_255] (rows=461162530 width=114)
                               Conds:RS_27._col3=RS_291._col0(Inner),Output:["_col2","_col4","_col5"]
                             <-Map 21 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_291]
@@ -196,7 +196,7 @@ Stage-0
                             <-Reducer 11 [SIMPLE_EDGE]
                               SHUFFLE [RS_27]
                                 PartitionCols:_col3
-                                Merge Join Operator [MERGEJOIN_254] (rows=12627499 width=0)
+                                Merge Join Operator [MERGEJOIN_254] (rows=461162530 width=118)
                                   Conds:RS_24._col1=RS_287._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
                                 <-Map 20 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_287]
@@ -210,7 +210,7 @@ Stage-0
                                 <-Reducer 10 [SIMPLE_EDGE]
                                   SHUFFLE [RS_24]
                                     PartitionCols:_col1
-                                    Merge Join Operator [MERGEJOIN_253] (rows=13119234 width=4)
+                                    Merge Join Operator [MERGEJOIN_253] (rows=479120969 width=122)
                                       Conds:RS_284._col0=RS_272._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
                                     <-Map 14 [SIMPLE_EDGE] vectorized
                                       PARTITION_ONLY_SHUFFLE [RS_272]
@@ -249,7 +249,7 @@ Stage-0
                 PARTITION_ONLY_SHUFFLE [RS_81]
                   Group By Operator [GBY_80] (rows=1 width=112)
                     Output:["_col0"],aggregations:["sum(_col7)"]
-                    Merge Join Operator [MERGEJOIN_262] (rows=529208 width=0)
+                    Merge Join Operator [MERGEJOIN_262] (rows=482889610 width=106)
                       Conds:RS_76._col0=RS_77._col2(Inner),Output:["_col7"]
                     <-Reducer 2 [SIMPLE_EDGE]
                       SHUFFLE [RS_76]
@@ -258,7 +258,7 @@ Stage-0
                     <-Reducer 18 [SIMPLE_EDGE]
                       SHUFFLE [RS_77]
                         PartitionCols:_col2
-                        Merge Join Operator [MERGEJOIN_260] (rows=2646038 width=0)
+                        Merge Join Operator [MERGEJOIN_260] (rows=482889610 width=110)
                           Conds:RS_69._col3=RS_292._col0(Inner),Output:["_col2","_col4"]
                         <-Map 21 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_292]
@@ -267,7 +267,7 @@ Stage-0
                         <-Reducer 17 [SIMPLE_EDGE]
                           SHUFFLE [RS_69]
                             PartitionCols:_col3
-                            Merge Join Operator [MERGEJOIN_259] (rows=13222427 width=0)
+                            Merge Join Operator [MERGEJOIN_259] (rows=482889610 width=114)
                               Conds:RS_66._col1=RS_288._col0(Inner),Output:["_col2","_col3","_col4"]
                             <-Map 20 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_288]
@@ -276,7 +276,7 @@ Stage-0
                             <-Reducer 16 [SIMPLE_EDGE]
                               SHUFFLE [RS_66]
                                 PartitionCols:_col1
-                                Merge Join Operator [MERGEJOIN_258] (rows=13737330 width=4)
+                                Merge Join Operator [MERGEJOIN_258] (rows=501694138 width=118)
                                   Conds:RS_302._col0=RS_274._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
                                 <-Map 14 [SIMPLE_EDGE] vectorized
                                   PARTITION_ONLY_SHUFFLE [RS_274]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query63.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query63.q.out
index af85791..8d45d82 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query63.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query63.q.out
@@ -80,30 +80,30 @@ Stage-0
     Stage-1
       Reducer 5 vectorized
       File Output Operator [FS_81]
-        Limit [LIM_80] (rows=65 width=228)
+        Limit [LIM_80] (rows=100 width=228)
           Number of rows:100
-          Select Operator [SEL_79] (rows=65 width=228)
+          Select Operator [SEL_79] (rows=338 width=228)
             Output:["_col0","_col1","_col2"]
           <-Reducer 4 [SIMPLE_EDGE]
             SHUFFLE [RS_27]
-              Select Operator [SEL_24] (rows=65 width=228)
+              Select Operator [SEL_24] (rows=338 width=228)
                 Output:["_col0","_col1","_col2"]
-                Filter Operator [FIL_36] (rows=65 width=228)
+                Filter Operator [FIL_36] (rows=338 width=228)
                   predicate:CASE WHEN ((avg_window_0 > 0)) THEN (((abs((_col2 - avg_window_0)) / avg_window_0) > 0.1)) ELSE (false) END
-                  Select Operator [SEL_23] (rows=130 width=116)
+                  Select Operator [SEL_23] (rows=676 width=116)
                     Output:["avg_window_0","_col0","_col2"]
-                    PTF Operator [PTF_22] (rows=130 width=116)
+                    PTF Operator [PTF_22] (rows=676 width=116)
                       Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col0 ASC NULLS FIRST","partition by:":"_col0"}]
-                      Select Operator [SEL_19] (rows=130 width=116)
+                      Select Operator [SEL_19] (rows=676 width=116)
                         Output:["_col0","_col2"]
-                        Group By Operator [GBY_18] (rows=130 width=120)
+                        Group By Operator [GBY_18] (rows=676 width=120)
                           Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                         <-Reducer 3 [SIMPLE_EDGE]
                           SHUFFLE [RS_17]
                             PartitionCols:_col0
-                            Group By Operator [GBY_16] (rows=130 width=120)
+                            Group By Operator [GBY_16] (rows=676 width=120)
                               Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)"],keys:_col4, _col6
-                              Merge Join Operator [MERGEJOIN_62] (rows=98800 width=8)
+                              Merge Join Operator [MERGEJOIN_62] (rows=569118 width=8)
                                 Conds:RS_12._col0=RS_76._col0(Inner),Output:["_col2","_col4","_col6"]
                               <-Map 8 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_76]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query64.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query64.q.out
index e36adb5..77bd755 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query64.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query64.q.out
@@ -316,33 +316,33 @@ Stage-0
     Stage-1
       Reducer 18 vectorized
       File Output Operator [FS_1079]
-        Select Operator [SEL_1078] (rows=104628491644 width=1702)
+        Select Operator [SEL_1078] (rows=338108448450 width=1702)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20"]
         <-Reducer 17 [SIMPLE_EDGE]
           SHUFFLE [RS_201]
-            Select Operator [SEL_200] (rows=104628491644 width=1694)
+            Select Operator [SEL_200] (rows=338108448450 width=1694)
               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"]
-              Filter Operator [FIL_199] (rows=104628491644 width=1694)
+              Filter Operator [FIL_199] (rows=338108448450 width=1694)
                 predicate:(_col3 <= _col19)
-                Merge Join Operator [MERGEJOIN_977] (rows=313885474933 width=1694)
+                Merge Join Operator [MERGEJOIN_977] (rows=1014325345351 width=1694)
                   Conds:RS_1052._col1, _col0, _col2=RS_1077._col2, _col1, _col3(Inner),Output:["_col3","_col4","_col5","_col6","_col7","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22"]
                 <-Reducer 16 [SIMPLE_EDGE] vectorized
                   SHUFFLE [RS_1052]
                     PartitionCols:_col1, _col0, _col2
-                    Select Operator [SEL_1051] (rows=21304422 width=525)
+                    Select Operator [SEL_1051] (rows=68803026 width=525)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                      Filter Operator [FIL_1050] (rows=21304422 width=1255)
+                      Filter Operator [FIL_1050] (rows=68803026 width=1255)
                         predicate:_col13 is not null
-                        Select Operator [SEL_1049] (rows=21304422 width=1255)
+                        Select Operator [SEL_1049] (rows=68803026 width=1255)
                           Output:["_col0","_col1","_col2","_col13","_col14","_col15","_col16"]
-                          Group By Operator [GBY_1048] (rows=21304422 width=1255)
+                          Group By Operator [GBY_1048] (rows=68803026 width=1255)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, KEY._col12
                           <-Reducer 15 [SIMPLE_EDGE]
                             SHUFFLE [RS_93]
                               PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
-                              Group By Operator [GBY_92] (rows=21304422 width=1255)
+                              Group By Operator [GBY_92] (rows=68803026 width=1255)
                                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16"],aggregations:["count()","sum(_col8)","sum(_col9)","sum(_col10)"],keys:_col24, _col11, _col25, _col29, _col31, _col37, _col38, _col39, _col40, _col42, _col43, _col44, _col45
-                                Merge Join Operator [MERGEJOIN_961] (rows=21304422 width=1048)
+                                Merge Join Operator [MERGEJOIN_961] (rows=68803026 width=1155)
                                   Conds:RS_88._col17=RS_1045._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40","_col42","_col43","_col44","_col45"]
                                 <-Map 49 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_1045]
@@ -354,7 +354,7 @@ Stage-0
                                 <-Reducer 14 [SIMPLE_EDGE]
                                   SHUFFLE [RS_88]
                                     PartitionCols:_col17
-                                    Merge Join Operator [MERGEJOIN_960] (rows=21304422 width=691)
+                                    Merge Join Operator [MERGEJOIN_960] (rows=68803026 width=798)
                                       Conds:RS_85._col5=RS_1044._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col17","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40"]
                                     <-Map 49 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_1044]
@@ -363,9 +363,9 @@ Stage-0
                                     <-Reducer 13 [SIMPLE_EDGE]
                                       SHUFFLE [RS_85]
                                         PartitionCols:_col5
-                                        Filter Operator [FIL_84] (rows=21304422 width=502)
+                                        Filter Operator [FIL_84] (rows=68803026 width=610)
                                           predicate:(_col33 <> _col35)
-                                          Merge Join Operator [MERGEJOIN_959] (rows=21304422 width=502)
+                                          Merge Join Operator [MERGEJOIN_959] (rows=68803026 width=610)
                                             Conds:RS_81._col15=RS_1040._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col17","_col24","_col25","_col29","_col31","_col33","_col35"]
                                           <-Map 48 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_1040]
@@ -377,7 +377,7 @@ Stage-0
                                           <-Reducer 12 [SIMPLE_EDGE]
                                             SHUFFLE [RS_81]
                                               PartitionCols:_col15
-                                              Merge Join Operator [MERGEJOIN_958] (rows=21007353 width=418)
+                                              Merge Join Operator [MERGEJOIN_958] (rows=67843635 width=527)
                                                 Conds:RS_78._col3=RS_1039._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col15","_col17","_col24","_col25","_col29","_col31","_col33"]
                                               <-Map 48 [SIMPLE_EDGE] vectorized
                                                 SHUFFLE [RS_1039]
@@ -386,7 +386,7 @@ Stage-0
                                               <-Reducer 11 [SIMPLE_EDGE]
                                                 SHUFFLE [RS_78]
                                                   PartitionCols:_col3
-                                                  Merge Join Operator [MERGEJOIN_957] (rows=20714426 width=331)
+                                                  Merge Join Operator [MERGEJOIN_957] (rows=66897622 width=444)
                                                     Conds:RS_75._col18=RS_984._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col15","_col17","_col24","_col25","_col29","_col31"]
                                                   <-Map 44 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_984]
@@ -398,7 +398,7 @@ Stage-0
                                                   <-Reducer 10 [SIMPLE_EDGE]
                                                     SHUFFLE [RS_75]
                                                       PartitionCols:_col18
-                                                      Merge Join Operator [MERGEJOIN_956] (rows=20714426 width=331)
+                                                      Merge Join Operator [MERGEJOIN_956] (rows=66897622 width=444)
                                                         Conds:RS_72._col19=RS_986._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col15","_col17","_col18","_col24","_col25","_col29"]
                                                       <-Map 44 [SIMPLE_EDGE] vectorized
                                                         PARTITION_ONLY_SHUFFLE [RS_986]
@@ -409,7 +409,7 @@ Stage-0
                                                       <-Reducer 9 [SIMPLE_EDGE]
                                                         SHUFFLE [RS_72]
                                                           PartitionCols:_col19
-                                                          Merge Join Operator [MERGEJOIN_955] (rows=20714426 width=330)
+                                                          Merge Join Operator [MERGEJOIN_955] (rows=66897622 width=444)
                                                             Conds:RS_69._col16=RS_1035._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col15","_col17","_col18","_col19","_col24","_col25"]
                                                           <-Map 43 [SIMPLE_EDGE] vectorized
                                                             SHUFFLE [RS_1035]
@@ -423,7 +423,7 @@ Stage-0
                                                           <-Reducer 8 [SIMPLE_EDGE]
                                                             SHUFFLE [RS_69]
                                                               PartitionCols:_col16
-                                                              Merge Join Operator [MERGEJOIN_954] (rows=20714426 width=334)
+                                                              Merge Join Operator [MERGEJOIN_954] (rows=66897622 width=447)
                                                                 Conds:RS_66._col4=RS_1034._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
                                                               <-Map 43 [SIMPLE_EDGE] vectorized
                                                                 SHUFFLE [RS_1034]
@@ -432,7 +432,7 @@ Stage-0
                                                               <-Reducer 7 [SIMPLE_EDGE]
                                                                 SHUFFLE [RS_66]
                                                                   PartitionCols:_col4
-                                                                  Merge Join Operator [MERGEJOIN_953] (rows=20714426 width=336)
+                                                                  Merge Join Operator [MERGEJOIN_953] (rows=66897622 width=450)
                                                                     Conds:RS_63._col6=RS_1030._col0(Inner),Output:["_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
                                                                   <-Map 42 [SIMPLE_EDGE] vectorized
                                                                     SHUFFLE [RS_1030]
@@ -446,7 +446,7 @@ Stage-0
                                                                   <-Reducer 6 [SIMPLE_EDGE]
                                                                     SHUFFLE [RS_63]
                                                                       PartitionCols:_col6
-                                                                      Merge Join Operator [MERGEJOIN_952] (rows=20714426 width=160)
+                                                                      Merge Join Operator [MERGEJOIN_952] (rows=66897622 width=276)
                                                                         Conds:RS_60._col1, _col7=RS_1026._col0, _col1(Inner),Output:["_col3","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19"]
                                                                       <-Map 41 [SIMPLE_EDGE] vectorized
                                                                         SHUFFLE [RS_1026]
@@ -458,7 +458,7 @@ Stage-0
                                                                       <-Reducer 5 [SIMPLE_EDGE]
                                                                         SHUFFLE [RS_60]
                                                                           PartitionCols:_col1, _col7
-                                                                          Merge Join Operator [MERGEJOIN_951] (rows=12564038 width=28)
+                                                                          Merge Join Operator [MERGEJOIN_951] (rows=40575792 width=220)
                                                                             Conds:RS_57._col1=RS_1024._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19"]
                                                                           <-Reducer 37 [SIMPLE_EDGE] vectorized
                                                                             SHUFFLE [RS_1024]
@@ -511,7 +511,7 @@ Stage-0
                                                                           <-Reducer 4 [SIMPLE_EDGE]
                                                                             SHUFFLE [RS_57]
                                                                               PartitionCols:_col1
-                                                                              Merge Join Operator [MERGEJOIN_949] (rows=12564038 width=28)
+                                                                              Merge Join Operator [MERGEJOIN_949] (rows=40575792 width=220)
                                                                                 Conds:RS_54._col2=RS_1011._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19"]
                                                                               <-Map 21 [SIMPLE_EDGE] vectorized
                                                                                 SHUFFLE [RS_1011]
@@ -525,7 +525,7 @@ Stage-0
                                                                               <-Reducer 3 [SIMPLE_EDGE]
                                                                                 SHUFFLE [RS_54]
                                                                                   PartitionCols:_col2
-                                                                                  Merge Join Operator [MERGEJOIN_948] (rows=14487982 width=12)
+                                                                                  Merge Join Operator [MERGEJOIN_948] (rows=40575792 width=203)
                                                                                     Conds:RS_51._col0=RS_990._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                                                                   <-Map 44 [SIMPLE_EDGE] vectorized
                                                                                     PARTITION_ONLY_SHUFFLE [RS_990]
@@ -567,20 +567,20 @@ Stage-0
                 <-Reducer 34 [SIMPLE_EDGE] vectorized
                   SHUFFLE [RS_1077]
                     PartitionCols:_col2, _col1, _col3
-                    Select Operator [SEL_1076] (rows=21304422 width=1354)
+                    Select Operator [SEL_1076] (rows=68803026 width=1354)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15"]
-                      Filter Operator [FIL_1075] (rows=21304422 width=1362)
+                      Filter Operator [FIL_1075] (rows=68803026 width=1362)
                         predicate:_col14 is not null
-                        Select Operator [SEL_1074] (rows=21304422 width=1362)
+                        Select Operator [SEL_1074] (rows=68803026 width=1362)
                           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col14","_col15","_col16","_col17"]
-                          Group By Operator [GBY_1073] (rows=21304422 width=1362)
+                          Group By Operator [GBY_1073] (rows=68803026 width=1362)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, KEY._col12, KEY._col13
                           <-Reducer 33 [SIMPLE_EDGE]
                             SHUFFLE [RS_191]
                               PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-                              Group By Operator [GBY_190] (rows=21304422 width=1362)
+                              Group By Operator [GBY_190] (rows=68803026 width=1362)
                                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count()","sum(_col8)","sum(_col9)","sum(_col10)"],keys:_col24, _col11, _col25, _col12, _col29, _col31, _col37, _col38, _col39, _col40, _col42, _col43, _col44, _col45
-                                Merge Join Operator [MERGEJOIN_976] (rows=21304422 width=1155)
+                                Merge Join Operator [MERGEJOIN_976] (rows=68803026 width=1262)
                                   Conds:RS_186._col17=RS_1047._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col12","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40","_col42","_col43","_col44","_col45"]
                                 <-Map 49 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_1047]
@@ -589,7 +589,7 @@ Stage-0
                                 <-Reducer 32 [SIMPLE_EDGE]
                                   SHUFFLE [RS_186]
                                     PartitionCols:_col17
-                                    Merge Join Operator [MERGEJOIN_975] (rows=21304422 width=798)
+                                    Merge Join Operator [MERGEJOIN_975] (rows=68803026 width=905)
                                       Conds:RS_183._col5=RS_1046._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col12","_col17","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40"]
                                     <-Map 49 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_1046]
@@ -598,9 +598,9 @@ Stage-0
                                     <-Reducer 31 [SIMPLE_EDGE]
                                       SHUFFLE [RS_183]
                                         PartitionCols:_col5
-                                        Filter Operator [FIL_182] (rows=21304422 width=609)
+                                        Filter Operator [FIL_182] (rows=68803026 width=717)
                                           predicate:(_col33 <> _col35)
-                                          Merge Join Operator [MERGEJOIN_974] (rows=21304422 width=609)
+                                          Merge Join Operator [MERGEJOIN_974] (rows=68803026 width=717)
                                             Conds:RS_179._col15=RS_1042._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col12","_col17","_col24","_col25","_col29","_col31","_col33","_col35"]
                                           <-Map 48 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_1042]
@@ -609,7 +609,7 @@ Stage-0
                                           <-Reducer 30 [SIMPLE_EDGE]
                                             SHUFFLE [RS_179]
                                               PartitionCols:_col15
-                                              Merge Join Operator [MERGEJOIN_973] (rows=21007353 width=525)
+                                              Merge Join Operator [MERGEJOIN_973] (rows=67843635 width=634)
                                                 Conds:RS_176._col3=RS_1041._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col24","_col25","_col29","_col31","_col33"]
                                               <-Map 48 [SIMPLE_EDGE] vectorized
                                                 SHUFFLE [RS_1041]
@@ -618,7 +618,7 @@ Stage-0
                                               <-Reducer 29 [SIMPLE_EDGE]
                                                 SHUFFLE [RS_176]
                                                   PartitionCols:_col3
-                                                  Merge Join Operator [MERGEJOIN_972] (rows=20714426 width=438)
+                                                  Merge Join Operator [MERGEJOIN_972] (rows=66897622 width=551)
                                                     Conds:RS_173._col18=RS_988._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col24","_col25","_col29","_col31"]
                                                   <-Map 44 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_988]
@@ -629,7 +629,7 @@ Stage-0
                                                   <-Reducer 28 [SIMPLE_EDGE]
                                                     SHUFFLE [RS_173]
                                                       PartitionCols:_col18
-                                                      Merge Join Operator [MERGEJOIN_971] (rows=20714426 width=438)
+                                                      Merge Join Operator [MERGEJOIN_971] (rows=66897622 width=551)
                                                         Conds:RS_170._col19=RS_987._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col18","_col24","_col25","_col29"]
                                                       <-Map 44 [SIMPLE_EDGE] vectorized
                                                         PARTITION_ONLY_SHUFFLE [RS_987]
@@ -640,7 +640,7 @@ Stage-0
                                                       <-Reducer 27 [SIMPLE_EDGE]
                                                         SHUFFLE [RS_170]
                                                           PartitionCols:_col19
-                                                          Merge Join Operator [MERGEJOIN_970] (rows=20714426 width=437)
+                                                          Merge Join Operator [MERGEJOIN_970] (rows=66897622 width=551)
                                                             Conds:RS_167._col16=RS_1037._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col18","_col19","_col24","_col25"]
                                                           <-Map 43 [SIMPLE_EDGE] vectorized
                                                             SHUFFLE [RS_1037]
@@ -649,7 +649,7 @@ Stage-0
                                                           <-Reducer 26 [SIMPLE_EDGE]
                                                             SHUFFLE [RS_167]
                                                               PartitionCols:_col16
-                                                              Merge Join Operator [MERGEJOIN_969] (rows=20714426 width=441)
+                                                              Merge Join Operator [MERGEJOIN_969] (rows=66897622 width=554)
                                                                 Conds:RS_164._col4=RS_1036._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
                                                               <-Map 43 [SIMPLE_EDGE] vectorized
                                                                 SHUFFLE [RS_1036]
@@ -658,7 +658,7 @@ Stage-0
                                                               <-Reducer 25 [SIMPLE_EDGE]
                                                                 SHUFFLE [RS_164]
                                                                   PartitionCols:_col4
-                                                                  Merge Join Operator [MERGEJOIN_968] (rows=20714426 width=443)
+                                                                  Merge Join Operator [MERGEJOIN_968] (rows=66897622 width=557)
                                                                     Conds:RS_161._col6=RS_1031._col0(Inner),Output:["_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
                                                                   <-Map 42 [SIMPLE_EDGE] vectorized
                                                                     SHUFFLE [RS_1031]
@@ -667,7 +667,7 @@ Stage-0
                                                                   <-Reducer 24 [SIMPLE_EDGE]
                                                                     SHUFFLE [RS_161]
                                                                       PartitionCols:_col6
-                                                                      Merge Join Operator [MERGEJOIN_967] (rows=20714426 width=267)
+                                                                      Merge Join Operator [MERGEJOIN_967] (rows=66897622 width=383)
                                                                         Conds:RS_158._col1, _col7=RS_1027._col0, _col1(Inner),Output:["_col3","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
                                                                       <-Map 41 [SIMPLE_EDGE] vectorized
                                                                         SHUFFLE [RS_1027]
@@ -676,12 +676,12 @@ Stage-0
                                                                       <-Reducer 23 [SIMPLE_EDGE]
                                                                         SHUFFLE [RS_158]
                                                                           PartitionCols:_col1, _col7
-                                                                          Merge Join Operator [MERGEJOIN_966] (rows=12564038 width=135)
+                                                                          Merge Join Operator [MERGEJOIN_966] (rows=40575792 width=327)
                                                                             Conds:RS_155._col1=RS_1072._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
                                                                           <-Reducer 22 [SIMPLE_EDGE]
                                                                             SHUFFLE [RS_155]
                                                                               PartitionCols:_col1
-                                                                              Merge Join Operator [MERGEJOIN_964] (rows=12564038 width=135)
+                                                                              Merge Join Operator [MERGEJOIN_964] (rows=40575792 width=327)
                                                                                 Conds:RS_152._col2=RS_1012._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
                                                                               <-Map 21 [SIMPLE_EDGE] vectorized
                                                                                 SHUFFLE [RS_1012]
@@ -690,7 +690,7 @@ Stage-0
                                                                               <-Reducer 46 [SIMPLE_EDGE]
                                                                                 SHUFFLE [RS_152]
                                                                                   PartitionCols:_col2
-                                                                                  Merge Join Operator [MERGEJOIN_963] (rows=14487982 width=119)
+                                                                                  Merge Join Operator [MERGEJOIN_963] (rows=40575792 width=310)
                                                                                     Conds:RS_149._col0=RS_992._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"]
                                                                                   <-Map 44 [SIMPLE_EDGE] vectorized
                                                                                     PARTITION_ONLY_SHUFFLE [RS_992]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query65.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query65.q.out
index b200fe4..8b8a414 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query65.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query65.q.out
@@ -83,15 +83,15 @@ Stage-0
     Stage-1
       Reducer 7 vectorized
       File Output Operator [FS_164]
-        Limit [LIM_163] (rows=100 width=705)
+        Limit [LIM_163] (rows=100 width=708)
           Number of rows:100
-          Select Operator [SEL_162] (rows=61787 width=703)
+          Select Operator [SEL_162] (rows=1772841 width=707)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           <-Reducer 6 [SIMPLE_EDGE]
             SHUFFLE [RS_51]
-              Select Operator [SEL_50] (rows=61787 width=703)
+              Select Operator [SEL_50] (rows=1772841 width=707)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                Merge Join Operator [MERGEJOIN_137] (rows=61787 width=703)
+                Merge Join Operator [MERGEJOIN_137] (rows=1772841 width=707)
                   Conds:RS_47._col1=RS_161._col0(Inner),Output:["_col2","_col6","_col8","_col9","_col10","_col11"]
                 <-Map 12 [SIMPLE_EDGE] vectorized
                   SHUFFLE [RS_161]
@@ -103,7 +103,7 @@ Stage-0
                 <-Reducer 5 [SIMPLE_EDGE]
                   SHUFFLE [RS_47]
                     PartitionCols:_col1
-                    Merge Join Operator [MERGEJOIN_136] (rows=61787 width=204)
+                    Merge Join Operator [MERGEJOIN_136] (rows=1772841 width=204)
                       Conds:RS_44._col0=RS_159._col0(Inner),Output:["_col1","_col2","_col6"]
                     <-Map 11 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_159]
@@ -115,23 +115,23 @@ Stage-0
                     <-Reducer 4 [SIMPLE_EDGE]
                       SHUFFLE [RS_44]
                         PartitionCols:_col0
-                        Filter Operator [FIL_43] (rows=61787 width=231)
+                        Filter Operator [FIL_43] (rows=1772841 width=231)
                           predicate:(_col2 <= _col4)
-                          Merge Join Operator [MERGEJOIN_135] (rows=185361 width=231)
+                          Merge Join Operator [MERGEJOIN_135] (rows=5318523 width=231)
                             Conds:RS_151._col0=RS_157._col0(Inner),Output:["_col0","_col1","_col2","_col4"]
                           <-Reducer 3 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_151]
                               PartitionCols:_col0
-                              Filter Operator [FIL_150] (rows=173776 width=118)
+                              Filter Operator [FIL_150] (rows=5255208 width=119)
                                 predicate:_col2 is not null
-                                Group By Operator [GBY_149] (rows=173776 width=118)
+                                Group By Operator [GBY_149] (rows=5255208 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=5734608 width=118)
+                                    Group By Operator [GBY_10] (rows=525329897 width=119)
                                       Output:["_col0","_col1","_col2"],aggregations:["sum(_col3)"],keys:_col2, _col1
-                                      Merge Join Operator [MERGEJOIN_133] (rows=91197860 width=89)
+                                      Merge Join Operator [MERGEJOIN_133] (rows=525329897 width=114)
                                         Conds:RS_148._col0=RS_140._col0(Inner),Output:["_col1","_col2","_col3"]
                                       <-Map 9 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_140]
@@ -165,15 +165,15 @@ Stage-0
                           <-Reducer 8 [SIMPLE_EDGE] vectorized
                             SHUFFLE [RS_157]
                               PartitionCols:_col0
-                              Select Operator [SEL_156] (rows=16 width=115)
+                              Select Operator [SEL_156] (rows=84 width=115)
                                 Output:["_col0","_col1"]
-                                Filter Operator [FIL_155] (rows=16 width=123)
+                                Filter Operator [FIL_155] (rows=84 width=123)
                                   predicate:(_col1 is not null and _col2 is not null)
-                                  Group By Operator [GBY_154] (rows=16 width=123)
+                                  Group By Operator [GBY_154] (rows=84 width=123)
                                     Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
-                                    Select Operator [SEL_153] (rows=173776 width=118)
+                                    Select Operator [SEL_153] (rows=5255208 width=119)
                                       Output:["_col1","_col2"]
-                                      Group By Operator [GBY_152] (rows=173776 width=118)
+                                      Group By Operator [GBY_152] (rows=5255208 width=119)
                                         Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                                       <-Reducer 2 [SIMPLE_EDGE]
                                         SHUFFLE [RS_27]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query66.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query66.q.out
index 3f84383..26814b5 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query66.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query66.q.out
@@ -505,11 +505,11 @@ Stage-0
                             <-Reducer 14 [SIMPLE_EDGE]
                               SHUFFLE [RS_61]
                                 PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
-                                Group By Operator [GBY_60] (rows=12905590 width=3166)
+                                Group By Operator [GBY_60] (rows=15681803 width=3166)
                                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18 [...]
-                                  Select Operator [SEL_58] (rows=12905590 width=750)
+                                  Select Operator [SEL_58] (rows=15681803 width=750)
                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"]
-                                    Merge Join Operator [MERGEJOIN_202] (rows=12905590 width=750)
+                                    Merge Join Operator [MERGEJOIN_202] (rows=15681803 width=750)
                                       Conds:RS_55._col3=RS_240._col0(Inner),Output:["_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col22","_col23","_col24","_col25","_col26","_col27"]
                                     <-Map 20 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_240]
@@ -521,7 +521,7 @@ Stage-0
                                     <-Reducer 13 [SIMPLE_EDGE]
                                       SHUFFLE [RS_55]
                                         PartitionCols:_col3
-                                        Merge Join Operator [MERGEJOIN_201] (rows=12905590 width=275)
+                                        Merge Join Operator [MERGEJOIN_201] (rows=15681803 width=275)
                                           Conds:RS_52._col2=RS_219._col0(Inner),Output:["_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19"]
                                         <-Map 17 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_219]
@@ -535,7 +535,7 @@ Stage-0
                                         <-Reducer 12 [SIMPLE_EDGE]
                                           SHUFFLE [RS_52]
                                             PartitionCols:_col2
-                                            Merge Join Operator [MERGEJOIN_200] (rows=38716771 width=279)
+                                            Merge Join Operator [MERGEJOIN_200] (rows=282272460 width=279)
                                               Conds:RS_49._col0=RS_237._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19"]
                                             <-Map 16 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_237]
@@ -549,7 +549,7 @@ Stage-0
                                             <-Reducer 11 [SIMPLE_EDGE]
                                               SHUFFLE [RS_49]
                                                 PartitionCols:_col0
-                                                Merge Join Operator [MERGEJOIN_199] (rows=109204159 width=235)
+                                                Merge Join Operator [MERGEJOIN_199] (rows=282272460 width=235)
                                                   Conds:RS_256._col1=RS_233._col0(Inner),Output:["_col0","_col2","_col3","_col4","_col5"]
                                                 <-Map 10 [SIMPLE_EDGE] vectorized
                                                   SHUFFLE [RS_233]
@@ -596,9 +596,9 @@ Stage-0
                                 PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5
                                 Group By Operator [GBY_28] (rows=27 width=3166)
                                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18 [...]
-                                  Select Operator [SEL_26] (rows=6624114 width=750)
+                                  Select Operator [SEL_26] (rows=7992175 width=750)
                                     Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"]
-                                    Merge Join Operator [MERGEJOIN_198] (rows=6624114 width=750)
+                                    Merge Join Operator [MERGEJOIN_198] (rows=7992175 width=750)
                                       Conds:RS_23._col3=RS_239._col0(Inner),Output:["_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col22","_col23","_col24","_col25","_col26","_col27"]
                                     <-Map 20 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_239]
@@ -607,7 +607,7 @@ Stage-0
                                     <-Reducer 4 [SIMPLE_EDGE]
                                       SHUFFLE [RS_23]
                                         PartitionCols:_col3
-                                        Merge Join Operator [MERGEJOIN_197] (rows=6624114 width=275)
+                                        Merge Join Operator [MERGEJOIN_197] (rows=7992175 width=275)
                                           Conds:RS_20._col2=RS_217._col0(Inner),Output:["_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19"]
                                         <-Map 17 [SIMPLE_EDGE] vectorized
                                           SHUFFLE [RS_217]
@@ -616,7 +616,7 @@ Stage-0
                                         <-Reducer 3 [SIMPLE_EDGE]
                                           SHUFFLE [RS_20]
                                             PartitionCols:_col2
-                                            Merge Join Operator [MERGEJOIN_196] (rows=19872342 width=279)
+                                            Merge Join Operator [MERGEJOIN_196] (rows=143859154 width=279)
                                               Conds:RS_17._col0=RS_236._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19"]
                                             <-Map 16 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_236]
@@ -625,7 +625,7 @@ Stage-0
                                             <-Reducer 2 [SIMPLE_EDGE]
                                               SHUFFLE [RS_17]
                                                 PartitionCols:_col0
-                                                Merge Join Operator [MERGEJOIN_195] (rows=55655511 width=235)
+                                                Merge Join Operator [MERGEJOIN_195] (rows=143859154 width=235)
                                                   Conds:RS_229._col1=RS_232._col0(Inner),Output:["_col0","_col2","_col3","_col4","_col5"]
                                                 <-Map 10 [SIMPLE_EDGE] vectorized
                                                   SHUFFLE [RS_232]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query67.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query67.q.out
index 98de6d3..ee1bf12 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query67.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query67.q.out
@@ -114,31 +114,31 @@ Stage-0
       File Output Operator [FS_107]
         Limit [LIM_106] (rows=100 width=617)
           Number of rows:100
-          Select Operator [SEL_105] (rows=273593580 width=617)
+          Select Operator [SEL_105] (rows=1575989691 width=617)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
           <-Reducer 6 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_104]
-              Select Operator [SEL_103] (rows=273593580 width=617)
+              Select Operator [SEL_103] (rows=1575989691 width=617)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
-                Filter Operator [FIL_102] (rows=273593580 width=613)
+                Filter Operator [FIL_102] (rows=1575989691 width=613)
                   predicate:(rank_window_0 <= 100)
-                  PTF Operator [PTF_101] (rows=820780740 width=613)
+                  PTF Operator [PTF_101] (rows=4727969073 width=613)
                     Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col8 DESC NULLS LAST","partition by:":"_col6"}]
-                    Select Operator [SEL_100] (rows=820780740 width=613)
+                    Select Operator [SEL_100] (rows=4727969073 width=613)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
                     <-Reducer 5 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_99]
                         PartitionCols:_col6
-                        Select Operator [SEL_98] (rows=820780740 width=613)
+                        Select Operator [SEL_98] (rows=4727969073 width=613)
                           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
-                          Group By Operator [GBY_97] (rows=820780740 width=621)
+                          Group By Operator [GBY_97] (rows=4727969073 width=621)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col9"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8
                           <-Reducer 4 [SIMPLE_EDGE]
                             SHUFFLE [RS_21]
                               PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                              Group By Operator [GBY_20] (rows=820780740 width=621)
+                              Group By Operator [GBY_20] (rows=4727969073 width=621)
                                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["sum(_col3)"],keys:_col5, _col6, _col7, _col9, _col11, _col12, _col13, _col14, 0L
-                                Merge Join Operator [MERGEJOIN_81] (rows=91197860 width=613)
+                                Merge Join Operator [MERGEJOIN_81] (rows=525329897 width=613)
                                   Conds:RS_16._col1=RS_96._col0(Inner),Output:["_col3","_col5","_col6","_col7","_col9","_col11","_col12","_col13","_col14"]
                                 <-Map 11 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_96]
@@ -150,7 +150,7 @@ Stage-0
                                 <-Reducer 3 [SIMPLE_EDGE]
                                   SHUFFLE [RS_16]
                                     PartitionCols:_col1
-                                    Merge Join Operator [MERGEJOIN_80] (rows=91197860 width=228)
+                                    Merge Join Operator [MERGEJOIN_80] (rows=525329897 width=228)
                                       Conds:RS_13._col2=RS_94._col0(Inner),Output:["_col1","_col3","_col5","_col6","_col7","_col9"]
                                     <-Map 10 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_94]
@@ -162,7 +162,7 @@ Stage-0
                                     <-Reducer 2 [SIMPLE_EDGE]
                                       SHUFFLE [RS_13]
                                         PartitionCols:_col2
-                                        Merge Join Operator [MERGEJOIN_79] (rows=91197860 width=130)
+                                        Merge Join Operator [MERGEJOIN_79] (rows=525329897 width=131)
                                           Conds:RS_92._col0=RS_84._col0(Inner),Output:["_col1","_col2","_col3","_col5","_col6","_col7"]
                                         <-Map 8 [SIMPLE_EDGE] vectorized
                                           PARTITION_ONLY_SHUFFLE [RS_84]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query68.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query68.q.out
index 6c6f86a..97b97ce 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query68.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query68.q.out
@@ -116,15 +116,15 @@ Stage-0
       File Output Operator [FS_172]
         Limit [LIM_171] (rows=100 width=706)
           Number of rows:100
-          Select Operator [SEL_170] (rows=727776 width=706)
+          Select Operator [SEL_170] (rows=457565061 width=706)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
           <-Reducer 3 [SIMPLE_EDGE]
             SHUFFLE [RS_44]
-              Select Operator [SEL_43] (rows=727776 width=706)
+              Select Operator [SEL_43] (rows=457565061 width=706)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
-                Filter Operator [FIL_42] (rows=727776 width=706)
+                Filter Operator [FIL_42] (rows=457565061 width=706)
                   predicate:(_col5 <> _col8)
-                  Merge Join Operator [MERGEJOIN_143] (rows=727776 width=706)
+                  Merge Join Operator [MERGEJOIN_143] (rows=457565061 width=706)
                     Conds:RS_39._col0=RS_169._col1(Inner),Output:["_col2","_col3","_col5","_col6","_col8","_col9","_col10","_col11"]
                   <-Reducer 2 [SIMPLE_EDGE]
                     SHUFFLE [RS_39]
@@ -150,16 +150,16 @@ Stage-0
                   <-Reducer 7 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_169]
                       PartitionCols:_col1
-                      Select Operator [SEL_168] (rows=727776 width=433)
+                      Select Operator [SEL_168] (rows=457565061 width=436)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                        Group By Operator [GBY_167] (rows=727776 width=433)
+                        Group By Operator [GBY_167] (rows=457565061 width=440)
                           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
                         <-Reducer 6 [SIMPLE_EDGE]
                           SHUFFLE [RS_33]
                             PartitionCols:_col0, _col1, _col2, _col3
-                            Group By Operator [GBY_32] (rows=727776 width=433)
+                            Group By Operator [GBY_32] (rows=457565061 width=440)
                               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)"],keys:_col1, _col13, _col3, _col5
-                              Merge Join Operator [MERGEJOIN_142] (rows=727776 width=97)
+                              Merge Join Operator [MERGEJOIN_142] (rows=457565061 width=425)
                                 Conds:RS_28._col3=RS_149._col0(Inner),Output:["_col1","_col3","_col5","_col6","_col7","_col8","_col13"]
                               <-Map 5 [SIMPLE_EDGE] vectorized
                                 SHUFFLE [RS_149]
@@ -168,7 +168,7 @@ Stage-0
                               <-Reducer 11 [SIMPLE_EDGE]
                                 SHUFFLE [RS_28]
                                   PartitionCols:_col3
-                                  Merge Join Operator [MERGEJOIN_141] (rows=727776 width=4)
+                                  Merge Join Operator [MERGEJOIN_141] (rows=457565061 width=332)
                                     Conds:RS_25._col2=RS_166._col0(Inner),Output:["_col1","_col3","_col5","_col6","_col7","_col8"]
                                   <-Map 15 [SIMPLE_EDGE] vectorized
                                     SHUFFLE [RS_166]
@@ -182,7 +182,7 @@ Stage-0
                                   <-Reducer 10 [SIMPLE_EDGE]
                                     SHUFFLE [RS_25]
                                       PartitionCols:_col2
-                                      Merge Join Operator [MERGEJOIN_140] (rows=2824787 width=4)
+                                      Merge Join Operator [MERGEJOIN_140] (rows=457565061 width=336)
                                         Conds:RS_22._col4=RS_163._col0(Inner),Output:["_col1","_col2","_col3","_col5","_col6","_col7","_col8"]
                                       <-Map 14 [SIMPLE_EDGE] vectorized
                                         SHUFFLE [RS_163]
@@ -196,7 +196,7 @@ Stage-0
                                       <-Reducer 9 [SIMPLE_EDGE]
                                         SHUFFLE [RS_22]
                                           PartitionCols:_col4
-                                          Merge Join Operator [MERGEJOIN_139] (rows=42598570 width=185)
+                                          Merge Join Operator [MERGEJOIN_139] (rows=457565061 width=340)
                                             Conds:RS_160._col0=RS_152._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
                                           <-Map 12 [SIMPLE_EDGE] vectorized
                                             SHUFFLE [RS_152]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query69.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query69.q.out
index 56af2a6..ecbf687 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query69.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query69.q.out
@@ -109,20 +109,20 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 13 <- Reducer 16 (BROADCAST_EDGE)
-Map 21 <- Reducer 10 (BROADCAST_EDGE)
+Map 12 <- Reducer 15 (BROADCAST_EDGE)
+Map 21 <- Reducer 18 (BROADCAST_EDGE)
 Map 22 <- Reducer 9 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 14 <- Map 13 (SIMPLE_EDGE), Map 15 (SIMPLE_EDGE)
-Reducer 16 <- Map 15 (CUSTOM_SIMPLE_EDGE)
-Reducer 17 <- Map 15 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
-Reducer 18 <- Reducer 17 (SIMPLE_EDGE)
-Reducer 19 <- Map 15 (SIMPLE_EDGE), Map 22 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
+Reducer 13 <- Map 12 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE)
+Reducer 15 <- Map 14 (CUSTOM_SIMPLE_EDGE)
+Reducer 16 <- Map 14 (SIMPLE_EDGE), Map 21 (SIMPLE_EDGE)
+Reducer 17 <- Reducer 16 (SIMPLE_EDGE)
+Reducer 18 <- Map 14 (CUSTOM_SIMPLE_EDGE)
+Reducer 19 <- Map 14 (SIMPLE_EDGE), Map 22 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
 Reducer 20 <- Reducer 19 (SIMPLE_EDGE)
-Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Reducer 14 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 18 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 17 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
 Reducer 6 <- Reducer 20 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
@@ -133,16 +133,16 @@ Stage-0
     limit:100
     Stage-1
       Reducer 8 vectorized
-      File Output Operator [FS_226]
-        Limit [LIM_225] (rows=1 width=383)
+      File Output Operator [FS_229]
+        Limit [LIM_228] (rows=1 width=383)
           Number of rows:100
-          Select Operator [SEL_224] (rows=1 width=383)
+          Select Operator [SEL_227] (rows=1 width=383)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
           <-Reducer 7 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_223]
-              Select Operator [SEL_222] (rows=1 width=383)
+            SHUFFLE [RS_226]
+              Select Operator [SEL_225] (rows=1 width=383)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col6"]
-                Group By Operator [GBY_221] (rows=1 width=367)
+                Group By Operator [GBY_224] (rows=1 width=367)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4
                 <-Reducer 6 [SIMPLE_EDGE]
                   SHUFFLE [RS_68]
@@ -155,8 +155,8 @@ Stage-0
                           Output:["_col6","_col7","_col8","_col9","_col10"]
                           Filter Operator [FIL_65] (rows=1 width=363)
                             predicate:_col13 is null
-                            Merge Join Operator [MERGEJOIN_183] (rows=1 width=363)
-                              Conds:RS_62._col0=RS_220._col1(Left Outer),Output:["_col6","_col7","_col8","_col9","_col10","_col13"]
+                            Merge Join Operator [MERGEJOIN_183] (rows=1401496 width=363)
+                              Conds:RS_62._col0=RS_223._col1(Left Outer),Output:["_col6","_col7","_col8","_col9","_col10","_col13"]
                             <-Reducer 5 [SIMPLE_EDGE]
                               PARTITION_ONLY_SHUFFLE [RS_62]
                                 PartitionCols:_col0
@@ -164,47 +164,85 @@ Stage-0
                                   Output:["_col0","_col6","_col7","_col8","_col9","_col10"]
                                   Filter Operator [FIL_46] (rows=1 width=367)
                                     predicate:_col11 is null
-                                    Merge Join Operator [MERGEJOIN_182] (rows=30 width=367)
-                                      Conds:RS_43._col0=RS_212._col1(Left Outer),Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11"]
+                                    Merge Join Operator [MERGEJOIN_182] (rows=1414922 width=367)
+                                      Conds:RS_43._col0=RS_215._col1(Left Outer),Output:["_col0","_col6","_col7","_col8","_col9","_col10","_col11"]
+                                    <-Reducer 17 [SIMPLE_EDGE] vectorized
+                                      SHUFFLE [RS_215]
+                                        PartitionCols:_col1
+                                        Select Operator [SEL_214] (rows=1414922 width=7)
+                                          Output:["_col0","_col1"]
+                                          Group By Operator [GBY_213] (rows=1414922 width=3)
+                                            Output:["_col0"],keys:KEY._col0
+                                          <-Reducer 16 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_29]
+                                              PartitionCols:_col0
+                                              Group By Operator [GBY_28] (rows=143930993 width=3)
+                                                Output:["_col0"],keys:_col1
+                                                Merge Join Operator [MERGEJOIN_179] (rows=143930993 width=3)
+                                                  Conds:RS_212._col0=RS_196._col0(Inner),Output:["_col1"]
+                                                <-Map 14 [SIMPLE_EDGE] vectorized
+                                                  SHUFFLE [RS_196]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_193] (rows=150 width=4)
+                                                      Output:["_col0"]
+                                                      Filter Operator [FIL_192] (rows=150 width=12)
+                                                        predicate:((d_year = 1999) and d_moy BETWEEN 1 AND 3)
+                                                        TableScan [TS_11] (rows=73049 width=12)
+                                                          default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
+                                                <-Map 21 [SIMPLE_EDGE] vectorized
+                                                  SHUFFLE [RS_212]
+                                                    PartitionCols:_col0
+                                                    Select Operator [SEL_211] (rows=143930993 width=7)
+                                                      Output:["_col0","_col1"]
+                                                      Filter Operator [FIL_210] (rows=143930993 width=7)
+                                                        predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_sold_date_sk BETWEEN DynamicValue(RS_25_date_dim_d_date_sk_min) AND DynamicValue(RS_25_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_25_date_dim_d_date_sk_bloom_filter)))
+                                                        TableScan [TS_18] (rows=144002668 width=7)
+                                                          default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_bill_customer_sk"]
+                                                        <-Reducer 18 [BROADCAST_EDGE] vectorized
+                                                          BROADCAST [RS_209]
+                                                            Group By Operator [GBY_208] (rows=1 width=12)
+                                                              Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                            <-Map 14 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                              SHUFFLE [RS_202]
+                                                                Group By Operator [GBY_200] (rows=1 width=12)
+                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                  Select Operator [SEL_197] (rows=150 width=4)
+                                                                    Output:["_col0"]
+                                                                     Please refer to the previous Select Operator [SEL_193]
                                     <-Reducer 4 [SIMPLE_EDGE]
                                       SHUFFLE [RS_43]
                                         PartitionCols:_col0
-                                        Merge Join Operator [MERGEJOIN_181] (rows=6239 width=363)
+                                        Merge Join Operator [MERGEJOIN_181] (rows=525327388 width=363)
                                           Conds:RS_40._col0=RS_41._col0(Left Semi),Output:["_col0","_col6","_col7","_col8","_col9","_col10"]
-                                        <-Reducer 14 [SIMPLE_EDGE]
+                                        <-Reducer 13 [SIMPLE_EDGE]
                                           SHUFFLE [RS_41]
                                             PartitionCols:_col0
-                                            Group By Operator [GBY_39] (rows=106060 width=1)
+                                            Group By Operator [GBY_39] (rows=525327388 width=3)
                                               Output:["_col0"],keys:_col0
-                                              Select Operator [SEL_17] (rows=43153353 width=1)
+                                              Select Operator [SEL_17] (rows=525327388 width=3)
                                                 Output:["_col0"]
-                                                Merge Join Operator [MERGEJOIN_178] (rows=43153353 width=1)
-                                                  Conds:RS_204._col0=RS_194._col0(Inner),Output:["_col1"]
-                                                <-Map 15 [SIMPLE_EDGE] vectorized
+                                                Merge Join Operator [MERGEJOIN_178] (rows=525327388 width=3)
+                                                  Conds:RS_207._col0=RS_194._col0(Inner),Output:["_col1"]
+                                                <-Map 14 [SIMPLE_EDGE] vectorized
                                                   SHUFFLE [RS_194]
                                                     PartitionCols:_col0
-                                                    Select Operator [SEL_193] (rows=150 width=4)
-                                                      Output:["_col0"]
-                                                      Filter Operator [FIL_192] (rows=150 width=12)
-                                                        predicate:((d_year = 1999) and d_moy BETWEEN 1 AND 3)
-                                                        TableScan [TS_11] (rows=73049 width=12)
-                                                          default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
-                                                <-Map 13 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_204]
+                                                     Please refer to the previous Select Operator [SEL_193]
+                                                <-Map 12 [SIMPLE_EDGE] vectorized
+                                                  SHUFFLE [RS_207]
                                                     PartitionCols:_col0
-                                                    Select Operator [SEL_203] (rows=525327388 width=7)
+                                                    Select Operator [SEL_206] (rows=525327388 width=7)
                                                       Output:["_col0","_col1"]
-                                                      Filter Operator [FIL_202] (rows=525327388 width=7)
+                                                      Filter Operator [FIL_205] (rows=525327388 width=7)
                                                         predicate:(ss_sold_date_sk is not null and ss_customer_sk is not null and ss_sold_date_sk BETWEEN DynamicValue(RS_15_date_dim_d_date_sk_min) AND DynamicValue(RS_15_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_15_date_dim_d_date_sk_bloom_filter)))
                                                         TableScan [TS_8] (rows=575995635 width=7)
                                                           default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_customer_sk"]
-                                                        <-Reducer 16 [BROADCAST_EDGE] vectorized
-                                                          BROADCAST [RS_201]
-                                                            Group By Operator [GBY_200] (rows=1 width=12)
+                                                        <-Reducer 15 [BROADCAST_EDGE] vectorized
+                                                          BROADCAST [RS_204]
+                                                            Group By Operator [GBY_203] (rows=1 width=12)
                                                               Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                            <-Map 15 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                              SHUFFLE [RS_199]
-                                                                Group By Operator [GBY_198] (rows=1 width=12)
+                                                            <-Map 14 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                              SHUFFLE [RS_201]
+                                                                Group By Operator [GBY_199] (rows=1 width=12)
                                                                   Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
                                                                   Select Operator [SEL_195] (rows=150 width=4)
                                                                     Output:["_col0"]
@@ -214,7 +252,7 @@ Stage-0
                                             PartitionCols:_col0
                                             Merge Join Operator [MERGEJOIN_177] (rows=4605476 width=363)
                                               Conds:RS_35._col1=RS_191._col0(Inner),Output:["_col0","_col6","_col7","_col8","_col9","_col10"]
-                                            <-Map 12 [SIMPLE_EDGE] vectorized
+                                            <-Map 11 [SIMPLE_EDGE] vectorized
                                               SHUFFLE [RS_191]
                                                 PartitionCols:_col0
                                                 Select Operator [SEL_190] (rows=1861800 width=363)
@@ -235,7 +273,7 @@ Stage-0
                                                         predicate:(c_current_cdemo_sk is not null and c_current_addr_sk is not null)
                                                         TableScan [TS_0] (rows=80000000 width=11)
                                                           default@customer,c,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_current_cdemo_sk","c_current_addr_sk"]
-                                                <-Map 11 [SIMPLE_EDGE] vectorized
+                                                <-Map 10 [SIMPLE_EDGE] vectorized
                                                   SHUFFLE [RS_189]
                                                     PartitionCols:_col0
                                                     Select Operator [SEL_188] (rows=2352941 width=90)
@@ -244,74 +282,36 @@ Stage-0
                                                         predicate:(ca_state) IN ('CO', 'IL', 'MN')
                                                         TableScan [TS_3] (rows=40000000 width=90)
                                                           default@customer_address,ca,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_state"]
-                                    <-Reducer 18 [SIMPLE_EDGE] vectorized
-                                      SHUFFLE [RS_212]
-                                        PartitionCols:_col1
-                                        Select Operator [SEL_211] (rows=116231 width=7)
-                                          Output:["_col0","_col1"]
-                                          Group By Operator [GBY_210] (rows=116231 width=3)
-                                            Output:["_col0"],keys:KEY._col0
-                                          <-Reducer 17 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_29]
-                                              PartitionCols:_col0
-                                              Group By Operator [GBY_28] (rows=116231 width=3)
-                                                Output:["_col0"],keys:_col1
-                                                Merge Join Operator [MERGEJOIN_179] (rows=11823304 width=3)
-                                                  Conds:RS_209._col0=RS_196._col0(Inner),Output:["_col1"]
-                                                <-Map 15 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_196]
-                                                    PartitionCols:_col0
-                                                     Please refer to the previous Select Operator [SEL_193]
-                                                <-Map 21 [SIMPLE_EDGE] vectorized
-                                                  SHUFFLE [RS_209]
-                                                    PartitionCols:_col0
-                                                    Select Operator [SEL_208] (rows=143930993 width=7)
-                                                      Output:["_col0","_col1"]
-                                                      Filter Operator [FIL_207] (rows=143930993 width=7)
-                                                        predicate:(ws_bill_customer_sk is not null and ws_sold_date_sk is not null and ws_bill_customer_sk BETWEEN DynamicValue(RS_43_c_c_customer_sk_min) AND DynamicValue(RS_43_c_c_customer_sk_max) and in_bloom_filter(ws_bill_customer_sk, DynamicValue(RS_43_c_c_customer_sk_bloom_filter)))
-                                                        TableScan [TS_18] (rows=144002668 width=7)
-                                                          default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_bill_customer_sk"]
-                                                        <-Reducer 10 [BROADCAST_EDGE] vectorized
-                                                          BROADCAST [RS_206]
-                                                            Group By Operator [GBY_205] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                            <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-                                                              SHUFFLE [RS_152]
-                                                                Group By Operator [GBY_151] (rows=1 width=12)
-                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                  Select Operator [SEL_150] (rows=6239 width=4)
-                                                                    Output:["_col0"]
-                                                                     Please refer to the previous Merge Join Operator [MERGEJOIN_181]
                             <-Reducer 20 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_220]
+                              SHUFFLE [RS_223]
                                 PartitionCols:_col1
-                                Select Operator [SEL_219] (rows=114314 width=7)
+                                Select Operator [SEL_222] (rows=1401496 width=7)
                                   Output:["_col0","_col1"]
-                                  Group By Operator [GBY_218] (rows=114314 width=3)
+                                  Group By Operator [GBY_221] (rows=1401496 width=3)
                                     Output:["_col0"],keys:KEY._col0
                                   <-Reducer 19 [SIMPLE_EDGE]
                                     SHUFFLE [RS_59]
                                       PartitionCols:_col0
-                                      Group By Operator [GBY_58] (rows=114314 width=3)
+                                      Group By Operator [GBY_58] (rows=285115246 width=3)
                                         Output:["_col0"],keys:_col1
-                                        Merge Join Operator [MERGEJOIN_180] (rows=23255411 width=3)
-                                          Conds:RS_217._col0=RS_197._col0(Inner),Output:["_col1"]
-                                        <-Map 15 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_197]
+                                        Merge Join Operator [MERGEJOIN_180] (rows=285115246 width=3)
+                                          Conds:RS_220._col0=RS_198._col0(Inner),Output:["_col1"]
+                                        <-Map 14 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_198]
                                             PartitionCols:_col0
                                              Please refer to the previous Select Operator [SEL_193]
                                         <-Map 22 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_217]
+                                          SHUFFLE [RS_220]
                                             PartitionCols:_col0
-                                            Select Operator [SEL_216] (rows=285115246 width=7)
+                                            Select Operator [SEL_219] (rows=285115246 width=7)
                                               Output:["_col0","_col1"]
-                                              Filter Operator [FIL_215] (rows=285115246 width=7)
+                                              Filter Operator [FIL_218] (rows=285115246 width=7)
                                                 predicate:(cs_ship_customer_sk is not null and cs_sold_date_sk is not null and cs_ship_customer_sk BETWEEN DynamicValue(RS_62_c_c_customer_sk_min) AND DynamicValue(RS_62_c_c_customer_sk_max) and in_bloom_filter(cs_ship_customer_sk, DynamicValue(RS_62_c_c_customer_sk_bloom_filter)))
                                                 TableScan [TS_48] (rows=287989836 width=7)
                                                   default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_ship_customer_sk"]
                                                 <-Reducer 9 [BROADCAST_EDGE] vectorized
-                                                  BROADCAST [RS_214]
-                                                    Group By Operator [GBY_213] (rows=1 width=12)
+                                                  BROADCAST [RS_217]
+                                                    Group By Operator [GBY_216] (rows=1 width=12)
                                                       Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
                                                     <-Reducer 5 [CUSTOM_SIMPLE_EDGE]
                                                       PARTITION_ONLY_SHUFFLE [RS_167]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query7.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query7.q.out
index ed01a9e..ea07ca3 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query7.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query7.q.out
@@ -83,9 +83,9 @@ Stage-0
                     PartitionCols:_col0
                     Group By Operator [GBY_27] (rows=462000 width=476)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["sum(_col4)","count(_col4)","sum(_col5)","count(_col5)","sum(_col7)","count(_col7)","sum(_col6)","count(_col6)"],keys:_col12
-                      Top N Key Operator [TNK_54] (rows=1655321 width=100)
+                      Top N Key Operator [TNK_54] (rows=4635977 width=100)
                         keys:_col12,sort order:+,top n:100
-                        Merge Join Operator [MERGEJOIN_98] (rows=1655321 width=100)
+                        Merge Join Operator [MERGEJOIN_98] (rows=4635977 width=100)
                           Conds:RS_23._col1=RS_117._col0(Inner),Output:["_col4","_col5","_col6","_col7","_col12"]
                         <-Map 12 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_117]
@@ -97,7 +97,7 @@ Stage-0
                         <-Reducer 4 [SIMPLE_EDGE]
                           SHUFFLE [RS_23]
                             PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_97] (rows=1655321 width=4)
+                            Merge Join Operator [MERGEJOIN_97] (rows=4635977 width=4)
                               Conds:RS_20._col3=RS_115._col0(Inner),Output:["_col1","_col4","_col5","_col6","_col7"]
                             <-Map 11 [SIMPLE_EDGE] vectorized
                               SHUFFLE [RS_115]
@@ -111,7 +111,7 @@ Stage-0
                             <-Reducer 3 [SIMPLE_EDGE]
                               SHUFFLE [RS_20]
                                 PartitionCols:_col3
-                                Merge Join Operator [MERGEJOIN_96] (rows=1655321 width=4)
+                                Merge Join Operator [MERGEJOIN_96] (rows=4635977 width=4)
                                   Conds:RS_17._col0=RS_112._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7"]
                                 <-Map 10 [SIMPLE_EDGE] vectorized
                                   SHUFFLE [RS_112]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query70.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query70.q.out
index f1c93ee..c33e75b 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query70.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query70.q.out
@@ -101,38 +101,38 @@ Stage-0
     Stage-1
       Reducer 6 vectorized
       File Output Operator [FS_171]
-        Limit [LIM_170] (rows=1 width=492)
+        Limit [LIM_170] (rows=100 width=492)
           Number of rows:100
-          Select Operator [SEL_169] (rows=1 width=492)
+          Select Operator [SEL_169] (rows=720 width=492)
             Output:["_col0","_col1","_col2","_col3","_col4"]
           <-Reducer 5 [SIMPLE_EDGE] vectorized
             SHUFFLE [RS_168]
-              Select Operator [SEL_167] (rows=1 width=492)
+              Select Operator [SEL_167] (rows=720 width=492)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                PTF Operator [PTF_166] (rows=1 width=304)
+                PTF Operator [PTF_166] (rows=720 width=304)
                   Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col2 DESC NULLS LAST","partition by:":"(grouping(_col3, 1L) + grouping(_col3, 0L)), CASE WHEN ((grouping(_col3, 0L) = UDFToLong(0))) THEN (_col0) ELSE (CAST( null AS STRING)) END"}]
-                  Select Operator [SEL_165] (rows=1 width=304)
+                  Select Operator [SEL_165] (rows=720 width=304)
                     Output:["_col0","_col1","_col2","_col3"]
                   <-Reducer 4 [SIMPLE_EDGE] vectorized
                     SHUFFLE [RS_164]
                       PartitionCols:(grouping(_col3, 1L) + grouping(_col3, 0L)), CASE WHEN ((grouping(_col3, 0L) = UDFToLong(0))) THEN (_col0) ELSE (CAST( null AS STRING)) END
-                      Select Operator [SEL_163] (rows=1 width=304)
+                      Select Operator [SEL_163] (rows=720 width=304)
                         Output:["_col0","_col1","_col2","_col3"]
-                        Group By Operator [GBY_162] (rows=1 width=304)
+                        Group By Operator [GBY_162] (rows=720 width=304)
                           Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                         <-Reducer 3 [SIMPLE_EDGE]
                           SHUFFLE [RS_50]
                             PartitionCols:_col0, _col1, _col2
-                            Group By Operator [GBY_49] (rows=1 width=304)
+                            Group By Operator [GBY_49] (rows=430560 width=304)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col2)"],keys:_col0, _col1, 0L
-                              Select Operator [SEL_47] (rows=1 width=296)
+                              Select Operator [SEL_47] (rows=525329897 width=290)
                                 Output:["_col0","_col1","_col2"]
-                                Merge Join Operator [MERGEJOIN_137] (rows=1 width=296)
+                                Merge Join Operator [MERGEJOIN_137] (rows=525329897 width=290)
                                   Conds:RS_44._col1=RS_45._col0(Inner),Output:["_col2","_col6","_col7"]
                                 <-Reducer 2 [SIMPLE_EDGE]
                                   SHUFFLE [RS_44]
                                     PartitionCols:_col1
-                                    Merge Join Operator [MERGEJOIN_133] (rows=91197860 width=85)
+                                    Merge Join Operator [MERGEJOIN_133] (rows=525329897 width=110)
                                       Conds:RS_148._col0=RS_140._col0(Inner),Output:["_col1","_col2"]
                                     <-Map 11 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_140]
@@ -196,11 +196,11 @@ Stage-0
                                                   <-Reducer 7 [SIMPLE_EDGE]
                                                     SHUFFLE [RS_26]
                                                       PartitionCols:_col0
-                                                      Group By Operator [GBY_25] (rows=2989 width=198)
+                                                      Group By Operator [GBY_25] (rows=19404 width=198)
                                                         Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col5
-                                                        Top N Key Operator [TNK_87] (rows=91197860 width=168)
+                                                        Top N Key Operator [TNK_87] (rows=525329897 width=192)
                                                           keys:_col5,sort order:+,top n:6
-                                                          Merge Join Operator [MERGEJOIN_135] (rows=91197860 width=168)
+                                                          Merge Join Operator [MERGEJOIN_135] (rows=525329897 width=192)
                                                             Conds:RS_21._col1=RS_151._col0(Inner),Output:["_col2","_col5"]
                                                           <-Reducer 2 [SIMPLE_EDGE]
                                                             SHUFFLE [RS_21]
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query71.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query71.q.out
index 83065fd..1238e6a 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query71.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query71.q.out
@@ -111,20 +111,20 @@ Stage-0
     Stage-1
       Reducer 7 vectorized
       File Output Operator [FS_174]
-        Select Operator [SEL_173] (rows=1991967 width=223)
+        Select Operator [SEL_173] (rows=145803092 width=223)
           Output:["_col0","_col1","_col2","_col3","_col4"]
         <-Reducer 6 [SIMPLE_EDGE] vectorized
           SHUFFLE [RS_172]
-            Select Operator [SEL_171] (rows=1991967 width=227)
+            Select Operator [SEL_171] (rows=145803092 width=227)
               Output:["_col1","_col2","_col3","_col4","_col5"]
-              Group By Operator [GBY_170] (rows=1991967 width=223)
+              Group By Operator [GBY_170] (rows=145803092 width=223)
                 Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
               <-Reducer 5 [SIMPLE_EDGE]
                 SHUFFLE [RS_46]
                   PartitionCols:_col0, _col1, _col2, _col3
-                  Group By Operator [GBY_45] (rows=1991967 width=223)
+                  Group By Operator [GBY_45] (rows=145803092 width=223)
                     Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col0)"],keys:_col4, _col7, _col8, _col5
-                    Merge Join Operator [MERGEJOIN_140] (rows=1991967 width=112)
+                    Merge Join Operator [MERGEJOIN_140] (rows=145803092 width=204)
                       Conds:RS_41._col2=RS_169._col0(Inner),Output:["_col0","_col4","_col5","_col7","_col8"]
                     <-Map 19 [SIMPLE_EDGE] vectorized
                       SHUFFLE [RS_169]
@@ -138,7 +138,7 @@ Stage-0
                     <-Reducer 4 [SIMPLE_EDGE]
                       SHUFFLE [RS_41]
                         PartitionCols:_col2
-                        Merge Join Operator [MERGEJOIN_139] (rows=3983933 width=104)
+                        Merge Join Operator [MERGEJOIN_139] (rows=145803092 width=200)
                           Conds:Union 3._col1=RS_166._col0(Inner),Output:["_col0","_col2","_col4","_col5"]
                         <-Map 18 [SIMPLE_EDGE] vectorized
                           SHUFFLE [RS_166]
@@ -153,9 +153,9 @@ Stage-0
                           <-Reducer 11 [CONTAINS]
                             Reduce Output Operator [RS_148]
                               PartitionCols:_col1
-                              Select Operator [SEL_146] (rows=7751851 width=98)
+                              Select Operator [SEL_146] (rows=285116947 width=119)
                                 Output:["_col0","_col1","_col2"]
-                                Merge Join Operator [MERGEJOIN_145] (rows=7751851 width=98)
+                                Merge Join Operator [MERGEJOIN_145] (rows=285116947 width=119)
                                   Conds:RS_185._col0=RS_177._col0(Inner),Output:["_col1","_col2","_col3"]
                                 <-Map 12 [SIMPLE_EDGE] vectorized
                                   PARTITION_ONLY_SHUFFLE [RS_177]
@@ -189,9 +189,9 @@ Stage-0
                           <-Reducer 15 [CONTAINS]
                             Reduce Output Operator [RS_152]
                               PartitionCols:_col1
-                              Select Operator [SEL_150] (rows=14384397 width=4)
+                              Select Operator [SEL_150] (rows=525325345 width=114)
                                 Output:["_col0","_col1","_col2"]
-                                Merge Join Operator [MERGEJOIN_149] (rows=14384397 width=4)
+                                Merge Join Operator [MERGEJOIN_149] (rows=525325345 width=114)
                                   Conds:RS_196._col0=RS_188._col0(Inner),Output:["_col1","_col2","_col3"]
                                 <-Map 16 [SIMPLE_EDGE] vectorized
                                   PARTITION_ONLY_SHUFFLE [RS_188]
@@ -225,9 +225,9 @@ Stage-0
                           <-Reducer 2 [CONTAINS]
                             Reduce Output Operator [RS_144]
                               PartitionCols:_col1
-                              Select Operator [SEL_142] (rows=3941098 width=118)
+                              Select Operator [SEL_142] (rows=143930836 width=119)
                                 Output:["_col0","_col1","_col2"]
-                                Merge Join Operator [MERGEJOIN_141] (rows=3941098 width=118)
... 11909 lines suppressed ...