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

[hive] branch master updated: HIVE-21338: Remove order by and limit for aggregates (Vineet Garg, reviewed by Jesus Camacho Rodriguez)

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

vgarg pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new a074c7c  HIVE-21338: Remove order by and limit for aggregates (Vineet Garg, reviewed by Jesus Camacho Rodriguez)
a074c7c is described below

commit a074c7c2eddc85016baf54b2724c394253137e32
Author: Vineet Garg <vg...@apache.org>
AuthorDate: Wed Mar 13 10:40:57 2019 -0700

    HIVE-21338: Remove order by and limit for aggregates (Vineet Garg, reviewed by Jesus Camacho Rodriguez)
---
 .../calcite/rules/HiveSortLimitRemoveRule.java     |  57 ++
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |   7 +
 ql/src/test/queries/clientpositive/cbo_limit.q     |  20 +
 .../test/results/clientpositive/keep_uniform.q.out |  96 +--
 .../results/clientpositive/llap/cbo_limit.q.out    | 127 ++++
 .../results/clientpositive/llap/keep_uniform.q.out | 205 +++---
 .../llap/vector_binary_join_groupby.q.out          |  32 +-
 .../clientpositive/llap/vectorization_0.q.out      | 630 +-----------------
 .../llap/vectorized_date_funcs.q.out               | 117 +---
 .../llap/vectorized_shufflejoin.q.out              |  49 +-
 .../clientpositive/parquet_vectorization_0.q.out   | 609 +----------------
 .../clientpositive/perf/spark/query16.q.out        |  57 +-
 .../clientpositive/perf/spark/query23.q.out        |  21 +-
 .../clientpositive/perf/spark/query28.q.out        |  19 +-
 .../clientpositive/perf/spark/query32.q.out        |  18 +-
 .../clientpositive/perf/spark/query38.q.out        |  18 +-
 .../clientpositive/perf/spark/query61.q.out        | 131 ++--
 .../clientpositive/perf/spark/query90.q.out        |  92 ++-
 .../clientpositive/perf/spark/query92.q.out        |  90 +--
 .../clientpositive/perf/spark/query94.q.out        |  57 +-
 .../clientpositive/perf/spark/query95.q.out        | 115 ++--
 .../clientpositive/perf/spark/query96.q.out        |  44 +-
 .../clientpositive/perf/spark/query97.q.out        |  18 +-
 .../clientpositive/perf/tez/cbo_query16.q.out      |  57 +-
 .../clientpositive/perf/tez/cbo_query23.q.out      | 302 +++++----
 .../clientpositive/perf/tez/cbo_query28.q.out      |  39 +-
 .../clientpositive/perf/tez/cbo_query32.q.out      |  48 +-
 .../clientpositive/perf/tez/cbo_query38.q.out      | 104 ++-
 .../clientpositive/perf/tez/cbo_query61.q.out      |  99 ++-
 .../clientpositive/perf/tez/cbo_query90.q.out      |  75 ++-
 .../clientpositive/perf/tez/cbo_query92.q.out      |  49 +-
 .../clientpositive/perf/tez/cbo_query94.q.out      |  57 +-
 .../clientpositive/perf/tez/cbo_query95.q.out      |  85 ++-
 .../clientpositive/perf/tez/cbo_query96.q.out      |  35 +-
 .../clientpositive/perf/tez/cbo_query97.q.out      |  44 +-
 .../perf/tez/constraints/cbo_query16.q.out         |  55 +-
 .../perf/tez/constraints/cbo_query23.q.out         | 246 ++++---
 .../perf/tez/constraints/cbo_query28.q.out         |  39 +-
 .../perf/tez/constraints/cbo_query32.q.out         |  48 +-
 .../perf/tez/constraints/cbo_query38.q.out         |  98 ++-
 .../perf/tez/constraints/cbo_query61.q.out         |  99 ++-
 .../perf/tez/constraints/cbo_query90.q.out         |  75 ++-
 .../perf/tez/constraints/cbo_query92.q.out         |  49 +-
 .../perf/tez/constraints/cbo_query94.q.out         |  55 +-
 .../perf/tez/constraints/cbo_query95.q.out         |  73 +-
 .../perf/tez/constraints/cbo_query96.q.out         |  35 +-
 .../perf/tez/constraints/cbo_query97.q.out         |  44 +-
 .../perf/tez/constraints/query16.q.out             | 271 ++++----
 .../perf/tez/constraints/query23.q.out             | 688 ++++++++++---------
 .../perf/tez/constraints/query28.q.out             | 322 +++++----
 .../perf/tez/constraints/query32.q.out             | 200 +++---
 .../perf/tez/constraints/query38.q.out             | 356 +++++-----
 .../perf/tez/constraints/query61.q.out             | 389 ++++++-----
 .../perf/tez/constraints/query90.q.out             | 237 ++++---
 .../perf/tez/constraints/query92.q.out             | 227 +++----
 .../perf/tez/constraints/query94.q.out             | 271 ++++----
 .../perf/tez/constraints/query95.q.out             | 419 ++++++------
 .../perf/tez/constraints/query96.q.out             | 149 ++---
 .../perf/tez/constraints/query97.q.out             | 180 +++--
 .../results/clientpositive/perf/tez/query16.q.out  | 275 ++++----
 .../results/clientpositive/perf/tez/query23.q.out  | 736 ++++++++++-----------
 .../results/clientpositive/perf/tez/query28.q.out  | 322 +++++----
 .../results/clientpositive/perf/tez/query32.q.out  | 200 +++---
 .../results/clientpositive/perf/tez/query38.q.out  | 360 +++++-----
 .../results/clientpositive/perf/tez/query61.q.out  | 389 ++++++-----
 .../results/clientpositive/perf/tez/query90.q.out  | 237 ++++---
 .../results/clientpositive/perf/tez/query92.q.out  | 227 +++----
 .../results/clientpositive/perf/tez/query94.q.out  | 275 ++++----
 .../results/clientpositive/perf/tez/query95.q.out  | 427 ++++++------
 .../results/clientpositive/perf/tez/query96.q.out  | 149 ++---
 .../results/clientpositive/perf/tez/query97.q.out  | 180 +++--
 .../test/results/clientpositive/ppd_udf_col.q.out  |  34 +-
 .../results/clientpositive/spark/cbo_limit.q.out   | 127 ++++
 .../spark/parquet_vectorization_0.q.out            | 528 +--------------
 .../clientpositive/spark/vectorization_0.q.out     | 624 +----------------
 .../spark/vectorized_shufflejoin.q.out             |  49 +-
 .../vector_binary_join_groupby.q.out               |  50 +-
 .../clientpositive/vectorized_date_funcs.q.out     | 113 +---
 .../clientpositive/vectorized_shufflejoin.q.out    |  49 +-
 79 files changed, 5463 insertions(+), 8136 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitRemoveRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitRemoveRule.java
new file mode 100644
index 0000000..858aa1a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitRemoveRule.java
@@ -0,0 +1,57 @@
+/*
+ * 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.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+
+/**
+ * Planner rule that removes
+ * a {@link org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit}.
+ * Note that this is different from HiveSortRemoveRule because this is not based on statistics
+ */
+public final class HiveSortLimitRemoveRule extends RelOptRule {
+
+  public static final HiveSortLimitRemoveRule INSTANCE =
+      new HiveSortLimitRemoveRule();
+
+  private HiveSortLimitRemoveRule() {
+    super(operand(HiveSortLimit.class, any()), HiveRelFactories.HIVE_BUILDER, null);
+  }
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final HiveSortLimit sortLimit = call.rel(0);
+
+    Double maxRowCount = call.getMetadataQuery().getMaxRowCount(sortLimit.getInput());
+    if (maxRowCount != null &&(maxRowCount <= 1)) {
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final HiveSortLimit sortLimit = call.rel(0);
+
+    // We remove the limit operator
+    call.transformTo(sortLimit.getInput());
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index a7b9e62..b87c507 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -212,6 +212,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRulesRegistry;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSemiJoinRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortJoinReduceRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortLimitPullUpConstantsRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortLimitRemoveRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortMergeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortProjectTransposeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortRemoveRule;
@@ -2144,6 +2145,12 @@ public class CalcitePlanner extends SemanticAnalyzer {
       perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER,
         "Calcite: Prejoin ordering transformation, Push Down Semi Joins"); */
 
+      perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
+      basePlan = hepPlan(basePlan, false, mdProvider, executorProvider,
+                                    HiveSortLimitRemoveRule.INSTANCE);
+      perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER,
+                            "Calcite: Trying to remove Limit and Order by");
+
       // 6. Apply Partition Pruning
       perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
       basePlan = hepPlan(basePlan, false, mdProvider, executorProvider, new HivePartitionPruneRule(conf));
diff --git a/ql/src/test/queries/clientpositive/cbo_limit.q b/ql/src/test/queries/clientpositive/cbo_limit.q
index 24c1eae..d0b1bc9 100644
--- a/ql/src/test/queries/clientpositive/cbo_limit.q
+++ b/ql/src/test/queries/clientpositive/cbo_limit.q
@@ -17,3 +17,23 @@ select key, c_int from(select key, c_int from (select key, c_int from cbo_t1 ord
 select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0) group by c_float, cbo_t1.c_int, key order by a limit 5) cbo_t1 join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key order by q/10 desc, r asc limit 5) cbo_t2 on cbo_t1.a=p join cbo_t3 on cbo_t1 [...]
 
 select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b % c asc, b desc limit 5) cbo_t1 left outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)   [...]
+
+-- order by and limit
+explain cbo select count(*) cs from cbo_t1 where c_int > 1 order by cs limit 100;
+select count(*) cs from cbo_t1 where c_int > 1 order by cs limit 100;
+
+-- only order by
+explain cbo select count(*) cs from cbo_t1 where c_int > 1 order by cs ;
+select count(*) cs from cbo_t1 where c_int > 1 order by cs ;
+
+-- only LIMIT
+explain cbo select count(*) cs from cbo_t1 where c_int > 1 LIMIT 100;
+select count(*) cs from cbo_t1 where c_int > 1 LIMIT 100;
+
+-- LIMIT 1
+explain cbo select c_int from (select c_int from cbo_t1 where c_float > 1.0 limit 1) subq  where c_int > 1 order by c_int;
+select c_int from (select c_int from cbo_t1 where c_float > 1.0 limit 1) subq  where c_int > 1 order by c_int;
+
+-- prune un-necessary aggregates
+explain cbo select count(*) from cbo_t1 order by sum(c_int), count(*);
+select count(*) from cbo_t1 order by sum(c_int), count(*);
diff --git a/ql/src/test/results/clientpositive/keep_uniform.q.out b/ql/src/test/results/clientpositive/keep_uniform.q.out
index 959696f..1b18696 100644
--- a/ql/src/test/results/clientpositive/keep_uniform.q.out
+++ b/ql/src/test/results/clientpositive/keep_uniform.q.out
@@ -431,18 +431,17 @@ PLAN VECTORIZATION:
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-2 depends on stages: Stage-1
-  Stage-3 depends on stages: Stage-2, Stage-12
-  Stage-4 depends on stages: Stage-3, Stage-15
+  Stage-3 depends on stages: Stage-2, Stage-11
+  Stage-4 depends on stages: Stage-3, Stage-14
   Stage-5 depends on stages: Stage-4
   Stage-6 depends on stages: Stage-5
   Stage-7 depends on stages: Stage-6
-  Stage-8 depends on stages: Stage-7
-  Stage-11 is a root stage
-  Stage-12 depends on stages: Stage-11
-  Stage-16 is a root stage
-  Stage-14 depends on stages: Stage-16
-  Stage-15 depends on stages: Stage-14
-  Stage-0 depends on stages: Stage-8
+  Stage-10 is a root stage
+  Stage-11 depends on stages: Stage-10
+  Stage-15 is a root stage
+  Stage-13 depends on stages: Stage-15
+  Stage-14 depends on stages: Stage-13
+  Stage-0 depends on stages: Stage-7
 
 STAGE PLANS:
   Stage: Stage-1
@@ -785,72 +784,15 @@ STAGE PLANS:
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-          Select Operator
-            expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2)), _col0 (type: bigint)
-            outputColumnNames: _col1, _col2, _col3
-            Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-            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-8
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            TableScan Vectorization:
-                native: true
-                vectorizationSchemaColumns: [0:_col1:decimal(17,2), 1:_col2:decimal(17,2), 2:_col3:bigint]
-            Reduce Output Operator
-              key expressions: _col3 (type: bigint)
-              sort order: +
-              Reduce Sink Vectorization:
-                  className: VectorReduceSinkOperator
-                  native: false
-                  nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-              TopN Hash Memory Usage: 0.1
-              value expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2))
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-          rowBatchContext:
-              dataColumnCount: 3
-              includeColumns: [0, 1, 2]
-              dataColumns: _col1:decimal(17,2), _col2:decimal(17,2), _col3:bigint
-              partitionColumnCount: 0
-              scratchColumnTypeNames: []
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: decimal(17,2)), VALUE._col1 (type: decimal(17,2))
-          outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 100
+          File Output Operator
+            compressed: false
             Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            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-11
+  Stage: Stage-10
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -922,7 +864,7 @@ STAGE PLANS:
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-12
+  Stage: Stage-11
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -972,7 +914,7 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-16
+  Stage: Stage-15
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -1039,7 +981,7 @@ STAGE PLANS:
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                     serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-14
+  Stage: Stage-13
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -1092,7 +1034,7 @@ STAGE PLANS:
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-15
+  Stage: Stage-14
     Map Reduce
       Map Operator Tree:
           TableScan
diff --git a/ql/src/test/results/clientpositive/llap/cbo_limit.q.out b/ql/src/test/results/clientpositive/llap/cbo_limit.q.out
index 87a5770..ee62b31 100644
--- a/ql/src/test/results/clientpositive/llap/cbo_limit.q.out
+++ b/ql/src/test/results/clientpositive/llap/cbo_limit.q.out
@@ -86,3 +86,130 @@ POSTHOOK: Input: default@cbo_t3
 #### A masked pattern was here ####
 1	12	6
 1	2	6
+PREHOOK: query: explain cbo select count(*) cs from cbo_t1 where c_int > 1 order by cs limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select count(*) cs from cbo_t1 where c_int > 1 order by cs limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveFilter(condition=[>($2, 1)])
+    HiveTableScan(table=[[default, cbo_t1]], table:alias=[cbo_t1])
+
+PREHOOK: query: select count(*) cs from cbo_t1 where c_int > 1 order by cs limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) cs from cbo_t1 where c_int > 1 order by cs limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+0
+PREHOOK: query: explain cbo select count(*) cs from cbo_t1 where c_int > 1 order by cs
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select count(*) cs from cbo_t1 where c_int > 1 order by cs
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveFilter(condition=[>($2, 1)])
+    HiveTableScan(table=[[default, cbo_t1]], table:alias=[cbo_t1])
+
+PREHOOK: query: select count(*) cs from cbo_t1 where c_int > 1 order by cs
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) cs from cbo_t1 where c_int > 1 order by cs
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+0
+PREHOOK: query: explain cbo select count(*) cs from cbo_t1 where c_int > 1 LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select count(*) cs from cbo_t1 where c_int > 1 LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveFilter(condition=[>($2, 1)])
+    HiveTableScan(table=[[default, cbo_t1]], table:alias=[cbo_t1])
+
+PREHOOK: query: select count(*) cs from cbo_t1 where c_int > 1 LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) cs from cbo_t1 where c_int > 1 LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+0
+PREHOOK: query: explain cbo select c_int from (select c_int from cbo_t1 where c_float > 1.0 limit 1) subq  where c_int > 1 order by c_int
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select c_int from (select c_int from cbo_t1 where c_float > 1.0 limit 1) subq  where c_int > 1 order by c_int
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+CBO PLAN:
+HiveFilter(condition=[>($0, 1)])
+  HiveProject(c_int=[$0])
+    HiveSortLimit(fetch=[1])
+      HiveProject(c_int=[$2])
+        HiveFilter(condition=[>($3, 1.0E0)])
+          HiveTableScan(table=[[default, cbo_t1]], table:alias=[cbo_t1])
+
+PREHOOK: query: select c_int from (select c_int from cbo_t1 where c_float > 1.0 limit 1) subq  where c_int > 1 order by c_int
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: select c_int from (select c_int from cbo_t1 where c_float > 1.0 limit 1) subq  where c_int > 1 order by c_int
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+PREHOOK: query: explain cbo select count(*) from cbo_t1 order by sum(c_int), count(*)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select count(*) from cbo_t1 order by sum(c_int), count(*)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveTableScan(table=[[default, cbo_t1]], table:alias=[cbo_t1])
+
+PREHOOK: query: select count(*) from cbo_t1 order by sum(c_int), count(*)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from cbo_t1 order by sum(c_int), count(*)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+#### A masked pattern was here ####
+20
diff --git a/ql/src/test/results/clientpositive/llap/keep_uniform.q.out b/ql/src/test/results/clientpositive/llap/keep_uniform.q.out
index 44b1ba4..c3a4117 100644
--- a/ql/src/test/results/clientpositive/llap/keep_uniform.q.out
+++ b/ql/src/test/results/clientpositive/llap/keep_uniform.q.out
@@ -437,18 +437,17 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 13 <- Map 12 (SIMPLE_EDGE), Map 17 (SIMPLE_EDGE)
-        Reducer 14 <- Reducer 13 (SIMPLE_EDGE)
-        Reducer 15 <- Map 18 (SIMPLE_EDGE), Reducer 13 (ONE_TO_ONE_EDGE)
-        Reducer 16 <- Reducer 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 <- Reducer 14 (ONE_TO_ONE_EDGE), Reducer 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 16 (ONE_TO_ONE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE)
-        Reducer 6 <- Map 19 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 12 <- Map 11 (SIMPLE_EDGE), Map 16 (SIMPLE_EDGE)
+        Reducer 13 <- Reducer 12 (SIMPLE_EDGE)
+        Reducer 14 <- Map 17 (SIMPLE_EDGE), Reducer 12 (ONE_TO_ONE_EDGE)
+        Reducer 15 <- Reducer 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 <- Reducer 13 (ONE_TO_ONE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 15 (ONE_TO_ONE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE)
+        Reducer 6 <- Map 18 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
         Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
-        Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -507,57 +506,6 @@ STAGE PLANS:
         Map 10 
             Map Operator Tree:
                 TableScan
-                  alias: customer_address
-                  filterExpr: ((ca_state = 'TX') and ca_address_sk is not null) (type: boolean)
-                  Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  TableScan Vectorization:
-                      native: true
-                      vectorizationSchemaColumns: [0:ca_address_sk:int, 1:ca_address_id:string, 2:ca_street_number:string, 3:ca_street_name:string, 4:ca_street_type:string, 5:ca_suite_number:string, 6:ca_city:string, 7:ca_county:string, 8:ca_state:string, 9:ca_zip:string, 10:ca_country:string, 11:ca_gmt_offset:decimal(5,2)/DECIMAL_64, 12:ca_location_type:string, 13:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                  Filter Operator
-                    Filter Vectorization:
-                        className: VectorFilterOperator
-                        native: true
-                        predicateExpression: FilterExprAndExpr(children: FilterStringGroupColEqualStringScalar(col 8:string, val TX), SelectColumnIsNotNull(col 0:int))
-                    predicate: ((ca_state = 'TX') and ca_address_sk is not null) (type: boolean)
-                    Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: ca_address_sk (type: int)
-                      outputColumnNames: _col0
-                      Select Vectorization:
-                          className: VectorSelectOperator
-                          native: true
-                          projectedOutputColumnNums: [0]
-                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkLongOperator
-                            keyColumns: 0:int
-                            native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized, llap
-            LLAP IO: may be used (ACID table)
-            Map Vectorization:
-                enabled: true
-                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-                inputFormatFeatureSupport: [DECIMAL_64]
-                featureSupportInUse: [DECIMAL_64]
-                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: true
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 13
-                    includeColumns: [0, 8]
-                    dataColumns: ca_address_sk:int, ca_address_id:string, ca_street_number:string, ca_street_name:string, ca_street_type:string, ca_suite_number:string, ca_city:string, ca_county:string, ca_state:string, ca_zip:string, ca_country:string, ca_gmt_offset:decimal(5,2)/DECIMAL_64, ca_location_type:string
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-        Map 11 
-            Map Operator Tree:
-                TableScan
                   alias: web_site
                   filterExpr: ((web_company_name = 'pri') and web_site_sk is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
@@ -606,7 +554,7 @@ STAGE PLANS:
                     dataColumns: web_site_sk:int, web_site_id:string, web_rec_start_date:string, web_rec_end_date:string, web_name:string, web_open_date_sk:int, web_close_date_sk:int, web_class:string, web_manager:string, web_mkt_id:int, web_mkt_class:string, web_mkt_desc:string, web_market_manager:string, web_company_id:int, web_company_name:string, web_street_number:string, web_street_name:string, web_street_type:string, web_suite_number:string, web_city:string, web_county:string, web_ [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 12 
+        Map 11 
             Map Operator Tree:
                 TableScan
                   alias: ws1
@@ -659,7 +607,7 @@ STAGE PLANS:
                     dataColumns: ws_sold_date_sk:int, ws_sold_time_sk:int, ws_ship_date_sk:int, ws_item_sk:int, ws_bill_customer_sk:int, ws_bill_cdemo_sk:int, ws_bill_hdemo_sk:int, ws_bill_addr_sk:int, ws_ship_customer_sk:int, ws_ship_cdemo_sk:int, ws_ship_hdemo_sk:int, ws_ship_addr_sk:int, ws_web_page_sk:int, ws_web_site_sk:int, ws_ship_mode_sk:int, ws_warehouse_sk:int, ws_promo_sk:int, ws_order_number:int, ws_quantity:int, ws_wholesale_cost:decimal(7,2)/DECIMAL_64, ws_list_price:decima [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 17 
+        Map 16 
             Map Operator Tree:
                 TableScan
                   alias: ws2
@@ -713,7 +661,7 @@ STAGE PLANS:
                     dataColumns: ws_sold_date_sk:int, ws_sold_time_sk:int, ws_ship_date_sk:int, ws_item_sk:int, ws_bill_customer_sk:int, ws_bill_cdemo_sk:int, ws_bill_hdemo_sk:int, ws_bill_addr_sk:int, ws_ship_customer_sk:int, ws_ship_cdemo_sk:int, ws_ship_hdemo_sk:int, ws_ship_addr_sk:int, ws_web_page_sk:int, ws_web_site_sk:int, ws_ship_mode_sk:int, ws_warehouse_sk:int, ws_promo_sk:int, ws_order_number:int, ws_quantity:int, ws_wholesale_cost:decimal(7,2)/DECIMAL_64, ws_list_price:decima [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 18 
+        Map 17 
             Map Operator Tree:
                 TableScan
                   alias: web_returns
@@ -765,7 +713,7 @@ STAGE PLANS:
                     dataColumns: wr_returned_date_sk:int, wr_returned_time_sk:int, wr_item_sk:int, wr_refunded_customer_sk:int, wr_refunded_cdemo_sk:int, wr_refunded_hdemo_sk:int, wr_refunded_addr_sk:int, wr_returning_customer_sk:int, wr_returning_cdemo_sk:int, wr_returning_hdemo_sk:int, wr_returning_addr_sk:int, wr_web_page_sk:int, wr_reason_sk:int, wr_order_number:int, wr_return_quantity:int, wr_return_amt:decimal(7,2)/DECIMAL_64, wr_return_tax:decimal(7,2)/DECIMAL_64, wr_return_amt_in [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: []
-        Map 19 
+        Map 18 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -816,7 +764,58 @@ STAGE PLANS:
                     dataColumns: d_date_sk:int, d_date_id:string, d_date:string, d_month_seq:int, d_week_seq:int, d_quarter_seq:int, d_year:int, d_dow:int, d_moy:int, d_dom:int, d_qoy:int, d_fy_year:int, d_fy_quarter_seq:int, d_fy_week_seq:int, d_day_name:string, d_quarter_name:string, d_holiday:string, d_weekend:string, d_following_holiday:string, d_first_dom:int, d_last_dom:int, d_same_day_ly:int, d_same_day_lq:int, d_current_day:string, d_current_week:string, d_current_month:string, d [...]
                     partitionColumnCount: 0
                     scratchColumnTypeNames: [timestamp]
-        Reducer 13 
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: customer_address
+                  filterExpr: ((ca_state = 'TX') and ca_address_sk is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                  TableScan Vectorization:
+                      native: true
+                      vectorizationSchemaColumns: [0:ca_address_sk:int, 1:ca_address_id:string, 2:ca_street_number:string, 3:ca_street_name:string, 4:ca_street_type:string, 5:ca_suite_number:string, 6:ca_city:string, 7:ca_county:string, 8:ca_state:string, 9:ca_zip:string, 10:ca_country:string, 11:ca_gmt_offset:decimal(5,2)/DECIMAL_64, 12:ca_location_type:string, 13:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+                  Filter Operator
+                    Filter Vectorization:
+                        className: VectorFilterOperator
+                        native: true
+                        predicateExpression: FilterExprAndExpr(children: FilterStringGroupColEqualStringScalar(col 8:string, val TX), SelectColumnIsNotNull(col 0:int))
+                    predicate: ((ca_state = 'TX') and ca_address_sk is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ca_address_sk (type: int)
+                      outputColumnNames: _col0
+                      Select Vectorization:
+                          className: VectorSelectOperator
+                          native: true
+                          projectedOutputColumnNums: [0]
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Reduce Sink Vectorization:
+                            className: VectorReduceSinkLongOperator
+                            keyColumns: 0:int
+                            native: true
+                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: may be used (ACID table)
+            Map Vectorization:
+                enabled: true
+                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
+                inputFormatFeatureSupport: [DECIMAL_64]
+                featureSupportInUse: [DECIMAL_64]
+                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                allNative: true
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 13
+                    includeColumns: [0, 8]
+                    dataColumns: ca_address_sk:int, ca_address_id:string, ca_street_number:string, ca_street_name:string, ca_street_type:string, ca_suite_number:string, ca_city:string, ca_county:string, ca_state:string, ca_zip:string, ca_country:string, ca_gmt_offset:decimal(5,2)/DECIMAL_64, ca_location_type:string
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
+        Reducer 12 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -854,7 +853,7 @@ STAGE PLANS:
             MergeJoin Vectorization:
                 enabled: false
                 enableConditionsNotMet: Vectorizing MergeJoin Supported IS false
-        Reducer 14 
+        Reducer 13 
             Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
@@ -893,7 +892,7 @@ STAGE PLANS:
                       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
                       partitionColumns: 0:int
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-        Reducer 15 
+        Reducer 14 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -917,7 +916,7 @@ STAGE PLANS:
             MergeJoin Vectorization:
                 enabled: false
                 enableConditionsNotMet: Vectorizing MergeJoin Supported IS false
-        Reducer 16 
+        Reducer 15 
             Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
@@ -1141,66 +1140,16 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2)), _col0 (type: bigint)
-                  outputColumnNames: _col1, _col2, _col3
-                  Select Vectorization:
-                      className: VectorSelectOperator
-                      native: true
-                      projectedOutputColumnNums: [1, 2, 0]
-                  Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col3 (type: bigint)
-                    sort order: +
-                    Reduce Sink Vectorization:
-                        className: VectorReduceSinkObjectHashOperator
-                        keyColumns: 0:bigint
-                        native: true
-                        nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                        valueColumns: 1:decimal(17,2), 2:decimal(17,2)
-                    Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                    TopN Hash Memory Usage: 0.1
-                    value expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2))
-        Reducer 9 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: z
-                reduceColumnSortOrder: +
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 3
-                    dataColumns: KEY.reducesinkkey0:bigint, VALUE._col0:decimal(17,2), VALUE._col1:decimal(17,2)
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: decimal(17,2)), VALUE._col1 (type: decimal(17,2))
-                outputColumnNames: _col0, _col1, _col2
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0, 1, 2]
-                Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
-                  Limit Vectorization:
-                      className: VectorLimitOperator
-                      native: true
+                File Output Operator
+                  compressed: false
+                  File Sink Vectorization:
+                      className: VectorFileSinkOperator
+                      native: false
                   Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    File Sink Vectorization:
-                        className: VectorFileSinkOperator
-                        native: false
-                    Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  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
diff --git a/ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out b/ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out
index b7f2287..5881607 100644
--- a/ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out
@@ -122,9 +122,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 4 (BROADCAST_EDGE)
+        Map 1 <- Map 3 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -164,7 +163,7 @@ STAGE PLANS:
                             hashTableImplementationType: OPTIMIZED
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21
                         input vertices:
-                          1 Map 4
+                          1 Map 3
                         Statistics: Num rows: 10000 Data size: 6819968 Basic stats: COMPLETE Column stats: COMPLETE
                         Select Operator
                           expressions: hash(_col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7,_col8,_col9,_col10,_col11,_col12,_col13,_col14,_col15,_col16,_col17,_col18,_col19,_col20,_col21) (type: int)
@@ -206,7 +205,7 @@ STAGE PLANS:
                 allNative: false
                 usesVectorUDFAdaptor: true
                 vectorized: true
-        Map 4 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: t2
@@ -271,31 +270,6 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: bigint)
-                  sort order: +
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkObjectHashOperator
-                      native: true
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint)
-                outputColumnNames: _col0
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0]
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_0.q.out b/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
index 4c7fe06..e6ff717 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_0.q.out
@@ -23,149 +23,12 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Tez
-#### A masked pattern was here ####
-      Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: alltypesorc
-                  Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-                  TableScan Vectorization:
-                      native: true
-                      vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                  Select Operator
-                    expressions: ctinyint (type: tinyint)
-                    outputColumnNames: ctinyint
-                    Select Vectorization:
-                        className: VectorSelectOperator
-                        native: true
-                        projectedOutputColumnNums: [0]
-                    Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: min(ctinyint), max(ctinyint), count(ctinyint), count()
-                      Group By Vectorization:
-                          aggregators: VectorUDAFMinLong(col 0:tinyint) -> tinyint, VectorUDAFMaxLong(col 0:tinyint) -> tinyint, VectorUDAFCount(col 0:tinyint) -> bigint, VectorUDAFCountStar(*) -> bigint
-                          className: VectorGroupByOperator
-                          groupByMode: HASH
-                          native: false
-                          vectorProcessingMode: HASH
-                          projectedOutputColumnNums: [0, 1, 2, 3]
-                      mode: hash
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        sort order: 
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkEmptyKeyOperator
-                            native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                            valueColumns: 0:tinyint, 1:tinyint, 2:bigint, 3:bigint
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: tinyint), _col1 (type: tinyint), _col2 (type: bigint), _col3 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-            Map Vectorization:
-                enabled: true
-                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-                inputFormatFeatureSupport: [DECIMAL_64]
-                featureSupportInUse: [DECIMAL_64]
-                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 12
-                    includeColumns: [0]
-                    dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: 
-                reduceColumnSortOrder: 
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 4
-                    dataColumns: VALUE._col0:tinyint, VALUE._col1:tinyint, VALUE._col2:bigint, VALUE._col3:bigint
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3)
-                Group By Vectorization:
-                    aggregators: VectorUDAFMinLong(col 0:tinyint) -> tinyint, VectorUDAFMaxLong(col 1:tinyint) -> tinyint, VectorUDAFCountMerge(col 2:bigint) -> bigint, VectorUDAFCountMerge(col 3:bigint) -> bigint
-                    className: VectorGroupByOperator
-                    groupByMode: MERGEPARTIAL
-                    native: false
-                    vectorProcessingMode: GLOBAL
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: tinyint)
-                  sort order: +
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkObjectHashOperator
-                      keyColumns: 0:tinyint
-                      native: true
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      valueColumns: 1:tinyint, 2:bigint, 3:bigint
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: tinyint), _col2 (type: bigint), _col3 (type: bigint)
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: z
-                reduceColumnSortOrder: +
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 4
-                    dataColumns: KEY.reducesinkkey0:tinyint, VALUE._col0:tinyint, VALUE._col1:bigint, VALUE._col2:bigint
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: tinyint), VALUE._col0 (type: tinyint), VALUE._col1 (type: bigint), VALUE._col2 (type: bigint)
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  File Sink Vectorization:
-                      className: VectorFileSinkOperator
-                      native: false
-                  Statistics: Num rows: 1 Data size: 24 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
+      limit: 1
       Processor Tree:
         ListSink
 
@@ -216,7 +79,6 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -301,39 +163,6 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: bigint)
-                  sort order: +
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkObjectHashOperator
-                      keyColumns: 0:bigint
-                      native: true
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: z
-                reduceColumnSortOrder: +
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 1
-                    dataColumns: KEY.reducesinkkey0:bigint
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint)
-                outputColumnNames: _col0
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0]
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
@@ -408,7 +237,6 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -455,34 +283,16 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: (UDFToDouble(_col0) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END), 0.5) (type: double)
-                  outputColumnNames: _col0, _col1, _col3, _col4, _col7
+                  expressions: (UDFToDouble(_col0) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _c [...]
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                   Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: double)
-                    sort order: +
+                  File Output Operator
+                    compressed: false
                     Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: double), _col3 (type: double), _col4 (type: double), _col7 (type: double)
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: double), VALUE._col0 (type: double), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 1 Data size: 64 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
+                    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
@@ -544,149 +354,12 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Tez
-#### A masked pattern was here ####
-      Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: alltypesorc
-                  Statistics: Num rows: 12288 Data size: 73392 Basic stats: COMPLETE Column stats: COMPLETE
-                  TableScan Vectorization:
-                      native: true
-                      vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                  Select Operator
-                    expressions: cbigint (type: bigint)
-                    outputColumnNames: cbigint
-                    Select Vectorization:
-                        className: VectorSelectOperator
-                        native: true
-                        projectedOutputColumnNums: [3]
-                    Statistics: Num rows: 12288 Data size: 73392 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: min(cbigint), max(cbigint), count(cbigint), count()
-                      Group By Vectorization:
-                          aggregators: VectorUDAFMinLong(col 3:bigint) -> bigint, VectorUDAFMaxLong(col 3:bigint) -> bigint, VectorUDAFCount(col 3:bigint) -> bigint, VectorUDAFCountStar(*) -> bigint
-                          className: VectorGroupByOperator
-                          groupByMode: HASH
-                          native: false
-                          vectorProcessingMode: HASH
-                          projectedOutputColumnNums: [0, 1, 2, 3]
-                      mode: hash
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        sort order: 
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkEmptyKeyOperator
-                            native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                            valueColumns: 0:bigint, 1:bigint, 2:bigint, 3:bigint
-                        Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-            Map Vectorization:
-                enabled: true
-                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-                inputFormatFeatureSupport: [DECIMAL_64]
-                featureSupportInUse: [DECIMAL_64]
-                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 12
-                    includeColumns: [3]
-                    dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: 
-                reduceColumnSortOrder: 
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 4
-                    dataColumns: VALUE._col0:bigint, VALUE._col1:bigint, VALUE._col2:bigint, VALUE._col3:bigint
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3)
-                Group By Vectorization:
-                    aggregators: VectorUDAFMinLong(col 0:bigint) -> bigint, VectorUDAFMaxLong(col 1:bigint) -> bigint, VectorUDAFCountMerge(col 2:bigint) -> bigint, VectorUDAFCountMerge(col 3:bigint) -> bigint
-                    className: VectorGroupByOperator
-                    groupByMode: MERGEPARTIAL
-                    native: false
-                    vectorProcessingMode: GLOBAL
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: bigint)
-                  sort order: +
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkObjectHashOperator
-                      keyColumns: 0:bigint
-                      native: true
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      valueColumns: 1:bigint, 2:bigint, 3:bigint
-                  Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint)
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: z
-                reduceColumnSortOrder: +
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 4
-                    dataColumns: KEY.reducesinkkey0:bigint, VALUE._col0:bigint, VALUE._col1:bigint, VALUE._col2:bigint
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: bigint), VALUE._col1 (type: bigint), VALUE._col2 (type: bigint)
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  File Sink Vectorization:
-                      className: VectorFileSinkOperator
-                      native: false
-                  Statistics: Num rows: 1 Data size: 32 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
+      limit: 1
       Processor Tree:
         ListSink
 
@@ -737,7 +410,6 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -822,39 +494,6 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: bigint)
-                  sort order: +
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkObjectHashOperator
-                      keyColumns: 0:bigint
-                      native: true
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: z
-                reduceColumnSortOrder: +
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 1
-                    dataColumns: KEY.reducesinkkey0:bigint
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint)
-                outputColumnNames: _col0
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0]
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
@@ -929,7 +568,6 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -976,34 +614,16 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: (UDFToDouble(_col0) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END), 0.5) (type: double)
-                  outputColumnNames: _col0, _col1, _col3, _col4, _col7
+                  expressions: (UDFToDouble(_col0) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _c [...]
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                   Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: double)
-                    sort order: +
+                  File Output Operator
+                    compressed: false
                     Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: double), _col3 (type: double), _col4 (type: double), _col7 (type: double)
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: double), VALUE._col0 (type: double), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 1 Data size: 64 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
+                    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
@@ -1065,149 +685,12 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Tez
-#### A masked pattern was here ####
-      Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: alltypesorc
-                  Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-                  TableScan Vectorization:
-                      native: true
-                      vectorizationSchemaColumns: [0:ctinyint:tinyint, 1:csmallint:smallint, 2:cint:int, 3:cbigint:bigint, 4:cfloat:float, 5:cdouble:double, 6:cstring1:string, 7:cstring2:string, 8:ctimestamp1:timestamp, 9:ctimestamp2:timestamp, 10:cboolean1:boolean, 11:cboolean2:boolean, 12:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                  Select Operator
-                    expressions: cfloat (type: float)
-                    outputColumnNames: cfloat
-                    Select Vectorization:
-                        className: VectorSelectOperator
-                        native: true
-                        projectedOutputColumnNums: [4]
-                    Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: min(cfloat), max(cfloat), count(cfloat), count()
-                      Group By Vectorization:
-                          aggregators: VectorUDAFMinDouble(col 4:float) -> float, VectorUDAFMaxDouble(col 4:float) -> float, VectorUDAFCount(col 4:float) -> bigint, VectorUDAFCountStar(*) -> bigint
-                          className: VectorGroupByOperator
-                          groupByMode: HASH
-                          native: false
-                          vectorProcessingMode: HASH
-                          projectedOutputColumnNums: [0, 1, 2, 3]
-                      mode: hash
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        sort order: 
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkEmptyKeyOperator
-                            native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                            valueColumns: 0:float, 1:float, 2:bigint, 3:bigint
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: bigint), _col3 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-            Map Vectorization:
-                enabled: true
-                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-                inputFormatFeatureSupport: [DECIMAL_64]
-                featureSupportInUse: [DECIMAL_64]
-                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 12
-                    includeColumns: [4]
-                    dataColumns: ctinyint:tinyint, csmallint:smallint, cint:int, cbigint:bigint, cfloat:float, cdouble:double, cstring1:string, cstring2:string, ctimestamp1:timestamp, ctimestamp2:timestamp, cboolean1:boolean, cboolean2:boolean
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: 
-                reduceColumnSortOrder: 
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 4
-                    dataColumns: VALUE._col0:float, VALUE._col1:float, VALUE._col2:bigint, VALUE._col3:bigint
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3)
-                Group By Vectorization:
-                    aggregators: VectorUDAFMinDouble(col 0:float) -> float, VectorUDAFMaxDouble(col 1:float) -> float, VectorUDAFCountMerge(col 2:bigint) -> bigint, VectorUDAFCountMerge(col 3:bigint) -> bigint
-                    className: VectorGroupByOperator
-                    groupByMode: MERGEPARTIAL
-                    native: false
-                    vectorProcessingMode: GLOBAL
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: float)
-                  sort order: +
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkObjectHashOperator
-                      keyColumns: 0:float
-                      native: true
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      valueColumns: 1:float, 2:bigint, 3:bigint
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: float), _col2 (type: bigint), _col3 (type: bigint)
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: z
-                reduceColumnSortOrder: +
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 4
-                    dataColumns: KEY.reducesinkkey0:float, VALUE._col0:float, VALUE._col1:bigint, VALUE._col2:bigint
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: float), VALUE._col0 (type: float), VALUE._col1 (type: bigint), VALUE._col2 (type: bigint)
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  File Sink Vectorization:
-                      className: VectorFileSinkOperator
-                      native: false
-                  Statistics: Num rows: 1 Data size: 24 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
+      limit: 1
       Processor Tree:
         ListSink
 
@@ -1258,7 +741,6 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1343,39 +825,6 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: double)
-                  sort order: +
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkObjectHashOperator
-                      keyColumns: 0:double
-                      native: true
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                reduceColumnNullOrder: z
-                reduceColumnSortOrder: +
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 1
-                    dataColumns: KEY.reducesinkkey0:double
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: double)
-                outputColumnNames: _col0
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0]
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   File Sink Vectorization:
@@ -1450,7 +899,6 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1497,34 +945,16 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: (_col0 / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END), 0.5) (type: double)
-                  outputColumnNames: _col0, _col1, _col3, _col4, _col7
+                  expressions: (_col0 / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1) [...]
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                   Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: double)
-                    sort order: +
+                  File Output Operator
+                    compressed: false
                     Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: double), _col3 (type: double), _col4 (type: double), _col7 (type: double)
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: double), VALUE._col0 (type: double), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 1 Data size: 64 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
+                    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
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out b/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out
index c4dafee..108c8ee 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out
@@ -1248,125 +1248,12 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Tez
-#### A masked pattern was here ####
-      Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
-#### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: date_udf_flight_orc
-                  Statistics: Num rows: 137 Data size: 7672 Basic stats: COMPLETE Column stats: COMPLETE
-                  TableScan Vectorization:
-                      native: true
-                  Select Operator
-                    expressions: fl_date (type: date)
-                    outputColumnNames: fl_date
-                    Select Vectorization:
-                        className: VectorSelectOperator
-                        native: true
-                        projectedOutputColumnNums: [0]
-                    Statistics: Num rows: 137 Data size: 7672 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: min(fl_date), max(fl_date), count(fl_date), count()
-                      Group By Vectorization:
-                          aggregators: VectorUDAFMinLong(col 0:date) -> date, VectorUDAFMaxLong(col 0:date) -> date, VectorUDAFCount(col 0:date) -> bigint, VectorUDAFCountStar(*) -> bigint
-                          className: VectorGroupByOperator
-                          groupByMode: HASH
-                          native: false
-                          vectorProcessingMode: HASH
-                          projectedOutputColumnNums: [0, 1, 2, 3]
-                      mode: hash
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        sort order: 
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkEmptyKeyOperator
-                            native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                        Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: bigint), _col3 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-            Map Vectorization:
-                enabled: true
-                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-                inputFormatFeatureSupport: [DECIMAL_64]
-                featureSupportInUse: [DECIMAL_64]
-                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3)
-                Group By Vectorization:
-                    aggregators: VectorUDAFMinLong(col 0:date) -> date, VectorUDAFMaxLong(col 1:date) -> date, VectorUDAFCountMerge(col 2:bigint) -> bigint, VectorUDAFCountMerge(col 3:bigint) -> bigint
-                    className: VectorGroupByOperator
-                    groupByMode: MERGEPARTIAL
-                    native: false
-                    vectorProcessingMode: GLOBAL
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: date)
-                  sort order: +
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkObjectHashOperator
-                      native: true
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: date), _col2 (type: bigint), _col3 (type: bigint)
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: date), VALUE._col1 (type: bigint), VALUE._col2 (type: bigint)
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  File Sink Vectorization:
-                      className: VectorFileSinkOperator
-                      native: false
-                  Statistics: Num rows: 1 Data size: 128 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
+      limit: 1
       Processor Tree:
         ListSink
 
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out b/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out
index 3d517a6..24e1128 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_shufflejoin.q.out
@@ -23,9 +23,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -71,7 +70,7 @@ STAGE PLANS:
                 allNative: true
                 usesVectorUDFAdaptor: false
                 vectorized: true
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: t2
@@ -171,42 +170,16 @@ STAGE PLANS:
                       projectedOutputColumnNums: [0, 1, 2, 6]
                       selectExpressions: DoubleColDivideLongColumn(col 5:double, col 4:bigint)(children: CastLongToDouble(col 3:bigint) -> 5:double) -> 6:double
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: bigint)
-                    sort order: +
-                    Reduce Sink Vectorization:
-                        className: VectorReduceSinkObjectHashOperator
-                        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
+                  File Output Operator
+                    compressed: false
+                    File Sink Vectorization:
+                        className: VectorFileSinkOperator
+                        native: false
                     Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: double)
-        Reducer 4 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: double)
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  File Sink Vectorization:
-                      className: VectorFileSinkOperator
-                      native: false
-                  Statistics: Num rows: 1 Data size: 24 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
+                    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
diff --git a/ql/src/test/results/clientpositive/parquet_vectorization_0.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_0.q.out
index 833f31e..c2d1514 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_0.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_0.q.out
@@ -23,121 +23,12 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: alltypesparquet
-            Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-            TableScan Vectorization:
-                native: true
-            Select Operator
-              expressions: ctinyint (type: tinyint)
-              outputColumnNames: ctinyint
-              Select Vectorization:
-                  className: VectorSelectOperator
-                  native: true
-                  projectedOutputColumnNums: [0]
-              Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-              Group By Operator
-                aggregations: min(ctinyint), max(ctinyint), count(ctinyint), count()
-                Group By Vectorization:
-                    aggregators: VectorUDAFMinLong(col 0:tinyint) -> tinyint, VectorUDAFMaxLong(col 0:tinyint) -> tinyint, VectorUDAFCount(col 0:tinyint) -> bigint, VectorUDAFCountStar(*) -> bigint
-                    className: VectorGroupByOperator
-                    groupByMode: HASH
-                    native: false
-                    vectorProcessingMode: HASH
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                mode: hash
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkOperator
-                      native: false
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: tinyint), _col1 (type: tinyint), _col2 (type: bigint), _col3 (type: bigint)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 1 Data size: 24 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
-            TableScan Vectorization:
-                native: true
-            Reduce Output Operator
-              key expressions: _col0 (type: tinyint)
-              sort order: +
-              Reduce Sink Vectorization:
-                  className: VectorReduceSinkOperator
-                  native: false
-                  nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: tinyint), _col2 (type: bigint), _col3 (type: bigint)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: tinyint), VALUE._col0 (type: tinyint), VALUE._col1 (type: bigint), VALUE._col2 (type: bigint)
-          outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 24 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
+      limit: 1
       Processor Tree:
         ListSink
 
@@ -180,8 +71,7 @@ PLAN VECTORIZATION:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
   Stage: Stage-1
@@ -243,47 +133,6 @@ STAGE PLANS:
           Statistics: Num rows: 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
-            TableScan Vectorization:
-                native: true
-            Reduce Output Operator
-              key expressions: _col0 (type: bigint)
-              sort order: +
-              Reduce Sink Vectorization:
-                  className: VectorReduceSinkOperator
-                  native: false
-                  nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: bigint)
-          outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -345,8 +194,7 @@ PLAN VECTORIZATION:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
   Stage: Stage-1
@@ -389,51 +237,16 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: (UDFToDouble(_col0) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END), 0.5) (type: double)
-            outputColumnNames: _col0, _col1, _col3, _col4, _col7
+            expressions: (UDFToDouble(_col0) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / [...]
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
             Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
+              Statistics: Num rows: 1 Data size: 64 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.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col0 (type: double)
-              sort order: +
-              Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: double), _col3 (type: double), _col4 (type: double), _col7 (type: double)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: double), VALUE._col0 (type: double), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-          Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 64 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
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -495,121 +308,12 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: alltypesparquet
-            Statistics: Num rows: 12288 Data size: 73392 Basic stats: COMPLETE Column stats: COMPLETE
-            TableScan Vectorization:
-                native: true
-            Select Operator
-              expressions: cbigint (type: bigint)
-              outputColumnNames: cbigint
-              Select Vectorization:
-                  className: VectorSelectOperator
-                  native: true
-                  projectedOutputColumnNums: [3]
-              Statistics: Num rows: 12288 Data size: 73392 Basic stats: COMPLETE Column stats: COMPLETE
-              Group By Operator
-                aggregations: min(cbigint), max(cbigint), count(cbigint), count()
-                Group By Vectorization:
-                    aggregators: VectorUDAFMinLong(col 3:bigint) -> bigint, VectorUDAFMaxLong(col 3:bigint) -> bigint, VectorUDAFCount(col 3:bigint) -> bigint, VectorUDAFCountStar(*) -> bigint
-                    className: VectorGroupByOperator
-                    groupByMode: HASH
-                    native: false
-                    vectorProcessingMode: HASH
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                mode: hash
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkOperator
-                      native: false
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                  Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 1 Data size: 32 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
-            TableScan Vectorization:
-                native: true
-            Reduce Output Operator
-              key expressions: _col0 (type: bigint)
-              sort order: +
-              Reduce Sink Vectorization:
-                  className: VectorReduceSinkOperator
-                  native: false
-                  nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: bigint), VALUE._col1 (type: bigint), VALUE._col2 (type: bigint)
-          outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 32 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
+      limit: 1
       Processor Tree:
         ListSink
 
@@ -652,8 +356,7 @@ PLAN VECTORIZATION:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
   Stage: Stage-1
@@ -715,47 +418,6 @@ STAGE PLANS:
           Statistics: Num rows: 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
-            TableScan Vectorization:
-                native: true
-            Reduce Output Operator
-              key expressions: _col0 (type: bigint)
-              sort order: +
-              Reduce Sink Vectorization:
-                  className: VectorReduceSinkOperator
-                  native: false
-                  nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: bigint)
-          outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -817,8 +479,7 @@ PLAN VECTORIZATION:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
   Stage: Stage-1
@@ -861,51 +522,16 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: (UDFToDouble(_col0) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END), 0.5) (type: double)
-            outputColumnNames: _col0, _col1, _col3, _col4, _col7
+            expressions: (UDFToDouble(_col0) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / [...]
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
             Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
+              Statistics: Num rows: 1 Data size: 64 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.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col0 (type: double)
-              sort order: +
-              Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: double), _col3 (type: double), _col4 (type: double), _col7 (type: double)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: double), VALUE._col0 (type: double), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-          Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 64 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
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -967,121 +593,12 @@ PLAN VECTORIZATION:
   enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
 
 STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: alltypesparquet
-            Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-            TableScan Vectorization:
-                native: true
-            Select Operator
-              expressions: cfloat (type: float)
-              outputColumnNames: cfloat
-              Select Vectorization:
-                  className: VectorSelectOperator
-                  native: true
-                  projectedOutputColumnNums: [4]
-              Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-              Group By Operator
-                aggregations: min(cfloat), max(cfloat), count(cfloat), count()
-                Group By Vectorization:
-                    aggregators: VectorUDAFMinDouble(col 4:float) -> float, VectorUDAFMaxDouble(col 4:float) -> float, VectorUDAFCount(col 4:float) -> bigint, VectorUDAFCountStar(*) -> bigint
-                    className: VectorGroupByOperator
-                    groupByMode: HASH
-                    native: false
-                    vectorProcessingMode: HASH
-                    projectedOutputColumnNums: [0, 1, 2, 3]
-                mode: hash
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkOperator
-                      native: false
-                      nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                      nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: bigint), _col3 (type: bigint)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3)
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 1 Data size: 24 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
-            TableScan Vectorization:
-                native: true
-            Reduce Output Operator
-              key expressions: _col0 (type: float)
-              sort order: +
-              Reduce Sink Vectorization:
-                  className: VectorReduceSinkOperator
-                  native: false
-                  nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: float), _col2 (type: bigint), _col3 (type: bigint)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: float), VALUE._col0 (type: float), VALUE._col1 (type: bigint), VALUE._col2 (type: bigint)
-          outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 24 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
+      limit: 1
       Processor Tree:
         ListSink
 
@@ -1124,8 +641,7 @@ PLAN VECTORIZATION:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
   Stage: Stage-1
@@ -1187,47 +703,6 @@ STAGE PLANS:
           Statistics: Num rows: 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
-            TableScan Vectorization:
-                native: true
-            Reduce Output Operator
-              key expressions: _col0 (type: double)
-              sort order: +
-              Reduce Sink Vectorization:
-                  className: VectorReduceSinkOperator
-                  native: false
-                  nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                  nativeConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: double)
-          outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -1289,8 +764,7 @@ PLAN VECTORIZATION:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
   Stage: Stage-1
@@ -1333,51 +807,16 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 32 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: (_col0 / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END), 0.5) (type: double)
-            outputColumnNames: _col0, _col1, _col3, _col4, _col7
+            expressions: (_col0 / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / _col1) (type: double), ((_col2 - ((_col3 * _col3) / _col1)) / CASE WHEN ((_col1 = 1L)) THEN (null) ELSE ((_col1 - 1)) END) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _col1), 0.5) (type: double), power(((_col2 - ((_col3 * _col3) / _col1)) / _c [...]
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
             Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
+              Statistics: Num rows: 1 Data size: 64 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.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col0 (type: double)
-              sort order: +
-              Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: double), _col3 (type: double), _col4 (type: double), _col7 (type: double)
-      Execution mode: vectorized
-      Map Vectorization:
-          enabled: true
-          enabledConditionsMet: hive.vectorized.use.vector.serde.deserialize IS true
-          inputFormatFeatureSupport: []
-          featureSupportInUse: []
-          inputFileFormats: org.apache.hadoop.mapred.SequenceFileInputFormat
-          allNative: false
-          usesVectorUDFAdaptor: false
-          vectorized: true
-      Reduce Vectorization:
-          enabled: false
-          enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true
-          enableConditionsNotMet: hive.execution.engine mr IN [tez, spark] IS false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: double), VALUE._col0 (type: double), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-          Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 64 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
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/perf/spark/query16.q.out b/ql/src/test/results/clientpositive/perf/spark/query16.q.out
index 52e038a..7007ac9 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query16.q.out
@@ -81,7 +81,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 10 
+        Map 9 
             Map Operator Tree:
                 TableScan
                   alias: call_center
@@ -106,7 +106,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -130,13 +130,12 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 13 <- Map 12 (GROUP, 24)
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 494), Map 9 (PARTITION-LEVEL SORT, 494)
-        Reducer 3 <- Map 11 (PARTITION-LEVEL SORT, 711), Reducer 2 (PARTITION-LEVEL SORT, 711)
-        Reducer 4 <- Reducer 13 (PARTITION-LEVEL SORT, 459), Reducer 3 (PARTITION-LEVEL SORT, 459)
+        Reducer 12 <- Map 11 (GROUP, 24)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 494), Map 8 (PARTITION-LEVEL SORT, 494)
+        Reducer 3 <- Map 10 (PARTITION-LEVEL SORT, 711), Reducer 2 (PARTITION-LEVEL SORT, 711)
+        Reducer 4 <- Reducer 12 (PARTITION-LEVEL SORT, 459), Reducer 3 (PARTITION-LEVEL SORT, 459)
         Reducer 5 <- Reducer 4 (GROUP, 246)
         Reducer 6 <- Reducer 5 (GROUP, 1)
-        Reducer 7 <- Reducer 6 (SORT, 1)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -160,7 +159,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6
                         input vertices:
-                          1 Map 8
+                          1 Map 7
                         Statistics: Num rows: 316788826 Data size: 42899570777 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
@@ -171,7 +170,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 11 
+        Map 10 
             Map Operator Tree:
                 TableScan
                   alias: cs2
@@ -196,7 +195,7 @@ STAGE PLANS:
                           Statistics: Num rows: 287989836 Data size: 38999608952 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col1 (type: int)
             Execution mode: vectorized
-        Map 12 
+        Map 11 
             Map Operator Tree:
                 TableScan
                   alias: cr1
@@ -216,7 +215,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 28798881 Data size: 3057234680 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 9 
+        Map 8 
             Map Operator Tree:
                 TableScan
                   alias: customer_address
@@ -235,7 +234,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Reducer 13 
+        Reducer 12 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -273,7 +272,7 @@ STAGE PLANS:
                     1 _col0 (type: int)
                   outputColumnNames: _col3, _col4, _col5, _col6
                   input vertices:
-                    1 Map 10
+                    1 Map 9
                   Statistics: Num rows: 383314495 Data size: 51908482889 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col4 (type: int)
@@ -357,33 +356,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2)), _col0 (type: bigint)
-                  outputColumnNames: _col1, _col2, _col3
-                  Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col3 (type: bigint)
-                    sort order: +
-                    Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                    TopN Hash Memory Usage: 0.1
-                    value expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2))
-        Reducer 7 
-            Execution mode: vectorized
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: decimal(17,2)), VALUE._col1 (type: decimal(17,2))
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  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
diff --git a/ql/src/test/results/clientpositive/perf/spark/query23.q.out b/ql/src/test/results/clientpositive/perf/spark/query23.q.out
index e85f710..d1637ae 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query23.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query23.q.out
@@ -1,5 +1,5 @@
+Warning: Map Join MAPJOIN[274][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
 Warning: Map Join MAPJOIN[275][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
-Warning: Map Join MAPJOIN[276][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: explain
 with frequent_ss_items as 
  (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
@@ -1054,7 +1054,6 @@ STAGE PLANS:
                     Reduce Output Operator
                       sort order: 
                       Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                      TopN Hash Memory Usage: 0.1
                       value expressions: _col0 (type: decimal(28,2))
         Reducer 37 
             Reduce Operator Tree:
@@ -1113,7 +1112,6 @@ STAGE PLANS:
                     Reduce Output Operator
                       sort order: 
                       Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                      TopN Hash Memory Usage: 0.1
                       value expressions: _col0 (type: decimal(28,2))
         Reducer 5 
             Execution mode: vectorized
@@ -1123,16 +1121,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 58 
             Execution mode: vectorized
             Local Work:
@@ -1172,7 +1167,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 100
+      limit: -1
       Processor Tree:
         ListSink
 
diff --git a/ql/src/test/results/clientpositive/perf/spark/query28.q.out b/ql/src/test/results/clientpositive/perf/spark/query28.q.out
index 61a77a4..8a630ba 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query28.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query28.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[94][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[93][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: explain
 select  *
 from (select avg(ss_list_price) B1_LP
@@ -550,20 +550,17 @@ STAGE PLANS:
                       expressions: _col0 (type: decimal(37,22)), _col1 (type: bigint), _col2 (type: bigint), _col15 (type: decimal(37,22)), _col16 (type: bigint), _col17 (type: bigint), _col12 (type: decimal(37,22)), _col13 (type: bigint), _col14 (type: bigint), _col9 (type: decimal(37,22)), _col10 (type: bigint), _col11 (type: bigint), _col6 (type: decimal(37,22)), _col7 (type: bigint), _col8 (type: bigint), _col3 (type: decimal(37,22)), _col4 (type: bigint), _col5 (type: bigint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
                       Statistics: Num rows: 1 Data size: 1393 Basic stats: COMPLETE Column stats: NONE
-                      Limit
-                        Number of rows: 100
+                      File Output Operator
+                        compressed: false
                         Statistics: Num rows: 1 Data size: 1393 Basic stats: COMPLETE Column stats: NONE
-                        File Output Operator
-                          compressed: false
-                          Statistics: Num rows: 1 Data size: 1393 Basic stats: COMPLETE Column stats: NONE
-                          table:
-                              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        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: 100
+      limit: -1
       Processor Tree:
         ListSink
 
diff --git a/ql/src/test/results/clientpositive/perf/spark/query32.q.out b/ql/src/test/results/clientpositive/perf/spark/query32.q.out
index 7b8b362..0467070 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query32.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query32.q.out
@@ -240,7 +240,6 @@ STAGE PLANS:
                       Reduce Output Operator
                         sort order: 
                         Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                        TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: decimal(17,2))
         Reducer 3 
             Execution mode: vectorized
@@ -250,16 +249,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 6 
             Execution mode: vectorized
             Reduce Operator Tree:
@@ -298,7 +294,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 100
+      limit: -1
       Processor Tree:
         ListSink
 
diff --git a/ql/src/test/results/clientpositive/perf/spark/query38.q.out b/ql/src/test/results/clientpositive/perf/spark/query38.q.out
index 98558ae..9cec7f3 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query38.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query38.q.out
@@ -439,7 +439,6 @@ STAGE PLANS:
                         Reduce Output Operator
                           sort order: 
                           Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                          TopN Hash Memory Usage: 0.1
                           value expressions: _col0 (type: bigint)
         Reducer 6 
             Execution mode: vectorized
@@ -449,20 +448,17 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  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: 100
+      limit: -1
       Processor Tree:
         ListSink
 
diff --git a/ql/src/test/results/clientpositive/perf/spark/query61.q.out b/ql/src/test/results/clientpositive/perf/spark/query61.q.out
index 5aad596..ec97811 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query61.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query61.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[105][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[102][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: explain
 select  promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
 from
@@ -113,7 +113,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 24 
+        Map 23 
             Map Operator Tree:
                 TableScan
                   alias: store
@@ -137,14 +137,14 @@ STAGE PLANS:
   Stage: Stage-2
     Spark
       Edges:
-        Reducer 15 <- Map 14 (PARTITION-LEVEL SORT, 697), Map 18 (PARTITION-LEVEL SORT, 697)
-        Reducer 16 <- Reducer 15 (PARTITION-LEVEL SORT, 1009), Reducer 21 (PARTITION-LEVEL SORT, 1009)
-        Reducer 17 <- Reducer 16 (GROUP, 1)
-        Reducer 20 <- Map 19 (PARTITION-LEVEL SORT, 398), Map 22 (PARTITION-LEVEL SORT, 398)
-        Reducer 21 <- Map 23 (PARTITION-LEVEL SORT, 440), Reducer 20 (PARTITION-LEVEL SORT, 440)
+        Reducer 14 <- Map 13 (PARTITION-LEVEL SORT, 697), Map 17 (PARTITION-LEVEL SORT, 697)
+        Reducer 15 <- Reducer 14 (PARTITION-LEVEL SORT, 1009), Reducer 20 (PARTITION-LEVEL SORT, 1009)
+        Reducer 16 <- Reducer 15 (GROUP, 1)
+        Reducer 19 <- Map 18 (PARTITION-LEVEL SORT, 398), Map 21 (PARTITION-LEVEL SORT, 398)
+        Reducer 20 <- Map 22 (PARTITION-LEVEL SORT, 440), Reducer 19 (PARTITION-LEVEL SORT, 440)
 #### A masked pattern was here ####
       Vertices:
-        Map 14 
+        Map 13 
             Map Operator Tree:
                 TableScan
                   alias: customer
@@ -164,7 +164,7 @@ STAGE PLANS:
                         Statistics: Num rows: 80000000 Data size: 68801615852 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: int)
             Execution mode: vectorized
-        Map 18 
+        Map 17 
             Map Operator Tree:
                 TableScan
                   alias: customer_address
@@ -183,7 +183,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 19 
+        Map 18 
             Map Operator Tree:
                 TableScan
                   alias: store_sales
@@ -203,7 +203,7 @@ STAGE PLANS:
                         Statistics: Num rows: 575995635 Data size: 50814502088 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: decimal(7,2))
             Execution mode: vectorized
-        Map 22 
+        Map 21 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -222,7 +222,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 23 
+        Map 22 
             Map Operator Tree:
                 TableScan
                   alias: item
@@ -241,7 +241,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Reducer 15 
+        Reducer 14 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -256,7 +256,7 @@ STAGE PLANS:
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 88000001 Data size: 75681779077 Basic stats: COMPLETE Column stats: NONE
-        Reducer 16 
+        Reducer 15 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -275,7 +275,7 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: decimal(17,2))
-        Reducer 17 
+        Reducer 16 
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
@@ -289,7 +289,7 @@ STAGE PLANS:
                   keys:
                     0 
                     1 
-        Reducer 20 
+        Reducer 19 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -305,7 +305,7 @@ STAGE PLANS:
                   Map-reduce partition columns: _col1 (type: int)
                   Statistics: Num rows: 633595212 Data size: 55895953508 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col2 (type: int), _col3 (type: int), _col4 (type: decimal(7,2))
-        Reducer 21 
+        Reducer 20 
             Local Work:
               Map Reduce Local Work
             Reduce Operator Tree:
@@ -325,7 +325,7 @@ STAGE PLANS:
                     1 _col0 (type: int)
                   outputColumnNames: _col2, _col4
                   input vertices:
-                    1 Map 24
+                    1 Map 23
                   Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: int)
@@ -338,7 +338,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 12 
+        Map 11 
             Map Operator Tree:
                 TableScan
                   alias: store
@@ -358,7 +358,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 13 
+        Map 12 
             Map Operator Tree:
                 TableScan
                   alias: promotion
@@ -382,12 +382,11 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 697), Map 6 (PARTITION-LEVEL SORT, 697)
-        Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1009), Reducer 9 (PARTITION-LEVEL SORT, 1009)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 697), Map 5 (PARTITION-LEVEL SORT, 697)
+        Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 1009), Reducer 8 (PARTITION-LEVEL SORT, 1009)
         Reducer 4 <- Reducer 3 (GROUP, 1)
-        Reducer 5 <- Reducer 4 (SORT, 1)
-        Reducer 8 <- Map 10 (PARTITION-LEVEL SORT, 398), Map 7 (PARTITION-LEVEL SORT, 398)
-        Reducer 9 <- Map 11 (PARTITION-LEVEL SORT, 440), Reducer 8 (PARTITION-LEVEL SORT, 440)
+        Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 398), Map 9 (PARTITION-LEVEL SORT, 398)
+        Reducer 8 <- Map 10 (PARTITION-LEVEL SORT, 440), Reducer 7 (PARTITION-LEVEL SORT, 440)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -413,25 +412,6 @@ STAGE PLANS:
         Map 10 
             Map Operator Tree:
                 TableScan
-                  alias: date_dim
-                  filterExpr: ((d_year = 1999) and (d_moy = 11) and d_date_sk is not null) (type: boolean)
-                  Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: ((d_moy = 11) and (d_year = 1999) and d_date_sk is not null) (type: boolean)
-                    Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: d_date_sk (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized
-        Map 11 
-            Map Operator Tree:
-                TableScan
                   alias: item
                   filterExpr: ((i_category = 'Electronics') and i_item_sk is not null) (type: boolean)
                   Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
@@ -448,7 +428,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 6 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: customer_address
@@ -467,7 +447,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 7 
+        Map 6 
             Map Operator Tree:
                 TableScan
                   alias: store_sales
@@ -487,6 +467,25 @@ STAGE PLANS:
                         Statistics: Num rows: 575995635 Data size: 50814502088 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: decimal(7,2))
             Execution mode: vectorized
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: date_dim
+                  filterExpr: ((d_year = 1999) and (d_moy = 11) and d_date_sk is not null) (type: boolean)
+                  Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((d_moy = 11) and (d_year = 1999) and d_date_sk is not null) (type: boolean)
+                    Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: d_date_sk (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
@@ -539,36 +538,20 @@ STAGE PLANS:
                     1 
                   outputColumnNames: _col0, _col1
                   input vertices:
-                    1 Reducer 17
+                    1 Reducer 16
                   Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: _col0 (type: decimal(17,2)), _col1 (type: decimal(17,2)), ((CAST( _col0 AS decimal(15,4)) / CAST( _col1 AS decimal(15,4))) * 100) (type: decimal(38,19))
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: decimal(17,2)), _col1 (type: decimal(17,2))
-                      sort order: ++
+                    File Output Operator
+                      compressed: false
                       Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: NONE
-                      TopN Hash Memory Usage: 0.1
-                      value expressions: _col2 (type: decimal(38,19))
-        Reducer 5 
-            Execution mode: vectorized
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: decimal(17,2)), KEY.reducesinkkey1 (type: decimal(17,2)), VALUE._col0 (type: decimal(38,19))
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
-                  Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 225 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 8 
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 7 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -584,7 +567,7 @@ STAGE PLANS:
                   Map-reduce partition columns: _col1 (type: int)
                   Statistics: Num rows: 633595212 Data size: 55895953508 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: decimal(7,2))
-        Reducer 9 
+        Reducer 8 
             Local Work:
               Map Reduce Local Work
             Reduce Operator Tree:
@@ -604,7 +587,7 @@ STAGE PLANS:
                     1 _col0 (type: int)
                   outputColumnNames: _col2, _col4, _col5
                   input vertices:
-                    1 Map 12
+                    1 Map 11
                   Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                   Map Join Operator
                     condition map:
@@ -614,7 +597,7 @@ STAGE PLANS:
                       1 _col0 (type: int)
                     outputColumnNames: _col2, _col5
                     input vertices:
-                      1 Map 13
+                      1 Map 12
                     Statistics: Num rows: 843315281 Data size: 74397518956 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col2 (type: int)
@@ -625,7 +608,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 100
+      limit: -1
       Processor Tree:
         ListSink
 
diff --git a/ql/src/test/results/clientpositive/perf/spark/query90.q.out b/ql/src/test/results/clientpositive/perf/spark/query90.q.out
index 56426c1..2a323b6 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query90.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query90.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[68][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[65][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: explain
 select  cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
  from ( select count(*) amc
@@ -65,7 +65,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 13 
+        Map 12 
             Map Operator Tree:
                 TableScan
                   alias: household_demographics
@@ -89,11 +89,11 @@ STAGE PLANS:
   Stage: Stage-2
     Spark
       Edges:
-        Reducer 10 <- Reducer 9 (GROUP, 1)
-        Reducer 9 <- Map 12 (PARTITION-LEVEL SORT, 169), Map 8 (PARTITION-LEVEL SORT, 169)
+        Reducer 8 <- Map 11 (PARTITION-LEVEL SORT, 169), Map 7 (PARTITION-LEVEL SORT, 169)
+        Reducer 9 <- Reducer 8 (GROUP, 1)
 #### A masked pattern was here ####
       Vertices:
-        Map 12 
+        Map 11 
             Map Operator Tree:
                 TableScan
                   alias: time_dim
@@ -112,7 +112,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 86400 Data size: 40694400 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: web_sales
@@ -133,7 +133,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         outputColumnNames: _col0, _col1
                         input vertices:
-                          1 Map 11
+                          1 Map 10
                         Statistics: Num rows: 158402938 Data size: 21538218500 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
@@ -144,21 +144,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Reducer 10 
-            Execution mode: vectorized
-            Local Work:
-              Map Reduce Local Work
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: count(VALUE._col0)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                Spark HashTable Sink Operator
-                  keys:
-                    0 
-                    1 
-        Reducer 9 
+        Reducer 8 
             Local Work:
               Map Reduce Local Work
             Reduce Operator Tree:
@@ -177,7 +163,7 @@ STAGE PLANS:
                     0 _col1 (type: int)
                     1 _col0 (type: int)
                   input vertices:
-                    1 Map 13
+                    1 Map 12
                   Statistics: Num rows: 191667562 Data size: 26061245514 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
@@ -188,12 +174,26 @@ STAGE PLANS:
                       sort order: 
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
+        Reducer 9 
+            Execution mode: vectorized
+            Local Work:
+              Map Reduce Local Work
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                Spark HashTable Sink Operator
+                  keys:
+                    0 
+                    1 
 
   Stage: Stage-5
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 7 
+        Map 6 
             Map Operator Tree:
                 TableScan
                   alias: household_demographics
@@ -218,7 +218,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: web_page
@@ -242,9 +242,8 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 169), Map 6 (PARTITION-LEVEL SORT, 169)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 169), Map 5 (PARTITION-LEVEL SORT, 169)
         Reducer 3 <- Reducer 2 (GROUP, 1)
-        Reducer 4 <- Reducer 3 (SORT, 1)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -268,7 +267,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         outputColumnNames: _col0, _col1
                         input vertices:
-                          1 Map 5
+                          1 Map 4
                         Statistics: Num rows: 158402938 Data size: 21538218500 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
@@ -279,7 +278,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 6 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: time_dim
@@ -317,7 +316,7 @@ STAGE PLANS:
                     0 _col1 (type: int)
                     1 _col0 (type: int)
                   input vertices:
-                    1 Map 7
+                    1 Map 6
                   Statistics: Num rows: 191667562 Data size: 26061245514 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
@@ -346,40 +345,25 @@ STAGE PLANS:
                     1 
                   outputColumnNames: _col0, _col1
                   input vertices:
-                    1 Reducer 10
+                    1 Reducer 9
                   Statistics: Num rows: 1 Data size: 17 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: (CAST( _col0 AS decimal(15,4)) / CAST( _col1 AS decimal(15,4))) (type: decimal(35,20))
                     outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 17 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: decimal(35,20))
-                      sort order: +
+                    File Output Operator
+                      compressed: false
                       Statistics: Num rows: 1 Data size: 17 Basic stats: COMPLETE Column stats: NONE
-                      TopN Hash Memory Usage: 0.1
-        Reducer 4 
-            Execution mode: vectorized
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: decimal(35,20))
-                outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 17 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
-                  Statistics: Num rows: 1 Data size: 17 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 17 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      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-4
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 11 
+        Map 10 
             Map Operator Tree:
                 TableScan
                   alias: web_page
@@ -402,7 +386,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 100
+      limit: -1
       Processor Tree:
         ListSink
 
diff --git a/ql/src/test/results/clientpositive/perf/spark/query92.q.out b/ql/src/test/results/clientpositive/perf/spark/query92.q.out
index 9aa135e..43250cb 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query92.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query92.q.out
@@ -75,7 +75,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -100,7 +100,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 9 
+        Map 8 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -124,11 +124,10 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 261), Reducer 8 (PARTITION-LEVEL SORT, 261)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 261), Reducer 7 (PARTITION-LEVEL SORT, 261)
         Reducer 3 <- Reducer 2 (GROUP, 1)
-        Reducer 4 <- Reducer 3 (SORT, 1)
-        Reducer 7 <- Map 6 (GROUP, 169)
-        Reducer 8 <- Map 10 (PARTITION-LEVEL SORT, 87), Reducer 7 (PARTITION-LEVEL SORT, 87)
+        Reducer 6 <- Map 5 (GROUP, 169)
+        Reducer 7 <- Map 9 (PARTITION-LEVEL SORT, 87), Reducer 6 (PARTITION-LEVEL SORT, 87)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -152,7 +151,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         outputColumnNames: _col1, _col2
                         input vertices:
-                          1 Map 5
+                          1 Map 4
                         Statistics: Num rows: 158402938 Data size: 21538218500 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
@@ -163,26 +162,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 10 
-            Map Operator Tree:
-                TableScan
-                  alias: item
-                  filterExpr: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
-                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
-                    Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: i_item_sk (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized
-        Map 6 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: web_sales
@@ -203,7 +183,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         outputColumnNames: _col1, _col2
                         input vertices:
-                          1 Map 9
+                          1 Map 8
                         Statistics: Num rows: 158402938 Data size: 21538218500 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
                           aggregations: sum(_col2), count(_col2)
@@ -220,6 +200,25 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: item
+                  filterExpr: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
+                  Statistics: Num rows: 462000 Data size: 663560457 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((i_manufact_id = 269) and i_item_sk is not null) (type: boolean)
+                    Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: i_item_sk (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 231000 Data size: 331780228 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
         Reducer 2 
             Reduce Operator Tree:
               Join Operator
@@ -254,33 +253,14 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: decimal(17,2))
-                  outputColumnNames: _col1
-                  Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: decimal(17,2))
-                    sort order: +
-                    Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                    TopN Hash Memory Usage: 0.1
-        Reducer 4 
-            Execution mode: vectorized
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: decimal(17,2))
-                outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 7 
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 6 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -299,7 +279,7 @@ STAGE PLANS:
                     Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 79201469 Data size: 10769109250 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col1 (type: decimal(14,7))
-        Reducer 8 
+        Reducer 7 
             Reduce Operator Tree:
               Join Operator
                 condition map:
diff --git a/ql/src/test/results/clientpositive/perf/spark/query94.q.out b/ql/src/test/results/clientpositive/perf/spark/query94.q.out
index acffe1f..044684b 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query94.q.out
@@ -77,7 +77,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 10 
+        Map 9 
             Map Operator Tree:
                 TableScan
                   alias: web_site
@@ -102,7 +102,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -126,13 +126,12 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 13 <- Map 12 (GROUP, 11)
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 327), Map 9 (PARTITION-LEVEL SORT, 327)
-        Reducer 3 <- Map 11 (PARTITION-LEVEL SORT, 357), Reducer 2 (PARTITION-LEVEL SORT, 357)
-        Reducer 4 <- Reducer 13 (PARTITION-LEVEL SORT, 230), Reducer 3 (PARTITION-LEVEL SORT, 230)
+        Reducer 12 <- Map 11 (GROUP, 11)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 327), Map 8 (PARTITION-LEVEL SORT, 327)
+        Reducer 3 <- Map 10 (PARTITION-LEVEL SORT, 357), Reducer 2 (PARTITION-LEVEL SORT, 357)
+        Reducer 4 <- Reducer 12 (PARTITION-LEVEL SORT, 230), Reducer 3 (PARTITION-LEVEL SORT, 230)
         Reducer 5 <- Reducer 4 (GROUP, 124)
         Reducer 6 <- Reducer 5 (GROUP, 1)
-        Reducer 7 <- Reducer 6 (SORT, 1)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -156,7 +155,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6
                         input vertices:
-                          1 Map 8
+                          1 Map 7
                         Statistics: Num rows: 158402938 Data size: 21538218500 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
@@ -167,7 +166,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 11 
+        Map 10 
             Map Operator Tree:
                 TableScan
                   alias: ws2
@@ -192,7 +191,7 @@ STAGE PLANS:
                           Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col1 (type: int)
             Execution mode: vectorized
-        Map 12 
+        Map 11 
             Map Operator Tree:
                 TableScan
                   alias: wr1
@@ -212,7 +211,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 14398467 Data size: 1325194184 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 9 
+        Map 8 
             Map Operator Tree:
                 TableScan
                   alias: customer_address
@@ -231,7 +230,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Reducer 13 
+        Reducer 12 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -269,7 +268,7 @@ STAGE PLANS:
                     1 _col0 (type: int)
                   outputColumnNames: _col3, _col4, _col5, _col6
                   input vertices:
-                    1 Map 10
+                    1 Map 9
                   Statistics: Num rows: 191667562 Data size: 26061245514 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col4 (type: int)
@@ -353,33 +352,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2)), _col0 (type: bigint)
-                  outputColumnNames: _col1, _col2, _col3
-                  Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col3 (type: bigint)
-                    sort order: +
-                    Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                    TopN Hash Memory Usage: 0.1
-                    value expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2))
-        Reducer 7 
-            Execution mode: vectorized
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: decimal(17,2)), VALUE._col1 (type: decimal(17,2))
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  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
diff --git a/ql/src/test/results/clientpositive/perf/spark/query95.q.out b/ql/src/test/results/clientpositive/perf/spark/query95.q.out
index 1d9fc7a..a146c6d 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query95.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query95.q.out
@@ -83,7 +83,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 9 
+        Map 8 
             Map Operator Tree:
                 TableScan
                   alias: web_site
@@ -108,7 +108,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 7 
+        Map 6 
             Map Operator Tree:
                 TableScan
                   alias: date_dim
@@ -132,16 +132,15 @@ STAGE PLANS:
   Stage: Stage-1
     Spark
       Edges:
-        Reducer 11 <- Map 10 (PARTITION-LEVEL SORT, 306), Map 13 (PARTITION-LEVEL SORT, 306)
-        Reducer 12 <- Reducer 11 (GROUP, 169)
-        Reducer 15 <- Map 14 (PARTITION-LEVEL SORT, 306), Map 18 (PARTITION-LEVEL SORT, 306)
-        Reducer 16 <- Map 19 (PARTITION-LEVEL SORT, 179), Reducer 15 (PARTITION-LEVEL SORT, 179)
-        Reducer 17 <- Reducer 16 (GROUP, 186)
-        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 327), Map 8 (PARTITION-LEVEL SORT, 327)
-        Reducer 3 <- Reducer 12 (PARTITION-LEVEL SORT, 381), Reducer 17 (PARTITION-LEVEL SORT, 381), Reducer 2 (PARTITION-LEVEL SORT, 381)
+        Reducer 10 <- Map 12 (PARTITION-LEVEL SORT, 306), Map 9 (PARTITION-LEVEL SORT, 306)
+        Reducer 11 <- Reducer 10 (GROUP, 169)
+        Reducer 14 <- Map 13 (PARTITION-LEVEL SORT, 306), Map 17 (PARTITION-LEVEL SORT, 306)
+        Reducer 15 <- Map 18 (PARTITION-LEVEL SORT, 179), Reducer 14 (PARTITION-LEVEL SORT, 179)
+        Reducer 16 <- Reducer 15 (GROUP, 186)
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 327), Map 7 (PARTITION-LEVEL SORT, 327)
+        Reducer 3 <- Reducer 11 (PARTITION-LEVEL SORT, 381), Reducer 16 (PARTITION-LEVEL SORT, 381), Reducer 2 (PARTITION-LEVEL SORT, 381)
         Reducer 4 <- Reducer 3 (GROUP, 448)
         Reducer 5 <- Reducer 4 (GROUP, 1)
-        Reducer 6 <- Reducer 5 (SORT, 1)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -165,7 +164,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         outputColumnNames: _col1, _col2, _col3, _col4, _col5
                         input vertices:
-                          1 Map 7
+                          1 Map 6
                         Statistics: Num rows: 158402938 Data size: 21538218500 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: int)
@@ -176,27 +175,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 10 
-            Map Operator Tree:
-                TableScan
-                  alias: ws1
-                  filterExpr: ws_order_number is not null (type: boolean)
-                  Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
-                  Filter Operator
-                    predicate: ws_order_number is not null (type: boolean)
-                    Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
-                    Select Operator
-                      expressions: ws_warehouse_sk (type: int), ws_order_number (type: int)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: int)
-                        Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int)
-            Execution mode: vectorized
-        Map 13 
+        Map 12 
             Map Operator Tree:
                 TableScan
                   alias: ws2
@@ -216,7 +195,7 @@ STAGE PLANS:
                         Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: int)
             Execution mode: vectorized
-        Map 14 
+        Map 13 
             Map Operator Tree:
                 TableScan
                   alias: ws1
@@ -236,7 +215,7 @@ STAGE PLANS:
                         Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: int)
             Execution mode: vectorized
-        Map 18 
+        Map 17 
             Map Operator Tree:
                 TableScan
                   alias: ws2
@@ -256,7 +235,7 @@ STAGE PLANS:
                         Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: int)
             Execution mode: vectorized
-        Map 19 
+        Map 18 
             Map Operator Tree:
                 TableScan
                   alias: web_returns
@@ -275,7 +254,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col13 (type: int)
                         Statistics: Num rows: 14398467 Data size: 1325194184 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: customer_address
@@ -294,7 +273,27 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20000000 Data size: 20297597642 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized
-        Reducer 11 
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: ws1
+                  filterExpr: ws_order_number is not null (type: boolean)
+                  Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ws_order_number is not null (type: boolean)
+                    Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ws_warehouse_sk (type: int), ws_order_number (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 144002668 Data size: 19580198212 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized
+        Reducer 10 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -321,7 +320,7 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 158402938 Data size: 21538218500 Basic stats: COMPLETE Column stats: NONE
-        Reducer 12 
+        Reducer 11 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -334,7 +333,7 @@ STAGE PLANS:
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 79201469 Data size: 10769109250 Basic stats: COMPLETE Column stats: NONE
-        Reducer 15 
+        Reducer 14 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -356,7 +355,7 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 158402938 Data size: 21538218500 Basic stats: COMPLETE Column stats: NONE
-        Reducer 16 
+        Reducer 15 
             Reduce Operator Tree:
               Join Operator
                 condition map:
@@ -376,7 +375,7 @@ STAGE PLANS:
                     sort order: +
                     Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 174243235 Data size: 23692040863 Basic stats: COMPLETE Column stats: NONE
-        Reducer 17 
+        Reducer 16 
             Execution mode: vectorized
             Reduce Operator Tree:
               Group By Operator
@@ -409,7 +408,7 @@ STAGE PLANS:
                     1 _col0 (type: int)
                   outputColumnNames: _col3, _col4, _col5
                   input vertices:
-                    1 Map 9
+                    1 Map 8
                   Statistics: Num rows: 191667562 Data size: 26061245514 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col3 (type: int)
@@ -467,33 +466,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2)), _col0 (type: bigint)
-                  outputColumnNames: _col1, _col2, _col3
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col3 (type: bigint)
-                    sort order: +
-                    Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                    TopN Hash Memory Usage: 0.1
-                    value expressions: _col1 (type: decimal(17,2)), _col2 (type: decimal(17,2))
-        Reducer 6 
-            Execution mode: vectorized
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: decimal(17,2)), VALUE._col1 (type: decimal(17,2))
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
-                  Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  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
diff --git a/ql/src/test/results/clientpositive/perf/spark/query96.q.out b/ql/src/test/results/clientpositive/perf/spark/query96.q.out
index c2ef6f0..e5900be 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query96.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query96.q.out
@@ -48,7 +48,7 @@ STAGE PLANS:
     Spark
 #### A masked pattern was here ####
       Vertices:
-        Map 4 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: time_dim
@@ -68,7 +68,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: household_demographics
@@ -88,7 +88,7 @@ STAGE PLANS:
             Execution mode: vectorized
             Local Work:
               Map Reduce Local Work
-        Map 6 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: store
@@ -113,7 +113,6 @@ STAGE PLANS:
     Spark
       Edges:
         Reducer 2 <- Map 1 (GROUP, 1)
-        Reducer 3 <- Reducer 2 (SORT, 1)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -137,7 +136,7 @@ STAGE PLANS:
                           1 _col0 (type: int)
                         outputColumnNames: _col1, _col2
                         input vertices:
-                          1 Map 4
+                          1 Map 3
                         Statistics: Num rows: 633595212 Data size: 55895953508 Basic stats: COMPLETE Column stats: NONE
                         Map Join Operator
                           condition map:
@@ -147,7 +146,7 @@ STAGE PLANS:
                             1 _col0 (type: int)
                           outputColumnNames: _col2
                           input vertices:
-                            1 Map 5
+                            1 Map 4
                           Statistics: Num rows: 696954748 Data size: 61485550191 Basic stats: COMPLETE Column stats: NONE
                           Map Join Operator
                             condition map:
@@ -156,7 +155,7 @@ STAGE PLANS:
                               0 _col2 (type: int)
                               1 _col0 (type: int)
                             input vertices:
-                              1 Map 6
+                              1 Map 5
                             Statistics: Num rows: 766650239 Data size: 67634106676 Basic stats: COMPLETE Column stats: NONE
                             Group By Operator
                               aggregations: count()
@@ -178,32 +177,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: bigint)
-                  outputColumnNames: _col1
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: bigint)
-                    sort order: +
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    TopN Hash Memory Usage: 0.1
-        Reducer 3 
-            Execution mode: vectorized
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: bigint)
-                outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  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
diff --git a/ql/src/test/results/clientpositive/perf/spark/query97.q.out b/ql/src/test/results/clientpositive/perf/spark/query97.q.out
index 1ace8e8..f9b5bc8 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query97.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query97.q.out
@@ -203,7 +203,6 @@ STAGE PLANS:
                     Reduce Output Operator
                       sort order: 
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                      TopN Hash Memory Usage: 0.1
                       value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint)
         Reducer 5 
             Execution mode: vectorized
@@ -213,16 +212,13 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                Limit
-                  Number of rows: 100
+                File Output Operator
+                  compressed: false
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 8 
             Reduce Operator Tree:
               Join Operator
@@ -259,7 +255,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 100
+      limit: -1
       Processor Tree:
         ListSink
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
index e7d1199..ef8712c 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
@@ -71,34 +71,31 @@ POSTHOOK: Input: default@customer_address
 POSTHOOK: Input: default@date_dim
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2])
-  HiveSortLimit(sort0=[$3], dir0=[ASC], fetch=[100])
-    HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2], (tok_functiondi count (tok_table_or_col cs_order_number))=[$0])
-      HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
-        HiveFilter(condition=[IS NULL($14)])
-          HiveJoin(condition=[=($4, $13)], joinType=[left], algorithm=[none], cost=[not available])
-            HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[inner])
-              HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
-                HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
-                      HiveFilter(condition=[AND(=($8, _UTF-16LE'NY'), IS NOT NULL($0))])
-                        HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                    HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
-                        HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($10), IS NOT NULL($11), IS NOT NULL($17))])
-                          HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
-                      HiveProject(d_date_sk=[$0], d_date=[$2])
-                        HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00, 2001-05-31 00:00:00), IS NOT NULL($0))])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                  HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
-                    HiveFilter(condition=[AND(IN($25, _UTF-16LE'Ziebach County', _UTF-16LE'Levy County', _UTF-16LE'Huron County', _UTF-16LE'Franklin Parish', _UTF-16LE'Daviess County'), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
-              HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
-                HiveFilter(condition=[AND(IS NOT NULL($17), IS NOT NULL($14))])
-                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
-            HiveProject(cr_order_number0=[$0], $f1=[true])
-              HiveAggregate(group=[{16}])
-                HiveFilter(condition=[IS NOT NULL($16)])
-                  HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
+HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
+  HiveFilter(condition=[IS NULL($14)])
+    HiveJoin(condition=[=($4, $13)], joinType=[left], algorithm=[none], cost=[not available])
+      HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[inner])
+        HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
+          HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
+                HiveFilter(condition=[AND(=($8, _UTF-16LE'NY'), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+              HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
+                  HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($10), IS NOT NULL($11), IS NOT NULL($17))])
+                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
+                HiveProject(d_date_sk=[$0], d_date=[$2])
+                  HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00, 2001-05-31 00:00:00), IS NOT NULL($0))])
+                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
+              HiveFilter(condition=[AND(IN($25, _UTF-16LE'Ziebach County', _UTF-16LE'Levy County', _UTF-16LE'Huron County', _UTF-16LE'Franklin Parish', _UTF-16LE'Daviess County'), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
+        HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
+          HiveFilter(condition=[AND(IS NOT NULL($17), IS NOT NULL($14))])
+            HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
+      HiveProject(cr_order_number0=[$0], $f1=[true])
+        HiveAggregate(group=[{16}])
+          HiveFilter(condition=[IS NOT NULL($16)])
+            HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
 
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 4019b26..377b587 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,7 +1,7 @@
-Warning: Shuffle Join MERGEJOIN[583][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 27' is a cross product
-Warning: Shuffle Join MERGEJOIN[584][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 28' is a cross product
-Warning: Shuffle Join MERGEJOIN[586][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 31' is a cross product
-Warning: Shuffle Join MERGEJOIN[587][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 32' is a cross product
+Warning: Shuffle Join MERGEJOIN[582][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 27' is a cross product
+Warning: Shuffle Join MERGEJOIN[583][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 28' is a cross product
+Warning: Shuffle Join MERGEJOIN[585][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 31' is a cross product
+Warning: Shuffle Join MERGEJOIN[586][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 32' 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
@@ -119,155 +119,153 @@ POSTHOOK: Input: default@store_sales
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0])
-    HiveAggregate(group=[{}], agg#0=[sum($0)])
-      HiveProject(sales=[$0])
-        HiveUnion(all=[true])
-          HiveProject(sales=[*(CAST($5):DECIMAL(10, 0), $6)])
-            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject($f0=[$0])
-                HiveJoin(condition=[>($1, *(0.95, $3))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(c_customer_sk=[$0], $f1=[$1])
-                    HiveAggregate(group=[{2}], agg#0=[sum($1)])
-                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                          HiveFilter(condition=[IS NOT NULL($3)])
-                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                        HiveProject(c_customer_sk=[$0])
-                          HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                  HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+HiveAggregate(group=[{}], agg#0=[sum($0)])
+  HiveProject(sales=[$0])
+    HiveUnion(all=[true])
+      HiveProject(sales=[*(CAST($5):DECIMAL(10, 0), $6)])
+        HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveProject($f0=[$0])
+            HiveJoin(condition=[>($1, *(0.95, $3))], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(c_customer_sk=[$0], $f1=[$1])
+                HiveAggregate(group=[{2}], agg#0=[sum($1)])
+                  HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveProject(ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                      HiveFilter(condition=[IS NOT NULL($3)])
+                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                    HiveProject(c_customer_sk=[$0])
+                      HiveFilter(condition=[IS NOT NULL($0)])
+                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+              HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(cnt=[$0])
+                  HiveFilter(condition=[<=(sq_count_check($0), 1)])
                     HiveProject(cnt=[$0])
-                      HiveFilter(condition=[<=(sq_count_check($0), 1)])
-                        HiveProject(cnt=[$0])
-                          HiveAggregate(group=[{}], cnt=[COUNT()])
-                            HiveProject
-                              HiveProject($f0=[$0])
-                                HiveAggregate(group=[{}], agg#0=[count($0)])
-                                  HiveProject(c_customer_sk=[$0], $f1=[$1])
-                                    HiveAggregate(group=[{0}], agg#0=[sum($3)])
-                                      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                                        HiveProject(c_customer_sk=[$0])
-                                          HiveFilter(condition=[IS NOT NULL($0)])
-                                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                                            HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                          HiveProject(d_date_sk=[$0])
-                                            HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
-                                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                    HiveProject($f0=[$0])
-                      HiveAggregate(group=[{}], agg#0=[max($1)])
-                        HiveProject(c_customer_sk=[$0], $f1=[$1])
-                          HiveAggregate(group=[{0}], agg#0=[sum($3)])
-                            HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                              HiveProject(c_customer_sk=[$0])
-                                HiveFilter(condition=[IS NOT NULL($0)])
-                                  HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                                  HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                HiveProject(d_date_sk=[$0])
-                                  HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
-                                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(i_item_sk=[$0])
-                  HiveAggregate(group=[{1}])
-                    HiveFilter(condition=[>($3, 4)])
-                      HiveProject(substr=[$2], i_item_sk=[$1], d_date=[$0], $f3=[$3])
-                        HiveAggregate(group=[{3, 4, 5}], agg#0=[count()])
-                          HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                              HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($2))])
-                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                              HiveProject(d_date_sk=[$0], d_date=[$2])
-                                HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
-                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                            HiveProject(i_item_sk=[$0], substr=[substr($4, 1, 30)])
-                              HiveFilter(condition=[IS NOT NULL($0)])
-                                HiveTableScan(table=[[default, item]], table:alias=[item])
-                HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
-                    HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($3), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                  HiveProject(d_date_sk=[$0])
-                    HiveFilter(condition=[AND(=($6, 1999), =($8, 1), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(sales=[*(CAST($5):DECIMAL(10, 0), $6)])
-            HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject($f0=[$0])
-                HiveJoin(condition=[>($1, *(0.95, $3))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(c_customer_sk=[$0], $f1=[$1])
-                    HiveAggregate(group=[{2}], agg#0=[sum($1)])
-                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                          HiveFilter(condition=[IS NOT NULL($3)])
-                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                        HiveProject(c_customer_sk=[$0])
+                      HiveAggregate(group=[{}], cnt=[COUNT()])
+                        HiveProject
+                          HiveProject($f0=[$0])
+                            HiveAggregate(group=[{}], agg#0=[count($0)])
+                              HiveProject(c_customer_sk=[$0], $f1=[$1])
+                                HiveAggregate(group=[{0}], agg#0=[sum($3)])
+                                  HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                                    HiveProject(c_customer_sk=[$0])
+                                      HiveFilter(condition=[IS NOT NULL($0)])
+                                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                                    HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                      HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                                        HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                      HiveProject(d_date_sk=[$0])
+                                        HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
+                                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveProject($f0=[$0])
+                  HiveAggregate(group=[{}], agg#0=[max($1)])
+                    HiveProject(c_customer_sk=[$0], $f1=[$1])
+                      HiveAggregate(group=[{0}], agg#0=[sum($3)])
+                        HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(c_customer_sk=[$0])
+                            HiveFilter(condition=[IS NOT NULL($0)])
+                              HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(i_item_sk=[$0])
+              HiveAggregate(group=[{1}])
+                HiveFilter(condition=[>($3, 4)])
+                  HiveProject(substr=[$2], i_item_sk=[$1], d_date=[$0], $f3=[$3])
+                    HiveAggregate(group=[{3, 4, 5}], agg#0=[count()])
+                      HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($2))])
+                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                          HiveProject(d_date_sk=[$0], d_date=[$2])
+                            HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
+                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                        HiveProject(i_item_sk=[$0], substr=[substr($4, 1, 30)])
                           HiveFilter(condition=[IS NOT NULL($0)])
-                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                  HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveTableScan(table=[[default, item]], table:alias=[item])
+            HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
+                HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($3), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[AND(=($6, 1999), =($8, 1), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(sales=[*(CAST($5):DECIMAL(10, 0), $6)])
+        HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveProject($f0=[$0])
+            HiveJoin(condition=[>($1, *(0.95, $3))], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(c_customer_sk=[$0], $f1=[$1])
+                HiveAggregate(group=[{2}], agg#0=[sum($1)])
+                  HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveProject(ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                      HiveFilter(condition=[IS NOT NULL($3)])
+                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                    HiveProject(c_customer_sk=[$0])
+                      HiveFilter(condition=[IS NOT NULL($0)])
+                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+              HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(cnt=[$0])
+                  HiveFilter(condition=[<=(sq_count_check($0), 1)])
                     HiveProject(cnt=[$0])
-                      HiveFilter(condition=[<=(sq_count_check($0), 1)])
-                        HiveProject(cnt=[$0])
-                          HiveAggregate(group=[{}], cnt=[COUNT()])
-                            HiveProject
-                              HiveProject($f0=[$0])
-                                HiveAggregate(group=[{}], agg#0=[count($0)])
-                                  HiveProject(c_customer_sk=[$0], $f1=[$1])
-                                    HiveAggregate(group=[{0}], agg#0=[sum($3)])
-                                      HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                                        HiveProject(c_customer_sk=[$0])
-                                          HiveFilter(condition=[IS NOT NULL($0)])
-                                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                                            HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                          HiveProject(d_date_sk=[$0])
-                                            HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
-                                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                    HiveProject($f0=[$0])
-                      HiveAggregate(group=[{}], agg#0=[max($1)])
-                        HiveProject(c_customer_sk=[$0], $f1=[$1])
-                          HiveAggregate(group=[{0}], agg#0=[sum($3)])
-                            HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                              HiveProject(c_customer_sk=[$0])
-                                HiveFilter(condition=[IS NOT NULL($0)])
-                                  HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                                  HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                HiveProject(d_date_sk=[$0])
-                                  HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
-                                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(i_item_sk=[$0])
-                  HiveAggregate(group=[{1}])
-                    HiveFilter(condition=[>($3, 4)])
-                      HiveProject(substr=[$2], i_item_sk=[$1], d_date=[$0], $f3=[$3])
-                        HiveAggregate(group=[{3, 4, 5}], agg#0=[count()])
-                          HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                              HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($2))])
-                                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                              HiveProject(d_date_sk=[$0], d_date=[$2])
-                                HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
-                                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                            HiveProject(i_item_sk=[$0], substr=[substr($4, 1, 30)])
-                              HiveFilter(condition=[IS NOT NULL($0)])
-                                HiveTableScan(table=[[default, item]], table:alias=[item])
-                HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_quantity=[$18], ws_list_price=[$20])
-                    HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                  HiveProject(d_date_sk=[$0])
-                    HiveFilter(condition=[AND(=($6, 1999), =($8, 1), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                      HiveAggregate(group=[{}], cnt=[COUNT()])
+                        HiveProject
+                          HiveProject($f0=[$0])
+                            HiveAggregate(group=[{}], agg#0=[count($0)])
+                              HiveProject(c_customer_sk=[$0], $f1=[$1])
+                                HiveAggregate(group=[{0}], agg#0=[sum($3)])
+                                  HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                                    HiveProject(c_customer_sk=[$0])
+                                      HiveFilter(condition=[IS NOT NULL($0)])
+                                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                                    HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                      HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                                        HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                      HiveProject(d_date_sk=[$0])
+                                        HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
+                                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveProject($f0=[$0])
+                  HiveAggregate(group=[{}], agg#0=[max($1)])
+                    HiveProject(c_customer_sk=[$0], $f1=[$1])
+                      HiveAggregate(group=[{0}], agg#0=[sum($3)])
+                        HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(c_customer_sk=[$0])
+                            HiveFilter(condition=[IS NOT NULL($0)])
+                              HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(i_item_sk=[$0])
+              HiveAggregate(group=[{1}])
+                HiveFilter(condition=[>($3, 4)])
+                  HiveProject(substr=[$2], i_item_sk=[$1], d_date=[$0], $f3=[$3])
+                    HiveAggregate(group=[{3, 4, 5}], agg#0=[count()])
+                      HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($2))])
+                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                          HiveProject(d_date_sk=[$0], d_date=[$2])
+                            HiveFilter(condition=[AND(IN($6, 1999, 2000, 2001, 2002), IS NOT NULL($0))])
+                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                        HiveProject(i_item_sk=[$0], substr=[substr($4, 1, 30)])
+                          HiveFilter(condition=[IS NOT NULL($0)])
+                            HiveTableScan(table=[[default, item]], table:alias=[item])
+            HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_quantity=[$18], ws_list_price=[$20])
+                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($4), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[AND(=($6, 1999), =($8, 1), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query28.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query28.q.out
index 643e5b6..e0e6153 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query28.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query28.q.out
@@ -1,8 +1,8 @@
-Warning: Shuffle Join MERGEJOIN[102][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product
-Warning: Shuffle Join MERGEJOIN[103][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
-Warning: Shuffle Join MERGEJOIN[104][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
-Warning: Shuffle Join MERGEJOIN[105][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 7' is a cross product
-Warning: Shuffle Join MERGEJOIN[106][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 8' is a cross product
+Warning: Shuffle Join MERGEJOIN[101][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[102][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[103][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[104][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[105][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 8' is a cross product
 PREHOOK: query: explain cbo
 select  *
 from (select avg(ss_list_price) B1_LP
@@ -112,35 +112,34 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f00=[$15], $f10=[$16], $f20=[$17], $f01=[$12], $f11=[$13], $f21=[$14], $f02=[$9], $f12=[$10], $f22=[$11], $f03=[$6], $f13=[$7], $f23=[$8], $f04=[$3], $f14=[$4], $f24=[$5])
+HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f00=[$15], $f10=[$16], $f20=[$17], $f01=[$12], $f11=[$13], $f21=[$14], $f02=[$9], $f12=[$10], $f22=[$11], $f03=[$6], $f13=[$7], $f23=[$8], $f04=[$3], $f14=[$4], $f24=[$5])
+  HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
     HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
       HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
         HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
           HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
-                HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-                  HiveFilter(condition=[AND(BETWEEN(false, $10, 0, 5), OR(BETWEEN(false, $12, 11, 21), BETWEEN(false, $19, 460, 1460), BETWEEN(false, $11, 14, 34)))])
-                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-              HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
-                HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-                  HiveFilter(condition=[AND(BETWEEN(false, $10, 26, 30), OR(BETWEEN(false, $12, 28, 38), BETWEEN(false, $19, 2513, 3513), BETWEEN(false, $11, 42, 62)))])
-                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
             HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
               HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-                HiveFilter(condition=[AND(BETWEEN(false, $10, 21, 25), OR(BETWEEN(false, $12, 135, 145), BETWEEN(false, $19, 14180, 15180), BETWEEN(false, $11, 38, 58)))])
+                HiveFilter(condition=[AND(BETWEEN(false, $10, 0, 5), OR(BETWEEN(false, $12, 11, 21), BETWEEN(false, $19, 460, 1460), BETWEEN(false, $11, 14, 34)))])
+                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+            HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
+              HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
+                HiveFilter(condition=[AND(BETWEEN(false, $10, 26, 30), OR(BETWEEN(false, $12, 28, 38), BETWEEN(false, $19, 2513, 3513), BETWEEN(false, $11, 42, 62)))])
                   HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
           HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
             HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-              HiveFilter(condition=[AND(BETWEEN(false, $10, 16, 20), OR(BETWEEN(false, $12, 142, 152), BETWEEN(false, $19, 3054, 4054), BETWEEN(false, $11, 80, 100)))])
+              HiveFilter(condition=[AND(BETWEEN(false, $10, 21, 25), OR(BETWEEN(false, $12, 135, 145), BETWEEN(false, $19, 14180, 15180), BETWEEN(false, $11, 38, 58)))])
                 HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
         HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
           HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-            HiveFilter(condition=[AND(BETWEEN(false, $10, 11, 15), OR(BETWEEN(false, $12, 66, 76), BETWEEN(false, $19, 920, 1920), BETWEEN(false, $11, 4, 24)))])
+            HiveFilter(condition=[AND(BETWEEN(false, $10, 16, 20), OR(BETWEEN(false, $12, 142, 152), BETWEEN(false, $19, 3054, 4054), BETWEEN(false, $11, 80, 100)))])
               HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
       HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
         HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-          HiveFilter(condition=[AND(BETWEEN(false, $10, 6, 10), OR(BETWEEN(false, $12, 91, 101), BETWEEN(false, $19, 1430, 2430), BETWEEN(false, $11, 32, 52)))])
+          HiveFilter(condition=[AND(BETWEEN(false, $10, 11, 15), OR(BETWEEN(false, $12, 66, 76), BETWEEN(false, $19, 920, 1920), BETWEEN(false, $11, 4, 24)))])
             HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+    HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
+      HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
+        HiveFilter(condition=[AND(BETWEEN(false, $10, 6, 10), OR(BETWEEN(false, $12, 91, 101), BETWEEN(false, $19, 1430, 2430), BETWEEN(false, $11, 32, 52)))])
+          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query32.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query32.q.out
index 837a743..fc353fb 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query32.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query32.q.out
@@ -61,29 +61,27 @@ POSTHOOK: Input: default@date_dim
 POSTHOOK: Input: default@item
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0])
-    HiveAggregate(group=[{}], agg#0=[sum($2)])
-      HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
-            HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
-              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-          HiveProject(d_date_sk=[$0])
-            HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00), IS NOT NULL($0))])
-              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(cs_item_sk=[$0], CAST=[$1], i_item_sk=[$2])
-          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(cs_item_sk=[$0], CAST=[CAST(*(1.3, /($1, $2))):DECIMAL(14, 7)])
-              HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-                HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
-                    HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                  HiveProject(d_date_sk=[$0])
-                    HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-            HiveProject(i_item_sk=[$0])
-              HiveFilter(condition=[AND(=($13, 269), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, item]], table:alias=[item])
+HiveAggregate(group=[{}], agg#0=[sum($2)])
+  HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
+        HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
+          HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+      HiveProject(d_date_sk=[$0])
+        HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00), IS NOT NULL($0))])
+          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+    HiveProject(cs_item_sk=[$0], CAST=[$1], i_item_sk=[$2])
+      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(cs_item_sk=[$0], CAST=[CAST(*(1.3, /($1, $2))):DECIMAL(14, 7)])
+          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
+                HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(i_item_sk=[$0])
+          HiveFilter(condition=[AND(=($13, 269), IS NOT NULL($0))])
+            HiveTableScan(table=[[default, item]], table:alias=[item])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query38.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query38.q.out
index cbf9bca..b23be25 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query38.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query38.q.out
@@ -55,57 +55,55 @@ POSTHOOK: Input: default@store_sales
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0])
-    HiveAggregate(group=[{}], agg#0=[count()])
-      HiveProject(c_last_name=[$0], c_first_name=[$1], d_date=[$2], $f3=[$3])
-        HiveFilter(condition=[=($3, 3)])
-          HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
-            HiveProject(c_last_name=[$0], c_first_name=[$1], d_date=[$2], $f3=[$3])
-              HiveUnion(all=[true])
-                HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
-                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
-                    HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
-                      HiveAggregate(group=[{1, 2, 6}])
-                        HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
-                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                            HiveProject(d_date_sk=[$0], d_date=[$2])
-                              HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
-                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
-                    HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
-                      HiveAggregate(group=[{1, 2, 6}])
-                        HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
-                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                            HiveProject(d_date_sk=[$0], d_date=[$2])
-                              HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
-                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
-                    HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
-                      HiveAggregate(group=[{1, 2, 6}])
-                        HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
-                                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                            HiveProject(d_date_sk=[$0], d_date=[$2])
-                              HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveProject(c_last_name=[$0], c_first_name=[$1], d_date=[$2], $f3=[$3])
+    HiveFilter(condition=[=($3, 3)])
+      HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
+        HiveProject(c_last_name=[$0], c_first_name=[$1], d_date=[$2], $f3=[$3])
+          HiveUnion(all=[true])
+            HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
+              HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
+                HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
+                  HiveAggregate(group=[{1, 2, 6}])
+                    HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
+                        HiveFilter(condition=[IS NOT NULL($0)])
+                          HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                        HiveProject(d_date_sk=[$0], d_date=[$2])
+                          HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
+              HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
+                HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
+                  HiveAggregate(group=[{1, 2, 6}])
+                    HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
+                        HiveFilter(condition=[IS NOT NULL($0)])
+                          HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+                            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                        HiveProject(d_date_sk=[$0], d_date=[$2])
+                          HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
+              HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
+                HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
+                  HiveAggregate(group=[{1, 2, 6}])
+                    HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
+                        HiveFilter(condition=[IS NOT NULL($0)])
+                          HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
+                            HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                        HiveProject(d_date_sk=[$0], d_date=[$2])
+                          HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query61.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query61.q.out
index 8e8cf27..3c03347 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query61.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query61.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[266][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[263][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain cbo
 select  promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
 from
@@ -102,55 +102,25 @@ POSTHOOK: Input: default@store
 POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
-  HiveProject(promotions=[$0], total=[$1], _o__c2=[*(/(CAST($0):DECIMAL(15, 4), CAST($1):DECIMAL(15, 4)), CAST(100):DECIMAL(10, 0))])
-    HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveProject($f0=[$0])
-        HiveAggregate(group=[{}], agg#0=[sum($8)])
-          HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4])
-                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
-                  HiveTableScan(table=[[default, customer]], table:alias=[customer])
-              HiveProject(ca_address_sk=[$0])
-                HiveFilter(condition=[AND(=($11, -7), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_store_sk=[$3], ss_promo_sk=[$4], ss_ext_sales_price=[$5], d_date_sk=[$6], i_item_sk=[$7], s_store_sk=[$8], p_promo_sk=[$9])
-              HiveJoin(condition=[=($4, $9)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[=($3, $8)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($1, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_promo_sk=[$8], ss_ext_sales_price=[$15])
-                        HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($8), IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($2))])
-                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                      HiveProject(d_date_sk=[$0])
-                        HiveFilter(condition=[AND(=($6, 1999), =($8, 11), IS NOT NULL($0))])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                    HiveProject(i_item_sk=[$0])
-                      HiveFilter(condition=[AND(=($12, _UTF-16LE'Electronics'), IS NOT NULL($0))])
-                        HiveTableScan(table=[[default, item]], table:alias=[item])
-                  HiveProject(s_store_sk=[$0])
-                    HiveFilter(condition=[AND(=($27, -7), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, store]], table:alias=[store])
-                HiveProject(p_promo_sk=[$0])
-                  HiveFilter(condition=[AND(OR(=($8, _UTF-16LE'Y'), =($9, _UTF-16LE'Y'), =($11, _UTF-16LE'Y')), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
-      HiveProject($f0=[$0])
-        HiveAggregate(group=[{}], agg#0=[sum($7)])
-          HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4])
-                HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
-                  HiveTableScan(table=[[default, customer]], table:alias=[customer])
-              HiveProject(ca_address_sk=[$0])
-                HiveFilter(condition=[AND(=($11, -7), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_store_sk=[$3], ss_ext_sales_price=[$4], d_date_sk=[$5], i_item_sk=[$6], s_store_sk=[$7])
-              HiveJoin(condition=[=($3, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[=($1, $6)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ext_sales_price=[$15])
-                      HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($2))])
+HiveProject(promotions=[$0], total=[$1], _o__c2=[*(/(CAST($0):DECIMAL(15, 4), CAST($1):DECIMAL(15, 4)), CAST(100):DECIMAL(10, 0))])
+  HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveProject($f0=[$0])
+      HiveAggregate(group=[{}], agg#0=[sum($8)])
+        HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4])
+              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
+                HiveTableScan(table=[[default, customer]], table:alias=[customer])
+            HiveProject(ca_address_sk=[$0])
+              HiveFilter(condition=[AND(=($11, -7), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_store_sk=[$3], ss_promo_sk=[$4], ss_ext_sales_price=[$5], d_date_sk=[$6], i_item_sk=[$7], s_store_sk=[$8], p_promo_sk=[$9])
+            HiveJoin(condition=[=($4, $9)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveJoin(condition=[=($3, $8)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveJoin(condition=[=($1, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_promo_sk=[$8], ss_ext_sales_price=[$15])
+                      HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($8), IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($2))])
                         HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
                     HiveProject(d_date_sk=[$0])
                       HiveFilter(condition=[AND(=($6, 1999), =($8, 11), IS NOT NULL($0))])
@@ -161,4 +131,33 @@ HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
                 HiveProject(s_store_sk=[$0])
                   HiveFilter(condition=[AND(=($27, -7), IS NOT NULL($0))])
                     HiveTableScan(table=[[default, store]], table:alias=[store])
+              HiveProject(p_promo_sk=[$0])
+                HiveFilter(condition=[AND(OR(=($8, _UTF-16LE'Y'), =($9, _UTF-16LE'Y'), =($11, _UTF-16LE'Y')), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
+    HiveProject($f0=[$0])
+      HiveAggregate(group=[{}], agg#0=[sum($7)])
+        HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4])
+              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
+                HiveTableScan(table=[[default, customer]], table:alias=[customer])
+            HiveProject(ca_address_sk=[$0])
+              HiveFilter(condition=[AND(=($11, -7), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_store_sk=[$3], ss_ext_sales_price=[$4], d_date_sk=[$5], i_item_sk=[$6], s_store_sk=[$7])
+            HiveJoin(condition=[=($3, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveJoin(condition=[=($1, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ext_sales_price=[$15])
+                    HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($2))])
+                      HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                  HiveProject(d_date_sk=[$0])
+                    HiveFilter(condition=[AND(=($6, 1999), =($8, 11), IS NOT NULL($0))])
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveProject(i_item_sk=[$0])
+                  HiveFilter(condition=[AND(=($12, _UTF-16LE'Electronics'), IS NOT NULL($0))])
+                    HiveTableScan(table=[[default, item]], table:alias=[item])
+              HiveProject(s_store_sk=[$0])
+                HiveFilter(condition=[AND(=($27, -7), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, store]], table:alias=[store])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query90.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query90.q.out
index 123b3c2..ba67338 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query90.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query90.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[152][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[149][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
 PREHOOK: query: explain cbo
 select  cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
  from ( select count(*) amc
@@ -52,41 +52,40 @@ POSTHOOK: Input: default@web_page
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100])
-  HiveProject(am_pm_ratio=[/(CAST($0):DECIMAL(15, 4), CAST($1):DECIMAL(15, 4))])
-    HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveProject($f0=[$0])
-        HiveAggregate(group=[{}], agg#0=[count()])
-          HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ws_sold_time_sk=[$1], ws_ship_hdemo_sk=[$10], ws_web_page_sk=[$12])
-                  HiveFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($1), IS NOT NULL($12))])
-                    HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                HiveProject(wp_web_page_sk=[$0])
-                  HiveFilter(condition=[AND(BETWEEN(false, $10, 5000, 5200), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, web_page]], table:alias=[web_page])
-              HiveProject(t_time_sk=[$0])
-                HiveFilter(condition=[AND(BETWEEN(false, $3, 6, 7), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
-            HiveProject(hd_demo_sk=[$0])
-              HiveFilter(condition=[AND(=($3, 8), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
-      HiveProject($f0=[$0])
-        HiveAggregate(group=[{}], agg#0=[count()])
-          HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ws_sold_time_sk=[$1], ws_ship_hdemo_sk=[$10], ws_web_page_sk=[$12])
-                  HiveFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($1), IS NOT NULL($12))])
-                    HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                HiveProject(wp_web_page_sk=[$0])
-                  HiveFilter(condition=[AND(BETWEEN(false, $10, 5000, 5200), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, web_page]], table:alias=[web_page])
-              HiveProject(t_time_sk=[$0])
-                HiveFilter(condition=[AND(BETWEEN(false, $3, 14, 15), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
-            HiveProject(hd_demo_sk=[$0])
-              HiveFilter(condition=[AND(=($3, 8), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
+HiveProject(am_pm_ratio=[/(CAST($0):DECIMAL(15, 4), CAST($1):DECIMAL(15, 4))])
+  HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveProject($f0=[$0])
+      HiveAggregate(group=[{}], agg#0=[count()])
+        HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_sold_time_sk=[$1], ws_ship_hdemo_sk=[$10], ws_web_page_sk=[$12])
+                HiveFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($1), IS NOT NULL($12))])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+              HiveProject(wp_web_page_sk=[$0])
+                HiveFilter(condition=[AND(BETWEEN(false, $10, 5000, 5200), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, web_page]], table:alias=[web_page])
+            HiveProject(t_time_sk=[$0])
+              HiveFilter(condition=[AND(BETWEEN(false, $3, 6, 7), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
+          HiveProject(hd_demo_sk=[$0])
+            HiveFilter(condition=[AND(=($3, 8), IS NOT NULL($0))])
+              HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
+    HiveProject($f0=[$0])
+      HiveAggregate(group=[{}], agg#0=[count()])
+        HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_sold_time_sk=[$1], ws_ship_hdemo_sk=[$10], ws_web_page_sk=[$12])
+                HiveFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($1), IS NOT NULL($12))])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+              HiveProject(wp_web_page_sk=[$0])
+                HiveFilter(condition=[AND(BETWEEN(false, $10, 5000, 5200), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, web_page]], table:alias=[web_page])
+            HiveProject(t_time_sk=[$0])
+              HiveFilter(condition=[AND(BETWEEN(false, $3, 14, 15), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
+          HiveProject(hd_demo_sk=[$0])
+            HiveFilter(condition=[AND(=($3, 8), IS NOT NULL($0))])
+              HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query92.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query92.q.out
index 2a21fba..be7f364 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query92.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query92.q.out
@@ -65,30 +65,27 @@ POSTHOOK: Input: default@item
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(excess discount amount=[$0])
-  HiveSortLimit(sort0=[$1], dir0=[ASC], fetch=[100])
-    HiveProject(excess discount amount=[$0], (tok_function sum (tok_table_or_col ws_ext_discount_amt))=[$0])
-      HiveAggregate(group=[{}], agg#0=[sum($2)])
-        HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
-              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-            HiveProject(d_date_sk=[$0])
-              HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(ws_item_sk=[$0], CAST=[$1], i_item_sk=[$2])
-            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ws_item_sk=[$0], CAST=[CAST(*(1.3, /($1, $2))):DECIMAL(14, 7)])
-                HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-                  HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
-                      HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                        HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                    HiveProject(d_date_sk=[$0])
-                      HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00), IS NOT NULL($0))])
-                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveProject(i_item_sk=[$0])
-                HiveFilter(condition=[AND(=($13, 269), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, item]], table:alias=[item])
+HiveAggregate(group=[{}], agg#0=[sum($2)])
+  HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
+        HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+          HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+      HiveProject(d_date_sk=[$0])
+        HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00), IS NOT NULL($0))])
+          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+    HiveProject(ws_item_sk=[$0], CAST=[$1], i_item_sk=[$2])
+      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(ws_item_sk=[$0], CAST=[CAST(*(1.3, /($1, $2))):DECIMAL(14, 7)])
+          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
+                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(i_item_sk=[$0])
+          HiveFilter(condition=[AND(=($13, 269), IS NOT NULL($0))])
+            HiveTableScan(table=[[default, item]], table:alias=[item])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
index 4f97a67..e807882 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
@@ -67,34 +67,31 @@ POSTHOOK: Input: default@web_sales
 POSTHOOK: Input: default@web_site
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2])
-  HiveSortLimit(sort0=[$3], dir0=[ASC], fetch=[100])
-    HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2], (tok_functiondi count (tok_table_or_col ws_order_number))=[$0])
-      HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
-        HiveFilter(condition=[IS NULL($14)])
-          HiveJoin(condition=[=($4, $13)], joinType=[left], algorithm=[none], cost=[not available])
-            HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[inner])
-              HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$3], ws_web_site_sk=[$4], ws_warehouse_sk=[$5], ws_order_number=[$6], ws_ext_ship_cost=[$7], ws_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], web_site_sk=[$11], web_company_name=[$12])
-                HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'TX'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
-                      HiveFilter(condition=[AND(=($8, _UTF-16LE'TX'), IS NOT NULL($0))])
-                        HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                    HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_warehouse_sk=[$15], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
-                        HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($13), IS NOT NULL($17))])
-                          HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                      HiveProject(d_date_sk=[$0], d_date=[$2])
-                        HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00, 1999-06-30 00:00:00), IS NOT NULL($0))])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                  HiveProject(web_site_sk=[$0], web_company_name=[CAST(_UTF-16LE'pri'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
-                    HiveFilter(condition=[AND(=($14, _UTF-16LE'pri'), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
-              HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                HiveFilter(condition=[AND(IS NOT NULL($17), IS NOT NULL($15))])
-                  HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
-            HiveProject(wr_order_number0=[$0], $f1=[true])
-              HiveAggregate(group=[{13}])
-                HiveFilter(condition=[IS NOT NULL($13)])
-                  HiveTableScan(table=[[default, web_returns]], table:alias=[wr1])
+HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
+  HiveFilter(condition=[IS NULL($14)])
+    HiveJoin(condition=[=($4, $13)], joinType=[left], algorithm=[none], cost=[not available])
+      HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[inner])
+        HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$3], ws_web_site_sk=[$4], ws_warehouse_sk=[$5], ws_order_number=[$6], ws_ext_ship_cost=[$7], ws_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], web_site_sk=[$11], web_company_name=[$12])
+          HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'TX'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
+                HiveFilter(condition=[AND(=($8, _UTF-16LE'TX'), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+              HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_warehouse_sk=[$15], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
+                  HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($13), IS NOT NULL($17))])
+                    HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+                HiveProject(d_date_sk=[$0], d_date=[$2])
+                  HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00, 1999-06-30 00:00:00), IS NOT NULL($0))])
+                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(web_site_sk=[$0], web_company_name=[CAST(_UTF-16LE'pri'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
+              HiveFilter(condition=[AND(=($14, _UTF-16LE'pri'), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
+        HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+          HiveFilter(condition=[AND(IS NOT NULL($17), IS NOT NULL($15))])
+            HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
+      HiveProject(wr_order_number0=[$0], $f1=[true])
+        HiveAggregate(group=[{13}])
+          HiveFilter(condition=[IS NOT NULL($13)])
+            HiveTableScan(table=[[default, web_returns]], table:alias=[wr1])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query95.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query95.q.out
index 6a20165..68f3189 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query95.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query95.q.out
@@ -73,48 +73,45 @@ POSTHOOK: Input: default@web_sales
 POSTHOOK: Input: default@web_site
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2])
-  HiveSortLimit(sort0=[$3], dir0=[ASC], fetch=[100])
-    HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2], (tok_functiondi count (tok_table_or_col ws_order_number))=[$0])
-      HiveAggregate(group=[{}], agg#0=[count(DISTINCT $6)], agg#1=[sum($7)], agg#2=[sum($8)])
-        HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(wr_order_number=[$0])
-            HiveAggregate(group=[{14}])
-              HiveJoin(condition=[=($14, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ws_order_number=[$1])
-                  HiveJoin(condition=[AND(=($1, $3), <>($0, $2))], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                      HiveFilter(condition=[IS NOT NULL($17)])
-                        HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                    HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                      HiveFilter(condition=[IS NOT NULL($17)])
-                        HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
-                HiveProject(wr_returned_date_sk=[$0], wr_returned_time_sk=[$1], wr_item_sk=[$2], wr_refunded_customer_sk=[$3], wr_refunded_cdemo_sk=[$4], wr_refunded_hdemo_sk=[$5], wr_refunded_addr_sk=[$6], wr_returning_customer_sk=[$7], wr_returning_cdemo_sk=[$8], wr_returning_hdemo_sk=[$9], wr_returning_addr_sk=[$10], wr_web_page_sk=[$11], wr_reason_sk=[$12], wr_order_number=[$13], wr_return_quantity=[$14], wr_return_amt=[$15], wr_return_tax=[$16], wr_return_amt_inc_tax=[$17], wr_fee=[ [...]
-                  HiveFilter(condition=[IS NOT NULL($13)])
-                    HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
-          HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(ws_order_number=[$0])
-              HiveAggregate(group=[{1}])
-                HiveJoin(condition=[AND(=($1, $3), <>($0, $2))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                    HiveFilter(condition=[IS NOT NULL($17)])
-                      HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                  HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                    HiveFilter(condition=[IS NOT NULL($17)])
-                      HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
-            HiveJoin(condition=[=($3, $9)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ca_address_sk=[$0])
-                  HiveFilter(condition=[AND(=($8, _UTF-16LE'TX'), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
-                    HiveFilter(condition=[AND(IS NOT NULL($17), IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($13))])
-                      HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                  HiveProject(d_date_sk=[$0], d_date=[$2])
-                    HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00, 1999-06-30 00:00:00), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveProject(web_site_sk=[$0])
-                HiveFilter(condition=[AND(=($14, _UTF-16LE'pri'), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
+HiveAggregate(group=[{}], agg#0=[count(DISTINCT $6)], agg#1=[sum($7)], agg#2=[sum($8)])
+  HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveProject(wr_order_number=[$0])
+      HiveAggregate(group=[{14}])
+        HiveJoin(condition=[=($14, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveProject(ws_order_number=[$1])
+            HiveJoin(condition=[AND(=($1, $3), <>($0, $2))], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+                HiveFilter(condition=[IS NOT NULL($17)])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+              HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+                HiveFilter(condition=[IS NOT NULL($17)])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
+          HiveProject(wr_returned_date_sk=[$0], wr_returned_time_sk=[$1], wr_item_sk=[$2], wr_refunded_customer_sk=[$3], wr_refunded_cdemo_sk=[$4], wr_refunded_hdemo_sk=[$5], wr_refunded_addr_sk=[$6], wr_returning_customer_sk=[$7], wr_returning_cdemo_sk=[$8], wr_returning_hdemo_sk=[$9], wr_returning_addr_sk=[$10], wr_web_page_sk=[$11], wr_reason_sk=[$12], wr_order_number=[$13], wr_return_quantity=[$14], wr_return_amt=[$15], wr_return_tax=[$16], wr_return_amt_inc_tax=[$17], wr_fee=[$18],  [...]
+            HiveFilter(condition=[IS NOT NULL($13)])
+              HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
+    HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveProject(ws_order_number=[$0])
+        HiveAggregate(group=[{1}])
+          HiveJoin(condition=[AND(=($1, $3), <>($0, $2))], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+              HiveFilter(condition=[IS NOT NULL($17)])
+                HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+            HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+              HiveFilter(condition=[IS NOT NULL($17)])
+                HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
+      HiveJoin(condition=[=($3, $9)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveProject(ca_address_sk=[$0])
+            HiveFilter(condition=[AND(=($8, _UTF-16LE'TX'), IS NOT NULL($0))])
+              HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+          HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
+              HiveFilter(condition=[AND(IS NOT NULL($17), IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($13))])
+                HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+            HiveProject(d_date_sk=[$0], d_date=[$2])
+              HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00, 1999-06-30 00:00:00), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(web_site_sk=[$0])
+          HiveFilter(condition=[AND(=($14, _UTF-16LE'pri'), IS NOT NULL($0))])
+            HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query96.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query96.q.out
index ca9a51f..e142933 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query96.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query96.q.out
@@ -39,23 +39,20 @@ POSTHOOK: Input: default@store_sales
 POSTHOOK: Input: default@time_dim
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(_o__c0=[$0])
-  HiveSortLimit(sort0=[$1], dir0=[ASC], fetch=[100])
-    HiveProject(_o__c0=[$0], (tok_functionstar count)=[$0])
-      HiveAggregate(group=[{}], agg#0=[count()])
-        HiveJoin(condition=[=($2, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ss_sold_time_sk=[$1], ss_hdemo_sk=[$5], ss_store_sk=[$7])
-                HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($1), IS NOT NULL($7))])
-                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-              HiveProject(t_time_sk=[$0])
-                HiveFilter(condition=[AND(=($3, 8), >=($4, 30), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
-            HiveProject(hd_demo_sk=[$0])
-              HiveFilter(condition=[AND(=($3, 5), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
-          HiveProject(s_store_sk=[$0])
-            HiveFilter(condition=[AND(=($5, _UTF-16LE'ese'), IS NOT NULL($0))])
-              HiveTableScan(table=[[default, store]], table:alias=[store])
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveJoin(condition=[=($2, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(ss_sold_time_sk=[$1], ss_hdemo_sk=[$5], ss_store_sk=[$7])
+          HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($1), IS NOT NULL($7))])
+            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+        HiveProject(t_time_sk=[$0])
+          HiveFilter(condition=[AND(=($3, 8), >=($4, 30), IS NOT NULL($0))])
+            HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
+      HiveProject(hd_demo_sk=[$0])
+        HiveFilter(condition=[AND(=($3, 5), IS NOT NULL($0))])
+          HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
+    HiveProject(s_store_sk=[$0])
+      HiveFilter(condition=[AND(=($5, _UTF-16LE'ese'), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, store]], table:alias=[store])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query97.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query97.q.out
index f79dffb..a6f3939 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query97.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query97.q.out
@@ -55,27 +55,25 @@ POSTHOOK: Input: default@date_dim
 POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
-    HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[sum($1)], agg#2=[sum($2)])
-      HiveProject($f0=[CASE(AND(IS NOT NULL($0), IS NULL($2)), 1, 0)], $f1=[CASE(AND(IS NULL($0), IS NOT NULL($2)), 1, 0)], $f2=[CASE(AND(IS NOT NULL($0), IS NOT NULL($2)), 1, 0)])
-        HiveJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[full], algorithm=[none], cost=[not available])
-          HiveProject(ss_customer_sk=[$1], ss_item_sk=[$0])
-            HiveAggregate(group=[{1, 2}])
-              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3])
-                  HiveFilter(condition=[IS NOT NULL($0)])
-                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(cs_bill_customer_sk=[$0], cs_item_sk=[$1])
-            HiveAggregate(group=[{1, 2}])
-              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15])
-                  HiveFilter(condition=[IS NOT NULL($0)])
-                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[sum($1)], agg#2=[sum($2)])
+  HiveProject($f0=[CASE(AND(IS NOT NULL($0), IS NULL($2)), 1, 0)], $f1=[CASE(AND(IS NULL($0), IS NOT NULL($2)), 1, 0)], $f2=[CASE(AND(IS NOT NULL($0), IS NOT NULL($2)), 1, 0)])
+    HiveJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[full], algorithm=[none], cost=[not available])
+      HiveProject(ss_customer_sk=[$1], ss_item_sk=[$0])
+        HiveAggregate(group=[{1, 2}])
+          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3])
+              HiveFilter(condition=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(cs_bill_customer_sk=[$0], cs_item_sk=[$1])
+        HiveAggregate(group=[{1, 2}])
+          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15])
+              HiveFilter(condition=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[AND(BETWEEN(false, $3, 1212, 1223), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
index 251fd68..1de59b2 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
@@ -71,33 +71,30 @@ POSTHOOK: Input: default@customer_address
 POSTHOOK: Input: default@date_dim
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2])
-  HiveSortLimit(sort0=[$3], dir0=[ASC], fetch=[100])
-    HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2], (tok_functiondi count (tok_table_or_col cs_order_number))=[$0])
-      HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
-        HiveFilter(condition=[IS NULL($14)])
-          HiveJoin(condition=[=($4, $13)], joinType=[left], algorithm=[none], cost=[not available])
-            HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[inner])
-              HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
-                HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
-                      HiveFilter(condition=[=($8, _UTF-16LE'NY')])
-                        HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                    HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
-                        HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($10), IS NOT NULL($11))])
-                          HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
-                      HiveProject(d_date_sk=[$0], d_date=[$2])
-                        HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00, 2001-05-31 00:00:00)])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                  HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
-                    HiveFilter(condition=[IN($25, _UTF-16LE'Ziebach County', _UTF-16LE'Levy County', _UTF-16LE'Huron County', _UTF-16LE'Franklin Parish', _UTF-16LE'Daviess County')])
-                      HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
-              HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
-                HiveFilter(condition=[IS NOT NULL($14)])
-                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
-            HiveProject(cr_order_number0=[$0], $f1=[true])
-              HiveAggregate(group=[{16}])
-                HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
+HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
+  HiveFilter(condition=[IS NULL($14)])
+    HiveJoin(condition=[=($4, $13)], joinType=[left], algorithm=[none], cost=[not available])
+      HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[inner])
+        HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
+          HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
+                HiveFilter(condition=[=($8, _UTF-16LE'NY')])
+                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+              HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
+                  HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($10), IS NOT NULL($11))])
+                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
+                HiveProject(d_date_sk=[$0], d_date=[$2])
+                  HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00, 2001-05-31 00:00:00)])
+                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
+              HiveFilter(condition=[IN($25, _UTF-16LE'Ziebach County', _UTF-16LE'Levy County', _UTF-16LE'Huron County', _UTF-16LE'Franklin Parish', _UTF-16LE'Daviess County')])
+                HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
+        HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
+          HiveFilter(condition=[IS NOT NULL($14)])
+            HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
+      HiveProject(cr_order_number0=[$0], $f1=[true])
+        HiveAggregate(group=[{16}])
+          HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
 
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 7e4644a..7efcd5f 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,7 +1,7 @@
-Warning: Shuffle Join MERGEJOIN[443][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 19' is a cross product
-Warning: Shuffle Join MERGEJOIN[444][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 20' is a cross product
-Warning: Shuffle Join MERGEJOIN[446][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 23' is a cross product
-Warning: Shuffle Join MERGEJOIN[447][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 24' is a cross product
+Warning: Shuffle Join MERGEJOIN[442][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 19' is a cross product
+Warning: Shuffle Join MERGEJOIN[443][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 20' is a cross product
+Warning: Shuffle Join MERGEJOIN[445][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 23' is a cross product
+Warning: Shuffle Join MERGEJOIN[446][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] 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
@@ -119,127 +119,125 @@ POSTHOOK: Input: default@store_sales
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
+HiveAggregate(group=[{}], agg#0=[sum($0)])
   HiveProject($f0=[$0])
-    HiveAggregate(group=[{}], agg#0=[sum($0)])
-      HiveProject($f0=[$0])
-        HiveUnion(all=[true])
-          HiveProject($f0=[*(CAST($4):DECIMAL(10, 0), $5)])
-            HiveSemiJoin(condition=[=($3, $7)], joinType=[inner])
-              HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject($f0=[$0])
-                  HiveJoin(condition=[>($1, *(0.95, $3))], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                      HiveAggregate(group=[{0}], agg#0=[sum($1)])
-                        HiveProject(ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                          HiveFilter(condition=[IS NOT NULL($3)])
-                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                    HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveUnion(all=[true])
+      HiveProject($f0=[*(CAST($4):DECIMAL(10, 0), $5)])
+        HiveSemiJoin(condition=[=($3, $7)], joinType=[inner])
+          HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject($f0=[$0])
+              HiveJoin(condition=[>($1, *(0.95, $3))], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(ss_customer_sk=[$0], $f1=[$1])
+                  HiveAggregate(group=[{0}], agg#0=[sum($1)])
+                    HiveProject(ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                      HiveFilter(condition=[IS NOT NULL($3)])
+                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(cnt=[$0])
+                    HiveFilter(condition=[<=(sq_count_check($0), 1)])
                       HiveProject(cnt=[$0])
-                        HiveFilter(condition=[<=(sq_count_check($0), 1)])
-                          HiveProject(cnt=[$0])
-                            HiveAggregate(group=[{}], cnt=[COUNT()])
-                              HiveProject
-                                HiveProject($f0=[$0])
-                                  HiveAggregate(group=[{}], agg#0=[count($0)])
-                                    HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                                      HiveAggregate(group=[{1}], agg#0=[sum($2)])
-                                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                                            HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                          HiveProject(d_date_sk=[$0])
-                                            HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
-                                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject($f0=[$0])
-                        HiveAggregate(group=[{}], agg#0=[max($1)])
-                          HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                            HiveAggregate(group=[{1}], agg#0=[sum($2)])
-                              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                                  HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                HiveProject(d_date_sk=[$0])
-                                  HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
-                                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
-                    HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                  HiveProject(d_date_sk=[$0])
-                    HiveFilter(condition=[AND(=($6, 1999), =($8, 1))])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveProject(i_item_sk=[$1])
-                HiveFilter(condition=[>($3, 4)])
-                  HiveProject(substr=[$2], i_item_sk=[$1], d_date=[$0], $f3=[$3])
-                    HiveAggregate(group=[{3, 4, 5}], agg#0=[count()])
-                      HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0], d_date=[$2])
-                            HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                        HiveProject(i_item_sk=[$0], substr=[substr($4, 1, 30)])
-                          HiveTableScan(table=[[default, item]], table:alias=[item])
-          HiveProject($f0=[*(CAST($4):DECIMAL(10, 0), $5)])
-            HiveSemiJoin(condition=[=($2, $7)], joinType=[inner])
-              HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject($f0=[$0])
-                  HiveJoin(condition=[>($1, *(0.95, $3))], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                      HiveAggregate(group=[{0}], agg#0=[sum($1)])
-                        HiveProject(ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                          HiveFilter(condition=[IS NOT NULL($3)])
-                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                    HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveAggregate(group=[{}], cnt=[COUNT()])
+                          HiveProject
+                            HiveProject($f0=[$0])
+                              HiveAggregate(group=[{}], agg#0=[count($0)])
+                                HiveProject(ss_customer_sk=[$0], $f1=[$1])
+                                  HiveAggregate(group=[{1}], agg#0=[sum($2)])
+                                    HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                      HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                                        HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                      HiveProject(d_date_sk=[$0])
+                                        HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
+                                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                  HiveProject($f0=[$0])
+                    HiveAggregate(group=[{}], agg#0=[max($1)])
+                      HiveProject(ss_customer_sk=[$0], $f1=[$1])
+                        HiveAggregate(group=[{1}], agg#0=[sum($2)])
+                          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_quantity=[$18], cs_list_price=[$20])
+                HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[AND(=($6, 1999), =($8, 1))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(i_item_sk=[$1])
+            HiveFilter(condition=[>($3, 4)])
+              HiveProject(substr=[$2], i_item_sk=[$1], d_date=[$0], $f3=[$3])
+                HiveAggregate(group=[{3, 4, 5}], agg#0=[count()])
+                  HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                        HiveFilter(condition=[IS NOT NULL($0)])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(d_date_sk=[$0], d_date=[$2])
+                        HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
+                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                    HiveProject(i_item_sk=[$0], substr=[substr($4, 1, 30)])
+                      HiveTableScan(table=[[default, item]], table:alias=[item])
+      HiveProject($f0=[*(CAST($4):DECIMAL(10, 0), $5)])
+        HiveSemiJoin(condition=[=($2, $7)], joinType=[inner])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject($f0=[$0])
+              HiveJoin(condition=[>($1, *(0.95, $3))], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(ss_customer_sk=[$0], $f1=[$1])
+                  HiveAggregate(group=[{0}], agg#0=[sum($1)])
+                    HiveProject(ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                      HiveFilter(condition=[IS NOT NULL($3)])
+                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(cnt=[$0])
+                    HiveFilter(condition=[<=(sq_count_check($0), 1)])
                       HiveProject(cnt=[$0])
-                        HiveFilter(condition=[<=(sq_count_check($0), 1)])
-                          HiveProject(cnt=[$0])
-                            HiveAggregate(group=[{}], cnt=[COUNT()])
-                              HiveProject
-                                HiveProject($f0=[$0])
-                                  HiveAggregate(group=[{}], agg#0=[count($0)])
-                                    HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                                      HiveAggregate(group=[{1}], agg#0=[sum($2)])
-                                        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                          HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                                            HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                          HiveProject(d_date_sk=[$0])
-                                            HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
-                                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                      HiveProject($f0=[$0])
-                        HiveAggregate(group=[{}], agg#0=[max($1)])
-                          HiveProject(ss_customer_sk=[$0], $f1=[$1])
-                            HiveAggregate(group=[{1}], agg#0=[sum($2)])
-                              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                                HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
-                                  HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
-                                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                                HiveProject(d_date_sk=[$0])
-                                  HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
-                                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_quantity=[$18], ws_list_price=[$20])
-                    HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
-                      HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                  HiveProject(d_date_sk=[$0])
-                    HiveFilter(condition=[AND(=($6, 1999), =($8, 1))])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveProject(i_item_sk=[$1])
-                HiveFilter(condition=[>($3, 4)])
-                  HiveProject(substr=[$2], i_item_sk=[$1], d_date=[$0], $f3=[$3])
-                    HiveAggregate(group=[{3, 4, 5}], agg#0=[count()])
-                      HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
-                            HiveFilter(condition=[IS NOT NULL($0)])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0], d_date=[$2])
-                            HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                        HiveProject(i_item_sk=[$0], substr=[substr($4, 1, 30)])
-                          HiveTableScan(table=[[default, item]], table:alias=[item])
+                        HiveAggregate(group=[{}], cnt=[COUNT()])
+                          HiveProject
+                            HiveProject($f0=[$0])
+                              HiveAggregate(group=[{}], agg#0=[count($0)])
+                                HiveProject(ss_customer_sk=[$0], $f1=[$1])
+                                  HiveAggregate(group=[{1}], agg#0=[sum($2)])
+                                    HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                                      HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                                        HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                                      HiveProject(d_date_sk=[$0])
+                                        HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
+                                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                  HiveProject($f0=[$0])
+                    HiveAggregate(group=[{}], agg#0=[max($1)])
+                      HiveProject(ss_customer_sk=[$0], $f1=[$1])
+                        HiveAggregate(group=[{1}], agg#0=[sum($2)])
+                          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                            HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[CAST($3):INTEGER NOT NULL], *=[*(CAST($10):DECIMAL(10, 0), $13)])
+                              HiveFilter(condition=[AND(IS NOT NULL($3), IS NOT NULL($0))])
+                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                            HiveProject(d_date_sk=[$0])
+                              HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
+                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_quantity=[$18], ws_list_price=[$20])
+                HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[AND(=($6, 1999), =($8, 1))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(i_item_sk=[$1])
+            HiveFilter(condition=[>($3, 4)])
+              HiveProject(substr=[$2], i_item_sk=[$1], d_date=[$0], $f3=[$3])
+                HiveAggregate(group=[{3, 4, 5}], agg#0=[count()])
+                  HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+                        HiveFilter(condition=[IS NOT NULL($0)])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(d_date_sk=[$0], d_date=[$2])
+                        HiveFilter(condition=[IN($6, 1999, 2000, 2001, 2002)])
+                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                    HiveProject(i_item_sk=[$0], substr=[substr($4, 1, 30)])
+                      HiveTableScan(table=[[default, item]], table:alias=[item])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query28.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query28.q.out
index 643e5b6..e0e6153 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query28.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query28.q.out
@@ -1,8 +1,8 @@
-Warning: Shuffle Join MERGEJOIN[102][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product
-Warning: Shuffle Join MERGEJOIN[103][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
-Warning: Shuffle Join MERGEJOIN[104][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
-Warning: Shuffle Join MERGEJOIN[105][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 7' is a cross product
-Warning: Shuffle Join MERGEJOIN[106][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 8' is a cross product
+Warning: Shuffle Join MERGEJOIN[101][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[102][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[103][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[104][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[105][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 8' is a cross product
 PREHOOK: query: explain cbo
 select  *
 from (select avg(ss_list_price) B1_LP
@@ -112,35 +112,34 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f00=[$15], $f10=[$16], $f20=[$17], $f01=[$12], $f11=[$13], $f21=[$14], $f02=[$9], $f12=[$10], $f22=[$11], $f03=[$6], $f13=[$7], $f23=[$8], $f04=[$3], $f14=[$4], $f24=[$5])
+HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f00=[$15], $f10=[$16], $f20=[$17], $f01=[$12], $f11=[$13], $f21=[$14], $f02=[$9], $f12=[$10], $f22=[$11], $f03=[$6], $f13=[$7], $f23=[$8], $f04=[$3], $f14=[$4], $f24=[$5])
+  HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
     HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
       HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
         HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
           HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
-                HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-                  HiveFilter(condition=[AND(BETWEEN(false, $10, 0, 5), OR(BETWEEN(false, $12, 11, 21), BETWEEN(false, $19, 460, 1460), BETWEEN(false, $11, 14, 34)))])
-                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-              HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
-                HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-                  HiveFilter(condition=[AND(BETWEEN(false, $10, 26, 30), OR(BETWEEN(false, $12, 28, 38), BETWEEN(false, $19, 2513, 3513), BETWEEN(false, $11, 42, 62)))])
-                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
             HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
               HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-                HiveFilter(condition=[AND(BETWEEN(false, $10, 21, 25), OR(BETWEEN(false, $12, 135, 145), BETWEEN(false, $19, 14180, 15180), BETWEEN(false, $11, 38, 58)))])
+                HiveFilter(condition=[AND(BETWEEN(false, $10, 0, 5), OR(BETWEEN(false, $12, 11, 21), BETWEEN(false, $19, 460, 1460), BETWEEN(false, $11, 14, 34)))])
+                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+            HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
+              HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
+                HiveFilter(condition=[AND(BETWEEN(false, $10, 26, 30), OR(BETWEEN(false, $12, 28, 38), BETWEEN(false, $19, 2513, 3513), BETWEEN(false, $11, 42, 62)))])
                   HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
           HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
             HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-              HiveFilter(condition=[AND(BETWEEN(false, $10, 16, 20), OR(BETWEEN(false, $12, 142, 152), BETWEEN(false, $19, 3054, 4054), BETWEEN(false, $11, 80, 100)))])
+              HiveFilter(condition=[AND(BETWEEN(false, $10, 21, 25), OR(BETWEEN(false, $12, 135, 145), BETWEEN(false, $19, 14180, 15180), BETWEEN(false, $11, 38, 58)))])
                 HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
         HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
           HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-            HiveFilter(condition=[AND(BETWEEN(false, $10, 11, 15), OR(BETWEEN(false, $12, 66, 76), BETWEEN(false, $19, 920, 1920), BETWEEN(false, $11, 4, 24)))])
+            HiveFilter(condition=[AND(BETWEEN(false, $10, 16, 20), OR(BETWEEN(false, $12, 142, 152), BETWEEN(false, $19, 3054, 4054), BETWEEN(false, $11, 80, 100)))])
               HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
       HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
         HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
-          HiveFilter(condition=[AND(BETWEEN(false, $10, 6, 10), OR(BETWEEN(false, $12, 91, 101), BETWEEN(false, $19, 1430, 2430), BETWEEN(false, $11, 32, 52)))])
+          HiveFilter(condition=[AND(BETWEEN(false, $10, 11, 15), OR(BETWEEN(false, $12, 66, 76), BETWEEN(false, $19, 920, 1920), BETWEEN(false, $11, 4, 24)))])
             HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+    HiveProject($f0=[/($0, $1)], $f1=[$1], $f2=[$2])
+      HiveAggregate(group=[{}], agg#0=[sum($12)], agg#1=[count($12)], agg#2=[count(DISTINCT $12)])
+        HiveFilter(condition=[AND(BETWEEN(false, $10, 6, 10), OR(BETWEEN(false, $12, 91, 101), BETWEEN(false, $19, 1430, 2430), BETWEEN(false, $11, 32, 52)))])
+          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query32.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query32.q.out
index c7fb1dd..b5cf714 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query32.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query32.q.out
@@ -61,29 +61,27 @@ POSTHOOK: Input: default@date_dim
 POSTHOOK: Input: default@item
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0])
-    HiveAggregate(group=[{}], agg#0=[sum($2)])
-      HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
-            HiveFilter(condition=[IS NOT NULL($0)])
-              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-          HiveProject(d_date_sk=[$0])
-            HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00)])
-              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(cs_item_sk=[$0], CAST3=[$1], i_item_sk=[$2])
-          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(cs_item_sk=[$0], CAST3=[CAST(*(1.3, /($1, $2))):DECIMAL(14, 7)])
-              HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-                HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
-                    HiveFilter(condition=[IS NOT NULL($0)])
-                      HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                  HiveProject(d_date_sk=[$0])
-                    HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00)])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-            HiveProject(i_item_sk=[$0])
-              HiveFilter(condition=[=($13, 269)])
-                HiveTableScan(table=[[default, item]], table:alias=[item])
+HiveAggregate(group=[{}], agg#0=[sum($2)])
+  HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
+        HiveFilter(condition=[IS NOT NULL($0)])
+          HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+      HiveProject(d_date_sk=[$0])
+        HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00)])
+          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+    HiveProject(cs_item_sk=[$0], CAST3=[$1], i_item_sk=[$2])
+      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(cs_item_sk=[$0], CAST3=[CAST(*(1.3, /($1, $2))):DECIMAL(14, 7)])
+          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(cs_sold_date_sk=[$0], cs_item_sk=[$15], cs_ext_discount_amt=[$22])
+                HiveFilter(condition=[IS NOT NULL($0)])
+                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00)])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(i_item_sk=[$0])
+          HiveFilter(condition=[=($13, 269)])
+            HiveTableScan(table=[[default, item]], table:alias=[item])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query38.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query38.q.out
index 01e87d2..a08b35d 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query38.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query38.q.out
@@ -55,54 +55,52 @@ POSTHOOK: Input: default@store_sales
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0])
-    HiveAggregate(group=[{}], agg#0=[count()])
-      HiveProject(c_last_name=[$0], c_first_name=[$1], d_date=[$2], $f3=[$3])
-        HiveFilter(condition=[=($3, 3)])
-          HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
-            HiveProject(c_last_name=[$0], c_first_name=[$1], d_date=[$2], $f3=[$3])
-              HiveUnion(all=[true])
-                HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
-                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
-                    HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
-                      HiveAggregate(group=[{1, 2, 6}])
-                        HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
-                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
-                                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                            HiveProject(d_date_sk=[$0], d_date=[$2])
-                              HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
-                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
-                    HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
-                      HiveAggregate(group=[{1, 2, 6}])
-                        HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
-                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
-                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                            HiveProject(d_date_sk=[$0], d_date=[$2])
-                              HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
-                  HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
-                    HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
-                      HiveAggregate(group=[{1, 2, 6}])
-                        HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
-                            HiveTableScan(table=[[default, customer]], table:alias=[customer])
-                          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                            HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
-                                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                            HiveProject(d_date_sk=[$0], d_date=[$2])
-                              HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
-                                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveProject(c_last_name=[$0], c_first_name=[$1], d_date=[$2], $f3=[$3])
+    HiveFilter(condition=[=($3, 3)])
+      HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)])
+        HiveProject(c_last_name=[$0], c_first_name=[$1], d_date=[$2], $f3=[$3])
+          HiveUnion(all=[true])
+            HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
+              HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
+                HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
+                  HiveAggregate(group=[{1, 2, 6}])
+                    HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
+                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                        HiveProject(d_date_sk=[$0], d_date=[$2])
+                          HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
+              HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
+                HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
+                  HiveAggregate(group=[{1, 2, 6}])
+                    HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
+                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+                            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                        HiveProject(d_date_sk=[$0], d_date=[$2])
+                          HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(c_last_name=[$1], c_first_name=[$0], d_date=[$2], $f3=[$3])
+              HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
+                HiveProject(c_first_name=[$0], c_last_name=[$1], d_date=[$2])
+                  HiveAggregate(group=[{1, 2, 6}])
+                    HiveJoin(condition=[=($4, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(c_customer_sk=[$0], c_first_name=[$8], c_last_name=[$9])
+                        HiveTableScan(table=[[default, customer]], table:alias=[customer])
+                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($4))])
+                            HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                        HiveProject(d_date_sk=[$0], d_date=[$2])
+                          HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query61.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query61.q.out
index 253190d..2bc02f5 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query61.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query61.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[266][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[263][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain cbo
 select  promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
 from
@@ -102,55 +102,25 @@ POSTHOOK: Input: default@store
 POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
-  HiveProject(promotions=[$0], total=[$1], _o__c2=[*(/(CAST($0):DECIMAL(15, 4), CAST($1):DECIMAL(15, 4)), CAST(100):DECIMAL(10, 0))])
-    HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveProject($f0=[$0])
-        HiveAggregate(group=[{}], agg#0=[sum($8)])
-          HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4])
-                HiveFilter(condition=[IS NOT NULL($4)])
-                  HiveTableScan(table=[[default, customer]], table:alias=[customer])
-              HiveProject(ca_address_sk=[$0])
-                HiveFilter(condition=[=($11, -7)])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_store_sk=[$3], ss_promo_sk=[$4], ss_ext_sales_price=[$5], d_date_sk=[$6], i_item_sk=[$7], s_store_sk=[$8], p_promo_sk=[$9])
-              HiveJoin(condition=[=($4, $9)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[=($3, $8)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($1, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_promo_sk=[$8], ss_ext_sales_price=[$15])
-                        HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($8), IS NOT NULL($0), IS NOT NULL($3))])
-                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                      HiveProject(d_date_sk=[$0])
-                        HiveFilter(condition=[AND(=($6, 1999), =($8, 11))])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                    HiveProject(i_item_sk=[$0])
-                      HiveFilter(condition=[=($12, _UTF-16LE'Electronics')])
-                        HiveTableScan(table=[[default, item]], table:alias=[item])
-                  HiveProject(s_store_sk=[$0])
-                    HiveFilter(condition=[=($27, -7)])
-                      HiveTableScan(table=[[default, store]], table:alias=[store])
-                HiveProject(p_promo_sk=[$0])
-                  HiveFilter(condition=[OR(=($8, _UTF-16LE'Y'), =($9, _UTF-16LE'Y'), =($11, _UTF-16LE'Y'))])
-                    HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
-      HiveProject($f0=[$0])
-        HiveAggregate(group=[{}], agg#0=[sum($7)])
-          HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4])
-                HiveFilter(condition=[IS NOT NULL($4)])
-                  HiveTableScan(table=[[default, customer]], table:alias=[customer])
-              HiveProject(ca_address_sk=[$0])
-                HiveFilter(condition=[=($11, -7)])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_store_sk=[$3], ss_ext_sales_price=[$4], d_date_sk=[$5], i_item_sk=[$6], s_store_sk=[$7])
-              HiveJoin(condition=[=($3, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveJoin(condition=[=($1, $6)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ext_sales_price=[$15])
-                      HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0), IS NOT NULL($3))])
+HiveProject(promotions=[$0], total=[$1], _o__c2=[*(/(CAST($0):DECIMAL(15, 4), CAST($1):DECIMAL(15, 4)), CAST(100):DECIMAL(10, 0))])
+  HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveProject($f0=[$0])
+      HiveAggregate(group=[{}], agg#0=[sum($8)])
+        HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4])
+              HiveFilter(condition=[IS NOT NULL($4)])
+                HiveTableScan(table=[[default, customer]], table:alias=[customer])
+            HiveProject(ca_address_sk=[$0])
+              HiveFilter(condition=[=($11, -7)])
+                HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_store_sk=[$3], ss_promo_sk=[$4], ss_ext_sales_price=[$5], d_date_sk=[$6], i_item_sk=[$7], s_store_sk=[$8], p_promo_sk=[$9])
+            HiveJoin(condition=[=($4, $9)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveJoin(condition=[=($3, $8)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveJoin(condition=[=($1, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_promo_sk=[$8], ss_ext_sales_price=[$15])
+                      HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($8), IS NOT NULL($0), IS NOT NULL($3))])
                         HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
                     HiveProject(d_date_sk=[$0])
                       HiveFilter(condition=[AND(=($6, 1999), =($8, 11))])
@@ -161,4 +131,33 @@ HiveSortLimit(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[100])
                 HiveProject(s_store_sk=[$0])
                   HiveFilter(condition=[=($27, -7)])
                     HiveTableScan(table=[[default, store]], table:alias=[store])
+              HiveProject(p_promo_sk=[$0])
+                HiveFilter(condition=[OR(=($8, _UTF-16LE'Y'), =($9, _UTF-16LE'Y'), =($11, _UTF-16LE'Y'))])
+                  HiveTableScan(table=[[default, promotion]], table:alias=[promotion])
+    HiveProject($f0=[$0])
+      HiveAggregate(group=[{}], agg#0=[sum($7)])
+        HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($2, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(c_customer_sk=[$0], c_current_addr_sk=[$4])
+              HiveFilter(condition=[IS NOT NULL($4)])
+                HiveTableScan(table=[[default, customer]], table:alias=[customer])
+            HiveProject(ca_address_sk=[$0])
+              HiveFilter(condition=[=($11, -7)])
+                HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_store_sk=[$3], ss_ext_sales_price=[$4], d_date_sk=[$5], i_item_sk=[$6], s_store_sk=[$7])
+            HiveJoin(condition=[=($3, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveJoin(condition=[=($1, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_store_sk=[$7], ss_ext_sales_price=[$15])
+                    HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0), IS NOT NULL($3))])
+                      HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                  HiveProject(d_date_sk=[$0])
+                    HiveFilter(condition=[AND(=($6, 1999), =($8, 11))])
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                HiveProject(i_item_sk=[$0])
+                  HiveFilter(condition=[=($12, _UTF-16LE'Electronics')])
+                    HiveTableScan(table=[[default, item]], table:alias=[item])
+              HiveProject(s_store_sk=[$0])
+                HiveFilter(condition=[=($27, -7)])
+                  HiveTableScan(table=[[default, store]], table:alias=[store])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query90.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query90.q.out
index b5aac32..96b7e88 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query90.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query90.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[152][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[149][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
 PREHOOK: query: explain cbo
 select  cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
  from ( select count(*) amc
@@ -52,41 +52,40 @@ POSTHOOK: Input: default@web_page
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(sort0=[$0], dir0=[ASC], fetch=[100])
-  HiveProject(am_pm_ratio=[/(CAST($0):DECIMAL(15, 4), CAST($1):DECIMAL(15, 4))])
-    HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
-      HiveProject($f0=[$0])
-        HiveAggregate(group=[{}], agg#0=[count()])
-          HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ws_sold_time_sk=[$1], ws_ship_hdemo_sk=[$10], ws_web_page_sk=[$12])
-                  HiveFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($1), IS NOT NULL($12))])
-                    HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                HiveProject(wp_web_page_sk=[$0])
-                  HiveFilter(condition=[BETWEEN(false, $10, 5000, 5200)])
-                    HiveTableScan(table=[[default, web_page]], table:alias=[web_page])
-              HiveProject(t_time_sk=[$0])
-                HiveFilter(condition=[BETWEEN(false, $3, 6, 7)])
-                  HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
-            HiveProject(hd_demo_sk=[$0])
-              HiveFilter(condition=[=($3, 8)])
-                HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
-      HiveProject($f0=[$0])
-        HiveAggregate(group=[{}], agg#0=[count()])
-          HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ws_sold_time_sk=[$1], ws_ship_hdemo_sk=[$10], ws_web_page_sk=[$12])
-                  HiveFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($1), IS NOT NULL($12))])
-                    HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                HiveProject(wp_web_page_sk=[$0])
-                  HiveFilter(condition=[BETWEEN(false, $10, 5000, 5200)])
-                    HiveTableScan(table=[[default, web_page]], table:alias=[web_page])
-              HiveProject(t_time_sk=[$0])
-                HiveFilter(condition=[BETWEEN(false, $3, 14, 15)])
-                  HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
-            HiveProject(hd_demo_sk=[$0])
-              HiveFilter(condition=[=($3, 8)])
-                HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
+HiveProject(am_pm_ratio=[/(CAST($0):DECIMAL(15, 4), CAST($1):DECIMAL(15, 4))])
+  HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveProject($f0=[$0])
+      HiveAggregate(group=[{}], agg#0=[count()])
+        HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_sold_time_sk=[$1], ws_ship_hdemo_sk=[$10], ws_web_page_sk=[$12])
+                HiveFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($1), IS NOT NULL($12))])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+              HiveProject(wp_web_page_sk=[$0])
+                HiveFilter(condition=[BETWEEN(false, $10, 5000, 5200)])
+                  HiveTableScan(table=[[default, web_page]], table:alias=[web_page])
+            HiveProject(t_time_sk=[$0])
+              HiveFilter(condition=[BETWEEN(false, $3, 6, 7)])
+                HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
+          HiveProject(hd_demo_sk=[$0])
+            HiveFilter(condition=[=($3, 8)])
+              HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
+    HiveProject($f0=[$0])
+      HiveAggregate(group=[{}], agg#0=[count()])
+        HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_sold_time_sk=[$1], ws_ship_hdemo_sk=[$10], ws_web_page_sk=[$12])
+                HiveFilter(condition=[AND(IS NOT NULL($10), IS NOT NULL($1), IS NOT NULL($12))])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+              HiveProject(wp_web_page_sk=[$0])
+                HiveFilter(condition=[BETWEEN(false, $10, 5000, 5200)])
+                  HiveTableScan(table=[[default, web_page]], table:alias=[web_page])
+            HiveProject(t_time_sk=[$0])
+              HiveFilter(condition=[BETWEEN(false, $3, 14, 15)])
+                HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
+          HiveProject(hd_demo_sk=[$0])
+            HiveFilter(condition=[=($3, 8)])
+              HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query92.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query92.q.out
index 16098d7..503c781 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query92.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query92.q.out
@@ -65,30 +65,27 @@ POSTHOOK: Input: default@item
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(excess discount amount=[$0])
-  HiveSortLimit(sort0=[$1], dir0=[ASC], fetch=[100])
-    HiveProject(excess discount amount=[$0], (tok_function sum (tok_table_or_col ws_ext_discount_amt))=[$0])
-      HiveAggregate(group=[{}], agg#0=[sum($2)])
-        HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
-              HiveFilter(condition=[IS NOT NULL($0)])
-                HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-            HiveProject(d_date_sk=[$0])
-              HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00)])
-                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(ws_item_sk=[$0], CAST3=[$1], i_item_sk=[$2])
-            HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ws_item_sk=[$0], CAST3=[CAST(*(1.3, /($1, $2))):DECIMAL(14, 7)])
-                HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
-                  HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
-                      HiveFilter(condition=[IS NOT NULL($0)])
-                        HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                    HiveProject(d_date_sk=[$0])
-                      HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00)])
-                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveProject(i_item_sk=[$0])
-                HiveFilter(condition=[=($13, 269)])
-                  HiveTableScan(table=[[default, item]], table:alias=[item])
+HiveAggregate(group=[{}], agg#0=[sum($2)])
+  HiveJoin(condition=[AND(>($2, $5), =($6, $1))], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
+        HiveFilter(condition=[IS NOT NULL($0)])
+          HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+      HiveProject(d_date_sk=[$0])
+        HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00)])
+          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+    HiveProject(ws_item_sk=[$0], CAST3=[$1], i_item_sk=[$2])
+      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(ws_item_sk=[$0], CAST3=[CAST(*(1.3, /($1, $2))):DECIMAL(14, 7)])
+          HiveAggregate(group=[{1}], agg#0=[sum($2)], agg#1=[count($2)])
+            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_ext_discount_amt=[$22])
+                HiveFilter(condition=[IS NOT NULL($0)])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+              HiveProject(d_date_sk=[$0])
+                HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1998-03-18 00:00:00, 1998-06-16 00:00:00)])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(i_item_sk=[$0])
+          HiveFilter(condition=[=($13, 269)])
+            HiveTableScan(table=[[default, item]], table:alias=[item])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
index 19c5312..25e0c8b 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
@@ -67,33 +67,30 @@ POSTHOOK: Input: default@web_sales
 POSTHOOK: Input: default@web_site
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2])
-  HiveSortLimit(sort0=[$3], dir0=[ASC], fetch=[100])
-    HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2], (tok_functiondi count (tok_table_or_col ws_order_number))=[$0])
-      HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
-        HiveFilter(condition=[IS NULL($14)])
-          HiveJoin(condition=[=($4, $13)], joinType=[left], algorithm=[none], cost=[not available])
-            HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[inner])
-              HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$3], ws_web_site_sk=[$4], ws_warehouse_sk=[$5], ws_order_number=[$6], ws_ext_ship_cost=[$7], ws_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], web_site_sk=[$11], web_company_name=[$12])
-                HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'TX'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
-                      HiveFilter(condition=[=($8, _UTF-16LE'TX')])
-                        HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                    HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_warehouse_sk=[$15], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
-                        HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($13))])
-                          HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                      HiveProject(d_date_sk=[$0], d_date=[$2])
-                        HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00, 1999-06-30 00:00:00)])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                  HiveProject(web_site_sk=[$0], web_company_name=[CAST(_UTF-16LE'pri'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
-                    HiveFilter(condition=[=($14, _UTF-16LE'pri')])
-                      HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
-              HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                HiveFilter(condition=[IS NOT NULL($15)])
-                  HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
-            HiveProject(wr_order_number0=[$0], $f1=[true])
-              HiveAggregate(group=[{13}])
-                HiveTableScan(table=[[default, web_returns]], table:alias=[wr1])
+HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
+  HiveFilter(condition=[IS NULL($14)])
+    HiveJoin(condition=[=($4, $13)], joinType=[left], algorithm=[none], cost=[not available])
+      HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[inner])
+        HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$3], ws_web_site_sk=[$4], ws_warehouse_sk=[$5], ws_order_number=[$6], ws_ext_ship_cost=[$7], ws_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], web_site_sk=[$11], web_company_name=[$12])
+          HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'TX'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
+                HiveFilter(condition=[=($8, _UTF-16LE'TX')])
+                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+              HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+                HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_warehouse_sk=[$15], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
+                  HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($13))])
+                    HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+                HiveProject(d_date_sk=[$0], d_date=[$2])
+                  HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00, 1999-06-30 00:00:00)])
+                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(web_site_sk=[$0], web_company_name=[CAST(_UTF-16LE'pri'):VARCHAR(2147483647) CHARACTER SET "UTF-16LE" COLLATE "ISO-8859-1$en_US$primary"])
+              HiveFilter(condition=[=($14, _UTF-16LE'pri')])
+                HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
+        HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+          HiveFilter(condition=[IS NOT NULL($15)])
+            HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
+      HiveProject(wr_order_number0=[$0], $f1=[true])
+        HiveAggregate(group=[{13}])
+          HiveTableScan(table=[[default, web_returns]], table:alias=[wr1])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query95.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query95.q.out
index 526dafb..606e162 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query95.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query95.q.out
@@ -73,42 +73,39 @@ POSTHOOK: Input: default@web_sales
 POSTHOOK: Input: default@web_site
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2])
-  HiveSortLimit(sort0=[$3], dir0=[ASC], fetch=[100])
-    HiveProject(order count=[$0], total shipping cost=[$1], total net profit=[$2], (tok_functiondi count (tok_table_or_col ws_order_number))=[$0])
-      HiveAggregate(group=[{}], agg#0=[count(DISTINCT $6)], agg#1=[sum($7)], agg#2=[sum($8)])
-        HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveProject(wr_order_number=[$0])
-            HiveAggregate(group=[{14}])
-              HiveJoin(condition=[=($14, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ws_order_number=[$1])
-                  HiveJoin(condition=[AND(=($1, $3), <>($0, $2))], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                      HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                    HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                      HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
-                HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
-          HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveProject(ws_order_number=[$0])
-              HiveAggregate(group=[{1}])
-                HiveJoin(condition=[AND(=($1, $3), <>($0, $2))], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                    HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                  HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-                    HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
-            HiveJoin(condition=[=($3, $9)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ca_address_sk=[$0])
-                  HiveFilter(condition=[=($8, _UTF-16LE'TX')])
-                    HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-                HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
-                  HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
-                    HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($13))])
-                      HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                  HiveProject(d_date_sk=[$0], d_date=[$2])
-                    HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00, 1999-06-30 00:00:00)])
-                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveProject(web_site_sk=[$0])
-                HiveFilter(condition=[=($14, _UTF-16LE'pri')])
-                  HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
+HiveAggregate(group=[{}], agg#0=[count(DISTINCT $6)], agg#1=[sum($7)], agg#2=[sum($8)])
+  HiveJoin(condition=[=($6, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveProject(wr_order_number=[$0])
+      HiveAggregate(group=[{14}])
+        HiveJoin(condition=[=($14, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveProject(ws_order_number=[$1])
+            HiveJoin(condition=[AND(=($1, $3), <>($0, $2))], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+                HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+              HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+                HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
+          HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
+    HiveJoin(condition=[=($5, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveProject(ws_order_number=[$0])
+        HiveAggregate(group=[{1}])
+          HiveJoin(condition=[AND(=($1, $3), <>($0, $2))], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+              HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+            HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+              HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
+      HiveJoin(condition=[=($3, $9)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveProject(ca_address_sk=[$0])
+            HiveFilter(condition=[=($8, _UTF-16LE'TX')])
+              HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+          HiveJoin(condition=[=($0, $6)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
+              HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($13))])
+                HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+            HiveProject(d_date_sk=[$0], d_date=[$2])
+              HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00, 1999-06-30 00:00:00)])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(web_site_sk=[$0])
+          HiveFilter(condition=[=($14, _UTF-16LE'pri')])
+            HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query96.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query96.q.out
index e32705a..b0611d3 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query96.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query96.q.out
@@ -39,23 +39,20 @@ POSTHOOK: Input: default@store_sales
 POSTHOOK: Input: default@time_dim
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveProject(_o__c0=[$0])
-  HiveSortLimit(sort0=[$1], dir0=[ASC], fetch=[100])
-    HiveProject(_o__c0=[$0], (tok_functionstar count)=[$0])
-      HiveAggregate(group=[{}], agg#0=[count()])
-        HiveJoin(condition=[=($2, $5)], joinType=[inner], algorithm=[none], cost=[not available])
-          HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ss_sold_time_sk=[$1], ss_hdemo_sk=[$5], ss_store_sk=[$7])
-                HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($1), IS NOT NULL($7))])
-                  HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-              HiveProject(t_time_sk=[$0])
-                HiveFilter(condition=[AND(=($3, 8), >=($4, 30))])
-                  HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
-            HiveProject(hd_demo_sk=[$0])
-              HiveFilter(condition=[=($3, 5)])
-                HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
-          HiveProject(s_store_sk=[$0])
-            HiveFilter(condition=[=($5, _UTF-16LE'ese')])
-              HiveTableScan(table=[[default, store]], table:alias=[store])
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveJoin(condition=[=($2, $5)], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveJoin(condition=[=($1, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(ss_sold_time_sk=[$1], ss_hdemo_sk=[$5], ss_store_sk=[$7])
+          HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($1), IS NOT NULL($7))])
+            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+        HiveProject(t_time_sk=[$0])
+          HiveFilter(condition=[AND(=($3, 8), >=($4, 30))])
+            HiveTableScan(table=[[default, time_dim]], table:alias=[time_dim])
+      HiveProject(hd_demo_sk=[$0])
+        HiveFilter(condition=[=($3, 5)])
+          HiveTableScan(table=[[default, household_demographics]], table:alias=[household_demographics])
+    HiveProject(s_store_sk=[$0])
+      HiveFilter(condition=[=($5, _UTF-16LE'ese')])
+        HiveTableScan(table=[[default, store]], table:alias=[store])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query97.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query97.q.out
index 6151627..c8fff90 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query97.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query97.q.out
@@ -55,27 +55,25 @@ POSTHOOK: Input: default@date_dim
 POSTHOOK: Input: default@store_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
-HiveSortLimit(fetch=[100])
-  HiveProject($f0=[$0], $f1=[$1], $f2=[$2])
-    HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[sum($1)], agg#2=[sum($2)])
-      HiveProject($f0=[CASE(AND(IS NOT NULL($0), IS NULL($2)), 1, 0)], $f1=[CASE(AND(IS NULL($0), IS NOT NULL($2)), 1, 0)], $f2=[CASE(AND(IS NOT NULL($0), IS NOT NULL($2)), 1, 0)])
-        HiveJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[full], algorithm=[none], cost=[not available])
-          HiveProject(ss_customer_sk=[$1], ss_item_sk=[$0])
-            HiveAggregate(group=[{1, 2}])
-              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3])
-                  HiveFilter(condition=[IS NOT NULL($0)])
-                    HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(cs_bill_customer_sk=[$0], cs_item_sk=[$1])
-            HiveAggregate(group=[{1, 2}])
-              HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15])
-                  HiveFilter(condition=[IS NOT NULL($0)])
-                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[sum($1)], agg#2=[sum($2)])
+  HiveProject($f0=[CASE(AND(IS NOT NULL($0), IS NULL($2)), 1, 0)], $f1=[CASE(AND(IS NULL($0), IS NOT NULL($2)), 1, 0)], $f2=[CASE(AND(IS NOT NULL($0), IS NOT NULL($2)), 1, 0)])
+    HiveJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[full], algorithm=[none], cost=[not available])
+      HiveProject(ss_customer_sk=[$1], ss_item_sk=[$0])
+        HiveAggregate(group=[{1, 2}])
+          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3])
+              HiveFilter(condition=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveProject(cs_bill_customer_sk=[$0], cs_item_sk=[$1])
+        HiveAggregate(group=[{1, 2}])
+          HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15])
+              HiveFilter(condition=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[BETWEEN(false, $3, 1212, 1223)])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
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 3c63887..2aacd24 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,156 +73,147 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 13 (BROADCAST_EDGE)
-Map 15 <- Reducer 10 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 13 <- Map 12 (CUSTOM_SIMPLE_EDGE)
-Reducer 17 <- Map 16 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
-Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 14 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 15 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-Reducer 6 <- Reducer 17 (ONE_TO_ONE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE)
+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 (ONE_TO_ONE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
-Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 9 vectorized
-      File Output Operator [FS_166]
-        Limit [LIM_165] (rows=1 width=240)
-          Number of rows:100
-          Select Operator [SEL_164] (rows=1 width=240)
-            Output:["_col0","_col1","_col2"]
-          <-Reducer 8 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_163]
-              Select Operator [SEL_162] (rows=1 width=240)
-                Output:["_col1","_col2","_col3"]
-                Group By Operator [GBY_161] (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_160]
-                    Group By Operator [GBY_159] (rows=1 width=232)
-                      Output:["_col0","_col1","_col2"],aggregations:["count(_col0)","sum(_col1)","sum(_col2)"]
-                      Group By Operator [GBY_158] (rows=5150256 width=228)
-                        Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
-                      <-Reducer 6 [SIMPLE_EDGE]
-                        SHUFFLE [RS_73]
-                          PartitionCols:_col0
-                          Group By Operator [GBY_72] (rows=5150256 width=228)
-                            Output:["_col0","_col2","_col3"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col4
-                            Select Operator [SEL_41] (rows=5150256 width=214)
+      Reducer 8 vectorized
+      File Output Operator [FS_158]
+        Group By Operator [GBY_157] (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)
+              Output:["_col0","_col1","_col2"],aggregations:["count(_col0)","sum(_col1)","sum(_col2)"]
+              Group By Operator [GBY_154] (rows=5150256 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)
+                    Output:["_col0","_col2","_col3"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col4
+                    Select Operator [SEL_41] (rows=5150256 width=214)
+                      Output:["_col4","_col5","_col6"]
+                      Filter Operator [FIL_40] (rows=5150256 width=214)
+                        predicate:_col14 is null
+                        Merge Join Operator [MERGEJOIN_125] (rows=10300512 width=214)
+                          Conds:RS_37._col4=RS_153._col0(Left Outer),Output:["_col4","_col5","_col6","_col14"]
+                        <-Reducer 16 [ONE_TO_ONE_EDGE] vectorized
+                          FORWARD [RS_153]
+                            PartitionCols:_col0
+                            Select Operator [SEL_152] (rows=18238808 width=8)
+                              Output:["_col0","_col1"]
+                              Group By Operator [GBY_151] (rows=18238808 width=4)
+                                Output:["_col0"],keys:KEY._col0
+                              <-Map 15 [SIMPLE_EDGE] vectorized
+                                SHUFFLE [RS_150]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_149] (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 [ONE_TO_ONE_EDGE]
+                          FORWARD [RS_37]
+                            PartitionCols:_col4
+                            Select Operator [SEL_36] (rows=5150256 width=200)
                               Output:["_col4","_col5","_col6"]
-                              Filter Operator [FIL_40] (rows=5150256 width=214)
-                                predicate:_col14 is null
-                                Merge Join Operator [MERGEJOIN_129] (rows=10300512 width=214)
-                                  Conds:RS_37._col4=RS_157._col0(Left Outer),Output:["_col4","_col5","_col6","_col14"]
-                                <-Reducer 17 [ONE_TO_ONE_EDGE] vectorized
-                                  FORWARD [RS_157]
-                                    PartitionCols:_col0
-                                    Select Operator [SEL_156] (rows=18238808 width=8)
-                                      Output:["_col0","_col1"]
-                                      Group By Operator [GBY_155] (rows=18238808 width=4)
-                                        Output:["_col0"],keys:KEY._col0
-                                      <-Map 16 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_154]
+                              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)}
+                              <-Reducer 4 [SIMPLE_EDGE]
+                                PARTITION_ONLY_SHUFFLE [RS_33]
+                                  PartitionCols:_col4
+                                  Merge Join Operator [MERGEJOIN_123] (rows=5150256 width=200)
+                                    Conds:RS_18._col2=RS_142._col0(Inner),Output:["_col3","_col4","_col5","_col6"]
+                                  <-Map 13 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_142]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_141] (rows=10 width=102)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_140] (rows=10 width=102)
+                                          predicate:(cc_county) IN ('Ziebach County', 'Levy County', 'Huron County', 'Franklin Parish', 'Daviess County')
+                                          TableScan [TS_9] (rows=60 width=102)
+                                            default@call_center,call_center,Tbl:COMPLETE,Col:COMPLETE,Output:["cc_call_center_sk","cc_county"]
+                                  <-Reducer 3 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_18]
+                                      PartitionCols:_col2
+                                      Merge Join Operator [MERGEJOIN_122] (rows=30901534 width=230)
+                                        Conds:RS_15._col1=RS_128._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
+                                      <-Map 11 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_128]
                                           PartitionCols:_col0
-                                          Group By Operator [GBY_153] (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 [ONE_TO_ONE_EDGE]
-                                  FORWARD [RS_37]
-                                    PartitionCols:_col4
-                                    Select Operator [SEL_36] (rows=5150256 width=200)
-                                      Output:["_col4","_col5","_col6"]
-                                      Merge Join Operator [MERGEJOIN_128] (rows=5150256 width=202)
-                                        Conds:RS_33._col4=RS_152._col0(Left Semi),Output:["_col3","_col4","_col5","_col6","_col14"],residual filter predicates:{(_col3 <> _col14)}
-                                      <-Reducer 4 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_33]
-                                          PartitionCols:_col4
-                                          Merge Join Operator [MERGEJOIN_127] (rows=5150256 width=200)
-                                            Conds:RS_18._col2=RS_146._col0(Inner),Output:["_col3","_col4","_col5","_col6"]
-                                          <-Map 14 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_146]
+                                          Select Operator [SEL_127] (rows=784314 width=90)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_126] (rows=784314 width=90)
+                                              predicate:(ca_state = 'NY')
+                                              TableScan [TS_6] (rows=40000000 width=90)
+                                                default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_state"]
+                                      <-Reducer 2 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_15]
+                                          PartitionCols:_col1
+                                          Merge Join Operator [MERGEJOIN_121] (rows=31519516 width=234)
+                                            Conds:RS_136._col0=RS_139._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
+                                          <-Map 1 [SIMPLE_EDGE] vectorized
+                                            SHUFFLE [RS_136]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_135] (rows=283695062 width=243)
+                                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                Filter Operator [FIL_134] (rows=283695062 width=243)
+                                                  predicate:((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))) and cs_call_center_sk is not null and cs_ship_addr_sk is not null and cs_ship_date_sk is not null)
+                                                  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
+                                                    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
+                                                        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
+                                            SHUFFLE [RS_139]
                                               PartitionCols:_col0
-                                              Select Operator [SEL_145] (rows=10 width=102)
+                                              Select Operator [SEL_138] (rows=8116 width=98)
                                                 Output:["_col0"]
-                                                Filter Operator [FIL_144] (rows=10 width=102)
-                                                  predicate:(cc_county) IN ('Ziebach County', 'Levy County', 'Huron County', 'Franklin Parish', 'Daviess County')
-                                                  TableScan [TS_9] (rows=60 width=102)
-                                                    default@call_center,call_center,Tbl:COMPLETE,Col:COMPLETE,Output:["cc_call_center_sk","cc_county"]
-                                          <-Reducer 3 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_18]
-                                              PartitionCols:_col2
-                                              Merge Join Operator [MERGEJOIN_126] (rows=30901534 width=230)
-                                                Conds:RS_15._col1=RS_132._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
-                                              <-Map 12 [SIMPLE_EDGE] vectorized
-                                                SHUFFLE [RS_132]
-                                                  PartitionCols:_col0
-                                                  Select Operator [SEL_131] (rows=784314 width=90)
+                                                Filter Operator [FIL_137] (rows=8116 width=98)
+                                                  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_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))) and 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 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"]
-                                                    Filter Operator [FIL_130] (rows=784314 width=90)
-                                                      predicate:(ca_state = 'NY')
-                                                      TableScan [TS_6] (rows=40000000 width=90)
-                                                        default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_state"]
-                                              <-Reducer 2 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_15]
-                                                  PartitionCols:_col1
-                                                  Merge Join Operator [MERGEJOIN_125] (rows=31519516 width=234)
-                                                    Conds:RS_140._col0=RS_143._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                  <-Map 1 [SIMPLE_EDGE] vectorized
-                                                    SHUFFLE [RS_140]
-                                                      PartitionCols:_col0
-                                                      Select Operator [SEL_139] (rows=283695062 width=243)
-                                                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                        Filter Operator [FIL_138] (rows=283695062 width=243)
-                                                          predicate:((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))) and cs_call_center_sk is not null and cs_ship_addr_sk is not null and cs_ship_date_sk is not null)
-                                                          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 13 [BROADCAST_EDGE] vectorized
-                                                            BROADCAST [RS_137]
-                                                              Group By Operator [GBY_136] (rows=1 width=12)
-                                                                Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                              <-Map 12 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                SHUFFLE [RS_135]
-                                                                  Group By Operator [GBY_134] (rows=1 width=12)
-                                                                    Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                    Select Operator [SEL_133] (rows=784314 width=4)
-                                                                      Output:["_col0"]
-                                                                       Please refer to the previous Select Operator [SEL_131]
-                                                  <-Map 11 [SIMPLE_EDGE] vectorized
-                                                    SHUFFLE [RS_143]
-                                                      PartitionCols:_col0
-                                                      Select Operator [SEL_142] (rows=8116 width=98)
-                                                        Output:["_col0"]
-                                                        Filter Operator [FIL_141] (rows=8116 width=98)
-                                                          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 15 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_152]
-                                          PartitionCols:_col0
-                                          Group By Operator [GBY_151] (rows=286548719 width=7)
-                                            Output:["_col0","_col1"],keys:_col0, _col1
-                                            Select Operator [SEL_150] (rows=286548719 width=7)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_149] (rows=286548719 width=7)
-                                                predicate:((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))) and 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 10 [BROADCAST_EDGE] vectorized
-                                                  BROADCAST [RS_148]
-                                                    Group By Operator [GBY_147] (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_115]
-                                                        Group By Operator [GBY_114] (rows=1 width=12)
-                                                          Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                          Select Operator [SEL_113] (rows=5150256 width=8)
-                                                            Output:["_col0"]
-                                                             Please refer to the previous Merge Join Operator [MERGEJOIN_127]
+                                                     Please refer to the previous Merge Join Operator [MERGEJOIN_123]
 
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 f848093..1441e17 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,7 +1,7 @@
-Warning: Shuffle Join MERGEJOIN[443][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 19' is a cross product
-Warning: Shuffle Join MERGEJOIN[444][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 20' is a cross product
-Warning: Shuffle Join MERGEJOIN[446][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 23' is a cross product
-Warning: Shuffle Join MERGEJOIN[447][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 24' is a cross product
+Warning: Shuffle Join MERGEJOIN[442][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 19' is a cross product
+Warning: Shuffle Join MERGEJOIN[443][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 20' is a cross product
+Warning: Shuffle Join MERGEJOIN[445][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 23' is a cross product
+Warning: Shuffle Join MERGEJOIN[446][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Reducer 24' 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
@@ -159,351 +159,349 @@ Reducer 9 <- Map 8 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 6 vectorized
-      File Output Operator [FS_537]
-        Limit [LIM_536] (rows=1 width=112)
-          Number of rows:100
-          Group By Operator [GBY_535] (rows=1 width=112)
-            Output:["_col0"],aggregations:["sum(VALUE._col0)"]
-          <-Union 5 [CUSTOM_SIMPLE_EDGE]
-            <-Reducer 12 [CONTAINS]
-              Reduce Output Operator [RS_460]
-                Group By Operator [GBY_459] (rows=1 width=112)
-                  Output:["_col0"],aggregations:["sum(_col0)"]
-                  Select Operator [SEL_457] (rows=155 width=112)
-                    Output:["_col0"]
-                    Merge Join Operator [MERGEJOIN_456] (rows=155 width=0)
-                      Conds:RS_198._col1=RS_564._col0(Left Semi),Output:["_col3","_col4"]
-                    <-Reducer 11 [SIMPLE_EDGE]
-                      SHUFFLE [RS_198]
-                        PartitionCols:_col1
-                        Merge Join Operator [MERGEJOIN_448] (rows=3941101 width=118)
-                          Conds:RS_193._col2=RS_194._col0(Inner),Output:["_col1","_col3","_col4"]
-                        <-Reducer 10 [SIMPLE_EDGE]
-                          PARTITION_ONLY_SHUFFLE [RS_193]
-                            PartitionCols:_col2
-                            Merge Join Operator [MERGEJOIN_438] (rows=3941102 width=122)
-                              Conds:RS_542._col0=RS_465._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                            <-Map 8 [SIMPLE_EDGE] vectorized
-                              PARTITION_ONLY_SHUFFLE [RS_465]
-                                PartitionCols:_col0
-                                Select Operator [SEL_462] (rows=50 width=4)
-                                  Output:["_col0"]
-                                  Filter Operator [FIL_461] (rows=50 width=12)
-                                    predicate:((d_moy = 1) and (d_year = 1999))
-                                    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 38 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_542]
-                                PartitionCols:_col0
-                                Select Operator [SEL_541] (rows=143930993 width=127)
-                                  Output:["_col0","_col1","_col2","_col3","_col4"]
-                                  Filter Operator [FIL_540] (rows=143930993 width=127)
-                                    predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_191_date_dim_d_date_sk_min) AND DynamicValue(RS_191_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_191_date_dim_d_date_sk_bloom_filter))) and ws_bill_customer_sk is not null and ws_sold_date_sk is not null)
-                                    TableScan [TS_101] (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_539]
-                                        Group By Operator [GBY_538] (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_470]
-                                            Group By Operator [GBY_468] (rows=1 width=12)
-                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                              Select Operator [SEL_466] (rows=50 width=4)
-                                                Output:["_col0"]
-                                                 Please refer to the previous Select Operator [SEL_462]
-                        <-Reducer 24 [SIMPLE_EDGE]
-                          SHUFFLE [RS_194]
-                            PartitionCols:_col0
-                            Select Operator [SEL_168] (rows=471875 width=3)
-                              Output:["_col0"]
-                              Filter Operator [FIL_167] (rows=471875 width=227)
-                                predicate:(_col3 > (0.95 * _col1))
-                                Merge Join Operator [MERGEJOIN_447] (rows=1415626 width=227)
-                                  Conds:(Inner),Output:["_col1","_col2","_col3"]
-                                <-Reducer 23 [CUSTOM_SIMPLE_EDGE]
-                                  PARTITION_ONLY_SHUFFLE [RS_164]
-                                    Merge Join Operator [MERGEJOIN_446] (rows=1 width=112)
-                                      Conds:(Inner),Output:["_col1"]
-                                    <-Reducer 22 [CUSTOM_SIMPLE_EDGE] vectorized
-                                      PARTITION_ONLY_SHUFFLE [RS_548]
-                                        Select Operator [SEL_547] (rows=1 width=8)
-                                          Filter Operator [FIL_546] (rows=1 width=8)
-                                            predicate:(sq_count_check(_col0) <= 1)
-                                            Group By Operator [GBY_545] (rows=1 width=8)
-                                              Output:["_col0"],aggregations:["count()"]
-                                              Select Operator [SEL_544] (rows=1 width=8)
-                                                Group By Operator [GBY_543] (rows=1 width=8)
-                                                  Output:["_col0"],aggregations:["count(VALUE._col0)"]
-                                                <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                  PARTITION_ONLY_SHUFFLE [RS_498]
-                                                    Group By Operator [GBY_494] (rows=1 width=8)
-                                                      Output:["_col0"],aggregations:["count(_col0)"]
-                                                      Select Operator [SEL_490] (rows=50562 width=112)
-                                                        Output:["_col0"]
-                                                        Group By Operator [GBY_487] (rows=50562 width=112)
-                                                          Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                                                        <-Reducer 16 [SIMPLE_EDGE]
-                                                          SHUFFLE [RS_17]
-                                                            PartitionCols:_col0
-                                                            Group By Operator [GBY_16] (rows=455058 width=112)
-                                                              Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col1
-                                                              Merge Join Operator [MERGEJOIN_434] (rows=18762463 width=112)
-                                                                Conds:RS_486._col0=RS_478._col0(Inner),Output:["_col1","_col2"]
-                                                              <-Map 26 [SIMPLE_EDGE] vectorized
-                                                                PARTITION_ONLY_SHUFFLE [RS_478]
-                                                                  PartitionCols:_col0
-                                                                  Select Operator [SEL_477] (rows=2609 width=4)
-                                                                    Output:["_col0"]
-                                                                    Filter Operator [FIL_476] (rows=2609 width=8)
-                                                                      predicate:(d_year) IN (1999, 2000, 2001, 2002)
-                                                                      TableScan [TS_9] (rows=73049 width=8)
-                                                                        default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
-                                                              <-Map 15 [SIMPLE_EDGE] vectorized
-                                                                SHUFFLE [RS_486]
-                                                                  PartitionCols:_col0
-                                                                  Select Operator [SEL_485] (rows=525327388 width=119)
-                                                                    Output:["_col0","_col1","_col2"]
-                                                                    Filter Operator [FIL_484] (rows=525327388 width=118)
-                                                                      predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_13_date_dim_d_date_sk_min) AND DynamicValue(RS_13_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_13_date_dim_d_date_sk_bloom_filter))) and ss_customer_sk is not null and ss_sold_date_sk is not null)
-                                                                      TableScan [TS_6] (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 27 [BROADCAST_EDGE] vectorized
-                                                                        BROADCAST [RS_483]
-                                                                          Group By Operator [GBY_482] (rows=1 width=12)
-                                                                            Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                          <-Map 26 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                            PARTITION_ONLY_SHUFFLE [RS_481]
-                                                                              Group By Operator [GBY_480] (rows=1 width=12)
-                                                                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                                Select Operator [SEL_479] (rows=2609 width=4)
-                                                                                  Output:["_col0"]
-                                                                                   Please refer to the previous Select Operator [SEL_477]
-                                    <-Reducer 25 [CUSTOM_SIMPLE_EDGE] vectorized
-                                      PARTITION_ONLY_SHUFFLE [RS_550]
-                                        Group By Operator [GBY_549] (rows=1 width=112)
-                                          Output:["_col0"],aggregations:["max(VALUE._col0)"]
-                                        <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
-                                          PARTITION_ONLY_SHUFFLE [RS_499]
-                                            Group By Operator [GBY_495] (rows=1 width=112)
-                                              Output:["_col0"],aggregations:["max(_col1)"]
-                                              Select Operator [SEL_491] (rows=50562 width=112)
-                                                Output:["_col1"]
-                                                 Please refer to the previous Group By Operator [GBY_487]
-                                <-Reducer 40 [CUSTOM_SIMPLE_EDGE] vectorized
-                                  PARTITION_ONLY_SHUFFLE [RS_558]
-                                    Group By Operator [GBY_557] (rows=1415626 width=115)
-                                      Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                                    <-Map 39 [SIMPLE_EDGE] vectorized
-                                      SHUFFLE [RS_556]
-                                        PartitionCols:_col0
-                                        Group By Operator [GBY_555] (rows=550080312 width=115)
-                                          Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
-                                          Select Operator [SEL_554] (rows=550080312 width=114)
-                                            Output:["_col0","_col1"]
-                                            Filter Operator [FIL_553] (rows=550080312 width=114)
-                                              predicate:((ss_customer_sk BETWEEN DynamicValue(RS_193_web_sales_ws_bill_customer_sk_min) AND DynamicValue(RS_193_web_sales_ws_bill_customer_sk_max) and in_bloom_filter(ss_customer_sk, DynamicValue(RS_193_web_sales_ws_bill_customer_sk_bloom_filter))) and ss_customer_sk is not null)
-                                              TableScan [TS_153] (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_552]
-                                                  Group By Operator [GBY_551] (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_412]
-                                                      Group By Operator [GBY_411] (rows=1 width=12)
-                                                        Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                        Select Operator [SEL_410] (rows=3941102 width=7)
-                                                          Output:["_col0"]
-                                                           Please refer to the previous Merge Join Operator [MERGEJOIN_438]
-                    <-Reducer 34 [SIMPLE_EDGE] vectorized
-                      SHUFFLE [RS_564]
-                        PartitionCols:_col0
-                        Group By Operator [GBY_563] (rows=2235 width=4)
-                          Output:["_col0"],keys:_col0
-                          Select Operator [SEL_562] (rows=6548799 width=4)
+      File Output Operator [FS_535]
+        Group By Operator [GBY_534] (rows=1 width=112)
+          Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+        <-Union 5 [CUSTOM_SIMPLE_EDGE]
+          <-Reducer 12 [CONTAINS]
+            Reduce Output Operator [RS_459]
+              Group By Operator [GBY_458] (rows=1 width=112)
+                Output:["_col0"],aggregations:["sum(_col0)"]
+                Select Operator [SEL_456] (rows=155 width=112)
+                  Output:["_col0"]
+                  Merge Join Operator [MERGEJOIN_455] (rows=155 width=0)
+                    Conds:RS_198._col1=RS_562._col0(Left Semi),Output:["_col3","_col4"]
+                  <-Reducer 11 [SIMPLE_EDGE]
+                    SHUFFLE [RS_198]
+                      PartitionCols:_col1
+                      Merge Join Operator [MERGEJOIN_447] (rows=3941101 width=118)
+                        Conds:RS_193._col2=RS_194._col0(Inner),Output:["_col1","_col3","_col4"]
+                      <-Reducer 10 [SIMPLE_EDGE]
+                        PARTITION_ONLY_SHUFFLE [RS_193]
+                          PartitionCols:_col2
+                          Merge Join Operator [MERGEJOIN_437] (rows=3941102 width=122)
+                            Conds:RS_540._col0=RS_464._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                          <-Map 8 [SIMPLE_EDGE] vectorized
+                            PARTITION_ONLY_SHUFFLE [RS_464]
+                              PartitionCols:_col0
+                              Select Operator [SEL_461] (rows=50 width=4)
+                                Output:["_col0"]
+                                Filter Operator [FIL_460] (rows=50 width=12)
+                                  predicate:((d_moy = 1) and (d_year = 1999))
+                                  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 38 [SIMPLE_EDGE] vectorized
+                            SHUFFLE [RS_540]
+                              PartitionCols:_col0
+                              Select Operator [SEL_539] (rows=143930993 width=127)
+                                Output:["_col0","_col1","_col2","_col3","_col4"]
+                                Filter Operator [FIL_538] (rows=143930993 width=127)
+                                  predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_191_date_dim_d_date_sk_min) AND DynamicValue(RS_191_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_191_date_dim_d_date_sk_bloom_filter))) and ws_bill_customer_sk is not null and ws_sold_date_sk is not null)
+                                  TableScan [TS_101] (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_537]
+                                      Group By Operator [GBY_536] (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_469]
+                                          Group By Operator [GBY_467] (rows=1 width=12)
+                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                            Select Operator [SEL_465] (rows=50 width=4)
+                                              Output:["_col0"]
+                                               Please refer to the previous Select Operator [SEL_461]
+                      <-Reducer 24 [SIMPLE_EDGE]
+                        SHUFFLE [RS_194]
+                          PartitionCols:_col0
+                          Select Operator [SEL_168] (rows=471875 width=3)
                             Output:["_col0"]
-                            Filter Operator [FIL_561] (rows=6548799 width=290)
-                              predicate:(_col3 > 4L)
-                              Select Operator [SEL_560] (rows=19646398 width=290)
-                                Output:["_col1","_col3"]
-                                Group By Operator [GBY_559] (rows=19646398 width=290)
-                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                <-Reducer 32 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_185]
-                                    PartitionCols:_col0, _col1, _col2
-                                    Group By Operator [GBY_83] (rows=19646398 width=290)
-                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col3, _col5
-                                      Merge Join Operator [MERGEJOIN_437] (rows=19646398 width=282)
-                                        Conds:RS_79._col1=RS_528._col0(Inner),Output:["_col3","_col4","_col5"]
-                                      <-Map 37 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_528]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_527] (rows=462000 width=188)
-                                            Output:["_col0","_col1"]
-                                            TableScan [TS_74] (rows=462000 width=188)
-                                              default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_item_desc"]
-                                      <-Reducer 31 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_79]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_436] (rows=19646398 width=98)
-                                            Conds:RS_526._col0=RS_518._col0(Inner),Output:["_col1","_col3"]
-                                          <-Map 35 [SIMPLE_EDGE] vectorized
-                                            PARTITION_ONLY_SHUFFLE [RS_518]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_517] (rows=2609 width=98)
-                                                Output:["_col0","_col1"]
-                                                Filter Operator [FIL_516] (rows=2609 width=102)
-                                                  predicate:(d_year) IN (1999, 2000, 2001, 2002)
-                                                  TableScan [TS_71] (rows=73049 width=102)
-                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_date","d_year"]
-                                          <-Map 30 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_526]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_525] (rows=550076554 width=7)
-                                                Output:["_col0","_col1"]
-                                                Filter Operator [FIL_524] (rows=550076554 width=7)
-                                                  predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_77_date_dim_d_date_sk_min) AND DynamicValue(RS_77_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_77_date_dim_d_date_sk_bloom_filter))) and ss_sold_date_sk is not null)
-                                                  TableScan [TS_68] (rows=575995635 width=7)
-                                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk"]
-                                                  <-Reducer 36 [BROADCAST_EDGE] vectorized
-                                                    BROADCAST [RS_523]
-                                                      Group By Operator [GBY_522] (rows=1 width=12)
-                                                        Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                      <-Map 35 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                        PARTITION_ONLY_SHUFFLE [RS_521]
-                                                          Group By Operator [GBY_520] (rows=1 width=12)
-                                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                            Select Operator [SEL_519] (rows=2609 width=4)
-                                                              Output:["_col0"]
-                                                               Please refer to the previous Select Operator [SEL_517]
-            <-Reducer 4 [CONTAINS]
-              Reduce Output Operator [RS_455]
-                Group By Operator [GBY_454] (rows=1 width=112)
-                  Output:["_col0"],aggregations:["sum(_col0)"]
-                  Select Operator [SEL_452] (rows=304 width=112)
-                    Output:["_col0"]
-                    Merge Join Operator [MERGEJOIN_451] (rows=304 width=0)
-                      Conds:RS_97._col2=RS_534._col0(Left Semi),Output:["_col3","_col4"]
-                    <-Reducer 3 [SIMPLE_EDGE]
-                      SHUFFLE [RS_97]
-                        PartitionCols:_col2
-                        Merge Join Operator [MERGEJOIN_445] (rows=7751875 width=98)
-                          Conds:RS_92._col1=RS_93._col0(Inner),Output:["_col2","_col3","_col4"]
-                        <-Reducer 2 [SIMPLE_EDGE]
-                          PARTITION_ONLY_SHUFFLE [RS_92]
-                            PartitionCols:_col1
-                            Merge Join Operator [MERGEJOIN_433] (rows=7751875 width=101)
-                              Conds:RS_475._col0=RS_463._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                            <-Map 8 [SIMPLE_EDGE] vectorized
-                              PARTITION_ONLY_SHUFFLE [RS_463]
-                                PartitionCols:_col0
-                                 Please refer to the previous Select Operator [SEL_462]
-                            <-Map 1 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_475]
-                                PartitionCols:_col0
-                                Select Operator [SEL_474] (rows=285117831 width=127)
-                                  Output:["_col0","_col1","_col2","_col3","_col4"]
-                                  Filter Operator [FIL_473] (rows=285117831 width=127)
-                                    predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_90_date_dim_d_date_sk_min) AND DynamicValue(RS_90_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_90_date_dim_d_date_sk_bloom_filter))) and cs_bill_customer_sk is not null and cs_sold_date_sk is not null)
-                                    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
-                                      BROADCAST [RS_472]
-                                        Group By Operator [GBY_471] (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_469]
-                                            Group By Operator [GBY_467] (rows=1 width=12)
-                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                              Select Operator [SEL_464] (rows=50 width=4)
-                                                Output:["_col0"]
-                                                 Please refer to the previous Select Operator [SEL_462]
-                        <-Reducer 20 [SIMPLE_EDGE]
-                          SHUFFLE [RS_93]
-                            PartitionCols:_col0
-                            Select Operator [SEL_67] (rows=471875 width=3)
-                              Output:["_col0"]
-                              Filter Operator [FIL_66] (rows=471875 width=227)
-                                predicate:(_col3 > (0.95 * _col1))
-                                Merge Join Operator [MERGEJOIN_444] (rows=1415626 width=227)
-                                  Conds:(Inner),Output:["_col1","_col2","_col3"]
-                                <-Reducer 19 [CUSTOM_SIMPLE_EDGE]
-                                  PARTITION_ONLY_SHUFFLE [RS_63]
-                                    Merge Join Operator [MERGEJOIN_443] (rows=1 width=112)
-                                      Conds:(Inner),Output:["_col1"]
-                                    <-Reducer 18 [CUSTOM_SIMPLE_EDGE] vectorized
-                                      PARTITION_ONLY_SHUFFLE [RS_505]
-                                        Select Operator [SEL_504] (rows=1 width=8)
-                                          Filter Operator [FIL_503] (rows=1 width=8)
-                                            predicate:(sq_count_check(_col0) <= 1)
-                                            Group By Operator [GBY_502] (rows=1 width=8)
-                                              Output:["_col0"],aggregations:["count()"]
-                                              Select Operator [SEL_501] (rows=1 width=8)
-                                                Group By Operator [GBY_500] (rows=1 width=8)
-                                                  Output:["_col0"],aggregations:["count(VALUE._col0)"]
-                                                <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                  PARTITION_ONLY_SHUFFLE [RS_496]
-                                                    Group By Operator [GBY_492] (rows=1 width=8)
-                                                      Output:["_col0"],aggregations:["count(_col0)"]
-                                                      Select Operator [SEL_488] (rows=50562 width=112)
+                            Filter Operator [FIL_167] (rows=471875 width=227)
+                              predicate:(_col3 > (0.95 * _col1))
+                              Merge Join Operator [MERGEJOIN_446] (rows=1415626 width=227)
+                                Conds:(Inner),Output:["_col1","_col2","_col3"]
+                              <-Reducer 23 [CUSTOM_SIMPLE_EDGE]
+                                PARTITION_ONLY_SHUFFLE [RS_164]
+                                  Merge Join Operator [MERGEJOIN_445] (rows=1 width=112)
+                                    Conds:(Inner),Output:["_col1"]
+                                  <-Reducer 22 [CUSTOM_SIMPLE_EDGE] vectorized
+                                    PARTITION_ONLY_SHUFFLE [RS_546]
+                                      Select Operator [SEL_545] (rows=1 width=8)
+                                        Filter Operator [FIL_544] (rows=1 width=8)
+                                          predicate:(sq_count_check(_col0) <= 1)
+                                          Group By Operator [GBY_543] (rows=1 width=8)
+                                            Output:["_col0"],aggregations:["count()"]
+                                            Select Operator [SEL_542] (rows=1 width=8)
+                                              Group By Operator [GBY_541] (rows=1 width=8)
+                                                Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                              <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                PARTITION_ONLY_SHUFFLE [RS_497]
+                                                  Group By Operator [GBY_493] (rows=1 width=8)
+                                                    Output:["_col0"],aggregations:["count(_col0)"]
+                                                    Select Operator [SEL_489] (rows=50562 width=112)
+                                                      Output:["_col0"]
+                                                      Group By Operator [GBY_486] (rows=50562 width=112)
+                                                        Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
+                                                      <-Reducer 16 [SIMPLE_EDGE]
+                                                        SHUFFLE [RS_17]
+                                                          PartitionCols:_col0
+                                                          Group By Operator [GBY_16] (rows=455058 width=112)
+                                                            Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col1
+                                                            Merge Join Operator [MERGEJOIN_433] (rows=18762463 width=112)
+                                                              Conds:RS_485._col0=RS_477._col0(Inner),Output:["_col1","_col2"]
+                                                            <-Map 26 [SIMPLE_EDGE] vectorized
+                                                              PARTITION_ONLY_SHUFFLE [RS_477]
+                                                                PartitionCols:_col0
+                                                                Select Operator [SEL_476] (rows=2609 width=4)
+                                                                  Output:["_col0"]
+                                                                  Filter Operator [FIL_475] (rows=2609 width=8)
+                                                                    predicate:(d_year) IN (1999, 2000, 2001, 2002)
+                                                                    TableScan [TS_9] (rows=73049 width=8)
+                                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"]
+                                                            <-Map 15 [SIMPLE_EDGE] vectorized
+                                                              SHUFFLE [RS_485]
+                                                                PartitionCols:_col0
+                                                                Select Operator [SEL_484] (rows=525327388 width=119)
+                                                                  Output:["_col0","_col1","_col2"]
+                                                                  Filter Operator [FIL_483] (rows=525327388 width=118)
+                                                                    predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_13_date_dim_d_date_sk_min) AND DynamicValue(RS_13_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_13_date_dim_d_date_sk_bloom_filter))) and ss_customer_sk is not null and ss_sold_date_sk is not null)
+                                                                    TableScan [TS_6] (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 27 [BROADCAST_EDGE] vectorized
+                                                                      BROADCAST [RS_482]
+                                                                        Group By Operator [GBY_481] (rows=1 width=12)
+                                                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                                        <-Map 26 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                          PARTITION_ONLY_SHUFFLE [RS_480]
+                                                                            Group By Operator [GBY_479] (rows=1 width=12)
+                                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                              Select Operator [SEL_478] (rows=2609 width=4)
+                                                                                Output:["_col0"]
+                                                                                 Please refer to the previous Select Operator [SEL_476]
+                                  <-Reducer 25 [CUSTOM_SIMPLE_EDGE] vectorized
+                                    PARTITION_ONLY_SHUFFLE [RS_548]
+                                      Group By Operator [GBY_547] (rows=1 width=112)
+                                        Output:["_col0"],aggregations:["max(VALUE._col0)"]
+                                      <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
+                                        PARTITION_ONLY_SHUFFLE [RS_498]
+                                          Group By Operator [GBY_494] (rows=1 width=112)
+                                            Output:["_col0"],aggregations:["max(_col1)"]
+                                            Select Operator [SEL_490] (rows=50562 width=112)
+                                              Output:["_col1"]
+                                               Please refer to the previous Group By Operator [GBY_486]
+                              <-Reducer 40 [CUSTOM_SIMPLE_EDGE] vectorized
+                                PARTITION_ONLY_SHUFFLE [RS_556]
+                                  Group By Operator [GBY_555] (rows=1415626 width=115)
+                                    Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
+                                  <-Map 39 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_554]
+                                      PartitionCols:_col0
+                                      Group By Operator [GBY_553] (rows=550080312 width=115)
+                                        Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
+                                        Select Operator [SEL_552] (rows=550080312 width=114)
+                                          Output:["_col0","_col1"]
+                                          Filter Operator [FIL_551] (rows=550080312 width=114)
+                                            predicate:((ss_customer_sk BETWEEN DynamicValue(RS_193_web_sales_ws_bill_customer_sk_min) AND DynamicValue(RS_193_web_sales_ws_bill_customer_sk_max) and in_bloom_filter(ss_customer_sk, DynamicValue(RS_193_web_sales_ws_bill_customer_sk_bloom_filter))) and ss_customer_sk is not null)
+                                            TableScan [TS_153] (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_550]
+                                                Group By Operator [GBY_549] (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_411]
+                                                    Group By Operator [GBY_410] (rows=1 width=12)
+                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                      Select Operator [SEL_409] (rows=3941102 width=7)
                                                         Output:["_col0"]
-                                                         Please refer to the previous Group By Operator [GBY_487]
-                                    <-Reducer 21 [CUSTOM_SIMPLE_EDGE] vectorized
-                                      PARTITION_ONLY_SHUFFLE [RS_507]
-                                        Group By Operator [GBY_506] (rows=1 width=112)
-                                          Output:["_col0"],aggregations:["max(VALUE._col0)"]
-                                        <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
-                                          PARTITION_ONLY_SHUFFLE [RS_497]
-                                            Group By Operator [GBY_493] (rows=1 width=112)
-                                              Output:["_col0"],aggregations:["max(_col1)"]
-                                              Select Operator [SEL_489] (rows=50562 width=112)
-                                                Output:["_col1"]
-                                                 Please refer to the previous Group By Operator [GBY_487]
-                                <-Reducer 29 [CUSTOM_SIMPLE_EDGE] vectorized
-                                  PARTITION_ONLY_SHUFFLE [RS_515]
-                                    Group By Operator [GBY_514] (rows=1415626 width=115)
-                                      Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
-                                    <-Map 28 [SIMPLE_EDGE] vectorized
-                                      SHUFFLE [RS_513]
+                                                         Please refer to the previous Merge Join Operator [MERGEJOIN_437]
+                  <-Reducer 34 [SIMPLE_EDGE] vectorized
+                    SHUFFLE [RS_562]
+                      PartitionCols:_col0
+                      Group By Operator [GBY_561] (rows=2235 width=4)
+                        Output:["_col0"],keys:_col0
+                        Select Operator [SEL_560] (rows=6548799 width=4)
+                          Output:["_col0"]
+                          Filter Operator [FIL_559] (rows=6548799 width=290)
+                            predicate:(_col3 > 4L)
+                            Select Operator [SEL_558] (rows=19646398 width=290)
+                              Output:["_col1","_col3"]
+                              Group By Operator [GBY_557] (rows=19646398 width=290)
+                                Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                              <-Reducer 32 [SIMPLE_EDGE]
+                                SHUFFLE [RS_185]
+                                  PartitionCols:_col0, _col1, _col2
+                                  Group By Operator [GBY_83] (rows=19646398 width=290)
+                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col3, _col5
+                                    Merge Join Operator [MERGEJOIN_436] (rows=19646398 width=282)
+                                      Conds:RS_79._col1=RS_527._col0(Inner),Output:["_col3","_col4","_col5"]
+                                    <-Map 37 [SIMPLE_EDGE] vectorized
+                                      SHUFFLE [RS_527]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_512] (rows=550080312 width=115)
-                                          Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
-                                          Select Operator [SEL_511] (rows=550080312 width=114)
-                                            Output:["_col0","_col1"]
-                                            Filter Operator [FIL_510] (rows=550080312 width=114)
-                                              predicate:((ss_customer_sk BETWEEN DynamicValue(RS_92_catalog_sales_cs_bill_customer_sk_min) AND DynamicValue(RS_92_catalog_sales_cs_bill_customer_sk_max) and in_bloom_filter(ss_customer_sk, DynamicValue(RS_92_catalog_sales_cs_bill_customer_sk_bloom_filter))) and ss_customer_sk is not null)
-                                              TableScan [TS_52] (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_509]
-                                                  Group By Operator [GBY_508] (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_339]
-                                                      Group By Operator [GBY_338] (rows=1 width=12)
-                                                        Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                        Select Operator [SEL_337] (rows=7751875 width=6)
-                                                          Output:["_col0"]
-                                                           Please refer to the previous Merge Join Operator [MERGEJOIN_433]
-                    <-Reducer 33 [SIMPLE_EDGE] vectorized
-                      SHUFFLE [RS_534]
-                        PartitionCols:_col0
-                        Group By Operator [GBY_533] (rows=2235 width=4)
-                          Output:["_col0"],keys:_col0
-                          Select Operator [SEL_532] (rows=6548799 width=4)
+                                        Select Operator [SEL_526] (rows=462000 width=188)
+                                          Output:["_col0","_col1"]
+                                          TableScan [TS_74] (rows=462000 width=188)
+                                            default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_item_desc"]
+                                    <-Reducer 31 [SIMPLE_EDGE]
+                                      SHUFFLE [RS_79]
+                                        PartitionCols:_col1
+                                        Merge Join Operator [MERGEJOIN_435] (rows=19646398 width=98)
+                                          Conds:RS_525._col0=RS_517._col0(Inner),Output:["_col1","_col3"]
+                                        <-Map 35 [SIMPLE_EDGE] vectorized
+                                          PARTITION_ONLY_SHUFFLE [RS_517]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_516] (rows=2609 width=98)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_515] (rows=2609 width=102)
+                                                predicate:(d_year) IN (1999, 2000, 2001, 2002)
+                                                TableScan [TS_71] (rows=73049 width=102)
+                                                  default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_date","d_year"]
+                                        <-Map 30 [SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_525]
+                                            PartitionCols:_col0
+                                            Select Operator [SEL_524] (rows=550076554 width=7)
+                                              Output:["_col0","_col1"]
+                                              Filter Operator [FIL_523] (rows=550076554 width=7)
+                                                predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_77_date_dim_d_date_sk_min) AND DynamicValue(RS_77_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_77_date_dim_d_date_sk_bloom_filter))) and ss_sold_date_sk is not null)
+                                                TableScan [TS_68] (rows=575995635 width=7)
+                                                  default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk"]
+                                                <-Reducer 36 [BROADCAST_EDGE] vectorized
+                                                  BROADCAST [RS_522]
+                                                    Group By Operator [GBY_521] (rows=1 width=12)
+                                                      Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                    <-Map 35 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                      PARTITION_ONLY_SHUFFLE [RS_520]
+                                                        Group By Operator [GBY_519] (rows=1 width=12)
+                                                          Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                          Select Operator [SEL_518] (rows=2609 width=4)
+                                                            Output:["_col0"]
+                                                             Please refer to the previous Select Operator [SEL_516]
+          <-Reducer 4 [CONTAINS]
+            Reduce Output Operator [RS_454]
+              Group By Operator [GBY_453] (rows=1 width=112)
+                Output:["_col0"],aggregations:["sum(_col0)"]
+                Select Operator [SEL_451] (rows=304 width=112)
+                  Output:["_col0"]
+                  Merge Join Operator [MERGEJOIN_450] (rows=304 width=0)
+                    Conds:RS_97._col2=RS_533._col0(Left Semi),Output:["_col3","_col4"]
+                  <-Reducer 3 [SIMPLE_EDGE]
+                    SHUFFLE [RS_97]
+                      PartitionCols:_col2
+                      Merge Join Operator [MERGEJOIN_444] (rows=7751875 width=98)
+                        Conds:RS_92._col1=RS_93._col0(Inner),Output:["_col2","_col3","_col4"]
+                      <-Reducer 2 [SIMPLE_EDGE]
+                        PARTITION_ONLY_SHUFFLE [RS_92]
+                          PartitionCols:_col1
+                          Merge Join Operator [MERGEJOIN_432] (rows=7751875 width=101)
+                            Conds:RS_474._col0=RS_462._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                          <-Map 8 [SIMPLE_EDGE] vectorized
+                            PARTITION_ONLY_SHUFFLE [RS_462]
+                              PartitionCols:_col0
+                               Please refer to the previous Select Operator [SEL_461]
+                          <-Map 1 [SIMPLE_EDGE] vectorized
+                            SHUFFLE [RS_474]
+                              PartitionCols:_col0
+                              Select Operator [SEL_473] (rows=285117831 width=127)
+                                Output:["_col0","_col1","_col2","_col3","_col4"]
+                                Filter Operator [FIL_472] (rows=285117831 width=127)
+                                  predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_90_date_dim_d_date_sk_min) AND DynamicValue(RS_90_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_90_date_dim_d_date_sk_bloom_filter))) and cs_bill_customer_sk is not null and cs_sold_date_sk is not null)
+                                  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
+                                    BROADCAST [RS_471]
+                                      Group By Operator [GBY_470] (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_468]
+                                          Group By Operator [GBY_466] (rows=1 width=12)
+                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                            Select Operator [SEL_463] (rows=50 width=4)
+                                              Output:["_col0"]
+                                               Please refer to the previous Select Operator [SEL_461]
+                      <-Reducer 20 [SIMPLE_EDGE]
+                        SHUFFLE [RS_93]
+                          PartitionCols:_col0
+                          Select Operator [SEL_67] (rows=471875 width=3)
                             Output:["_col0"]
-                            Filter Operator [FIL_531] (rows=6548799 width=290)
-                              predicate:(_col3 > 4L)
-                              Select Operator [SEL_530] (rows=19646398 width=290)
-                                Output:["_col1","_col3"]
-                                Group By Operator [GBY_529] (rows=19646398 width=290)
-                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                <-Reducer 32 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_84]
-                                    PartitionCols:_col0, _col1, _col2
-                                     Please refer to the previous Group By Operator [GBY_83]
+                            Filter Operator [FIL_66] (rows=471875 width=227)
+                              predicate:(_col3 > (0.95 * _col1))
+                              Merge Join Operator [MERGEJOIN_443] (rows=1415626 width=227)
+                                Conds:(Inner),Output:["_col1","_col2","_col3"]
+                              <-Reducer 19 [CUSTOM_SIMPLE_EDGE]
+                                PARTITION_ONLY_SHUFFLE [RS_63]
+                                  Merge Join Operator [MERGEJOIN_442] (rows=1 width=112)
+                                    Conds:(Inner),Output:["_col1"]
+                                  <-Reducer 18 [CUSTOM_SIMPLE_EDGE] vectorized
+                                    PARTITION_ONLY_SHUFFLE [RS_504]
+                                      Select Operator [SEL_503] (rows=1 width=8)
+                                        Filter Operator [FIL_502] (rows=1 width=8)
+                                          predicate:(sq_count_check(_col0) <= 1)
+                                          Group By Operator [GBY_501] (rows=1 width=8)
+                                            Output:["_col0"],aggregations:["count()"]
+                                            Select Operator [SEL_500] (rows=1 width=8)
+                                              Group By Operator [GBY_499] (rows=1 width=8)
+                                                Output:["_col0"],aggregations:["count(VALUE._col0)"]
+                                              <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                PARTITION_ONLY_SHUFFLE [RS_495]
+                                                  Group By Operator [GBY_491] (rows=1 width=8)
+                                                    Output:["_col0"],aggregations:["count(_col0)"]
+                                                    Select Operator [SEL_487] (rows=50562 width=112)
+                                                      Output:["_col0"]
+                                                       Please refer to the previous Group By Operator [GBY_486]
+                                  <-Reducer 21 [CUSTOM_SIMPLE_EDGE] vectorized
+                                    PARTITION_ONLY_SHUFFLE [RS_506]
+                                      Group By Operator [GBY_505] (rows=1 width=112)
+                                        Output:["_col0"],aggregations:["max(VALUE._col0)"]
+                                      <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
+                                        PARTITION_ONLY_SHUFFLE [RS_496]
+                                          Group By Operator [GBY_492] (rows=1 width=112)
+                                            Output:["_col0"],aggregations:["max(_col1)"]
+                                            Select Operator [SEL_488] (rows=50562 width=112)
+                                              Output:["_col1"]
+                                               Please refer to the previous Group By Operator [GBY_486]
+                              <-Reducer 29 [CUSTOM_SIMPLE_EDGE] vectorized
+                                PARTITION_ONLY_SHUFFLE [RS_514]
+                                  Group By Operator [GBY_513] (rows=1415626 width=115)
+                                    Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0
+                                  <-Map 28 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_512]
+                                      PartitionCols:_col0
+                                      Group By Operator [GBY_511] (rows=550080312 width=115)
+                                        Output:["_col0","_col1"],aggregations:["sum(_col1)"],keys:_col0
+                                        Select Operator [SEL_510] (rows=550080312 width=114)
+                                          Output:["_col0","_col1"]
+                                          Filter Operator [FIL_509] (rows=550080312 width=114)
+                                            predicate:((ss_customer_sk BETWEEN DynamicValue(RS_92_catalog_sales_cs_bill_customer_sk_min) AND DynamicValue(RS_92_catalog_sales_cs_bill_customer_sk_max) and in_bloom_filter(ss_customer_sk, DynamicValue(RS_92_catalog_sales_cs_bill_customer_sk_bloom_filter))) and ss_customer_sk is not null)
+                                            TableScan [TS_52] (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_508]
+                                                Group By Operator [GBY_507] (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_338]
+                                                    Group By Operator [GBY_337] (rows=1 width=12)
+                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                      Select Operator [SEL_336] (rows=7751875 width=6)
+                                                        Output:["_col0"]
+                                                         Please refer to the previous Merge Join Operator [MERGEJOIN_432]
+                  <-Reducer 33 [SIMPLE_EDGE] vectorized
+                    SHUFFLE [RS_533]
+                      PartitionCols:_col0
+                      Group By Operator [GBY_532] (rows=2235 width=4)
+                        Output:["_col0"],keys:_col0
+                        Select Operator [SEL_531] (rows=6548799 width=4)
+                          Output:["_col0"]
+                          Filter Operator [FIL_530] (rows=6548799 width=290)
+                            predicate:(_col3 > 4L)
+                            Select Operator [SEL_529] (rows=19646398 width=290)
+                              Output:["_col1","_col3"]
+                              Group By Operator [GBY_528] (rows=19646398 width=290)
+                                Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                              <-Reducer 32 [SIMPLE_EDGE]
+                                SHUFFLE [RS_84]
+                                  PartitionCols:_col0, _col1, _col2
+                                   Please refer to the previous Group By Operator [GBY_83]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query28.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query28.q.out
index c7fd970..e010dc7 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query28.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query28.q.out
@@ -1,8 +1,8 @@
-Warning: Shuffle Join MERGEJOIN[102][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product
-Warning: Shuffle Join MERGEJOIN[103][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
-Warning: Shuffle Join MERGEJOIN[104][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
-Warning: Shuffle Join MERGEJOIN[105][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 7' is a cross product
-Warning: Shuffle Join MERGEJOIN[106][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 8' is a cross product
+Warning: Shuffle Join MERGEJOIN[101][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[102][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[103][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[104][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[105][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 8' is a cross product
 PREHOOK: query: explain
 select  *
 from (select avg(ss_list_price) B1_LP
@@ -134,163 +134,161 @@ Reducer 9 <- Map 1 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 8
-      File Output Operator [FS_59]
-        Limit [LIM_58] (rows=1 width=768)
-          Number of rows:100
-          Select Operator [SEL_57] (rows=1 width=768)
-            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
-            Merge Join Operator [MERGEJOIN_106] (rows=1 width=768)
-              Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
-            <-Reducer 18 [CUSTOM_SIMPLE_EDGE] vectorized
-              PARTITION_ONLY_SHUFFLE [RS_166]
-                Select Operator [SEL_165] (rows=1 width=128)
-                  Output:["_col0","_col1","_col2"]
-                  Group By Operator [GBY_164] (rows=1 width=128)
-                    Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
-                  <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
-                    PARTITION_ONLY_SHUFFLE [RS_163]
-                      Group By Operator [GBY_162] (rows=1 width=128)
-                        Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
-                        Group By Operator [GBY_161] (rows=7618989 width=226)
-                          Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                        <-Map 1 [SIMPLE_EDGE] vectorized
-                          SHUFFLE [RS_130]
-                            PartitionCols:_col0
-                            Group By Operator [GBY_124] (rows=7618989 width=226)
-                              Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
-                              Select Operator [SEL_118] (rows=15237978 width=324)
-                                Output:["ss_list_price"]
-                                Filter Operator [FIL_112] (rows=15237978 width=324)
-                                  predicate:((ss_list_price BETWEEN 91 AND 101 or ss_coupon_amt BETWEEN 1430 AND 2430 or ss_wholesale_cost BETWEEN 32 AND 52) and ss_quantity BETWEEN 6 AND 10)
-                                  TableScan [TS_0] (rows=575995635 width=324)
-                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_quantity","ss_wholesale_cost","ss_list_price","ss_coupon_amt"]
-            <-Reducer 7 [CUSTOM_SIMPLE_EDGE]
-              PARTITION_ONLY_SHUFFLE [RS_54]
-                Merge Join Operator [MERGEJOIN_105] (rows=1 width=640)
-                  Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"]
-                <-Reducer 16 [CUSTOM_SIMPLE_EDGE] vectorized
-                  PARTITION_ONLY_SHUFFLE [RS_160]
-                    Select Operator [SEL_159] (rows=1 width=128)
-                      Output:["_col0","_col1","_col2"]
-                      Group By Operator [GBY_158] (rows=1 width=128)
-                        Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
-                      <-Reducer 15 [CUSTOM_SIMPLE_EDGE] vectorized
-                        PARTITION_ONLY_SHUFFLE [RS_157]
-                          Group By Operator [GBY_156] (rows=1 width=128)
-                            Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
-                            Group By Operator [GBY_155] (rows=7618989 width=226)
-                              Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                            <-Map 1 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_129]
-                                PartitionCols:_col0
-                                Group By Operator [GBY_123] (rows=7618989 width=226)
-                                  Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
-                                  Select Operator [SEL_117] (rows=15237978 width=324)
-                                    Output:["ss_list_price"]
-                                    Filter Operator [FIL_111] (rows=15237978 width=324)
-                                      predicate:((ss_list_price BETWEEN 66 AND 76 or ss_coupon_amt BETWEEN 920 AND 1920 or ss_wholesale_cost BETWEEN 4 AND 24) and ss_quantity BETWEEN 11 AND 15)
-                                       Please refer to the previous TableScan [TS_0]
-                <-Reducer 6 [CUSTOM_SIMPLE_EDGE]
-                  PARTITION_ONLY_SHUFFLE [RS_51]
-                    Merge Join Operator [MERGEJOIN_104] (rows=1 width=512)
-                      Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                    <-Reducer 14 [CUSTOM_SIMPLE_EDGE] vectorized
-                      PARTITION_ONLY_SHUFFLE [RS_154]
-                        Select Operator [SEL_153] (rows=1 width=128)
-                          Output:["_col0","_col1","_col2"]
-                          Group By Operator [GBY_152] (rows=1 width=128)
-                            Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
-                          <-Reducer 13 [CUSTOM_SIMPLE_EDGE] vectorized
-                            PARTITION_ONLY_SHUFFLE [RS_151]
-                              Group By Operator [GBY_150] (rows=1 width=128)
-                                Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
-                                Group By Operator [GBY_149] (rows=7618989 width=226)
-                                  Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                                <-Map 1 [SIMPLE_EDGE] vectorized
-                                  SHUFFLE [RS_128]
-                                    PartitionCols:_col0
-                                    Group By Operator [GBY_122] (rows=7618989 width=226)
-                                      Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
-                                      Select Operator [SEL_116] (rows=15237978 width=324)
-                                        Output:["ss_list_price"]
-                                        Filter Operator [FIL_110] (rows=15237978 width=324)
-                                          predicate:((ss_list_price BETWEEN 142 AND 152 or ss_coupon_amt BETWEEN 3054 AND 4054 or ss_wholesale_cost BETWEEN 80 AND 100) and ss_quantity BETWEEN 16 AND 20)
-                                           Please refer to the previous TableScan [TS_0]
-                    <-Reducer 5 [CUSTOM_SIMPLE_EDGE]
-                      PARTITION_ONLY_SHUFFLE [RS_48]
-                        Merge Join Operator [MERGEJOIN_103] (rows=1 width=384)
-                          Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
-                        <-Reducer 12 [CUSTOM_SIMPLE_EDGE] vectorized
-                          PARTITION_ONLY_SHUFFLE [RS_148]
-                            Select Operator [SEL_147] (rows=1 width=128)
-                              Output:["_col0","_col1","_col2"]
-                              Group By Operator [GBY_146] (rows=1 width=128)
-                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
-                              <-Reducer 11 [CUSTOM_SIMPLE_EDGE] vectorized
-                                PARTITION_ONLY_SHUFFLE [RS_145]
-                                  Group By Operator [GBY_144] (rows=1 width=128)
-                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
-                                    Group By Operator [GBY_143] (rows=7618989 width=226)
-                                      Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                                    <-Map 1 [SIMPLE_EDGE] vectorized
-                                      SHUFFLE [RS_127]
-                                        PartitionCols:_col0
-                                        Group By Operator [GBY_121] (rows=7618989 width=226)
-                                          Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
-                                          Select Operator [SEL_115] (rows=15237978 width=324)
-                                            Output:["ss_list_price"]
-                                            Filter Operator [FIL_109] (rows=15237978 width=324)
-                                              predicate:((ss_list_price BETWEEN 135 AND 145 or ss_coupon_amt BETWEEN 14180 AND 15180 or ss_wholesale_cost BETWEEN 38 AND 58) and ss_quantity BETWEEN 21 AND 25)
-                                               Please refer to the previous TableScan [TS_0]
-                        <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-                          PARTITION_ONLY_SHUFFLE [RS_45]
-                            Merge Join Operator [MERGEJOIN_102] (rows=1 width=256)
-                              Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                            <-Reducer 10 [CUSTOM_SIMPLE_EDGE] vectorized
-                              PARTITION_ONLY_SHUFFLE [RS_142]
-                                Select Operator [SEL_141] (rows=1 width=128)
-                                  Output:["_col0","_col1","_col2"]
-                                  Group By Operator [GBY_140] (rows=1 width=128)
-                                    Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
-                                  <-Reducer 9 [CUSTOM_SIMPLE_EDGE] vectorized
-                                    PARTITION_ONLY_SHUFFLE [RS_139]
-                                      Group By Operator [GBY_138] (rows=1 width=128)
-                                        Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
-                                        Group By Operator [GBY_137] (rows=7618989 width=226)
-                                          Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                                        <-Map 1 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_126]
-                                            PartitionCols:_col0
-                                            Group By Operator [GBY_120] (rows=7618989 width=226)
-                                              Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
-                                              Select Operator [SEL_114] (rows=15237978 width=324)
-                                                Output:["ss_list_price"]
-                                                Filter Operator [FIL_108] (rows=15237978 width=324)
-                                                  predicate:((ss_list_price BETWEEN 28 AND 38 or ss_coupon_amt BETWEEN 2513 AND 3513 or ss_wholesale_cost BETWEEN 42 AND 62) and ss_quantity BETWEEN 26 AND 30)
-                                                   Please refer to the previous TableScan [TS_0]
-                            <-Reducer 3 [CUSTOM_SIMPLE_EDGE] vectorized
-                              PARTITION_ONLY_SHUFFLE [RS_136]
-                                Select Operator [SEL_135] (rows=1 width=128)
-                                  Output:["_col0","_col1","_col2"]
-                                  Group By Operator [GBY_134] (rows=1 width=128)
-                                    Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
-                                  <-Reducer 2 [CUSTOM_SIMPLE_EDGE] vectorized
-                                    PARTITION_ONLY_SHUFFLE [RS_133]
-                                      Group By Operator [GBY_132] (rows=1 width=128)
-                                        Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
-                                        Group By Operator [GBY_131] (rows=7618989 width=226)
-                                          Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                                        <-Map 1 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_125]
-                                            PartitionCols:_col0
-                                            Group By Operator [GBY_119] (rows=7618989 width=226)
-                                              Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
-                                              Select Operator [SEL_113] (rows=15237978 width=324)
-                                                Output:["ss_list_price"]
-                                                Filter Operator [FIL_107] (rows=15237978 width=324)
-                                                  predicate:((ss_list_price BETWEEN 11 AND 21 or ss_coupon_amt BETWEEN 460 AND 1460 or ss_wholesale_cost BETWEEN 14 AND 34) and ss_quantity BETWEEN 0 AND 5)
-                                                   Please refer to the previous TableScan [TS_0]
+      File Output Operator [FS_58]
+        Select Operator [SEL_57] (rows=1 width=768)
+          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+          Merge Join Operator [MERGEJOIN_105] (rows=1 width=768)
+            Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"]
+          <-Reducer 18 [CUSTOM_SIMPLE_EDGE] vectorized
+            PARTITION_ONLY_SHUFFLE [RS_165]
+              Select Operator [SEL_164] (rows=1 width=128)
+                Output:["_col0","_col1","_col2"]
+                Group By Operator [GBY_163] (rows=1 width=128)
+                  Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+                <-Reducer 17 [CUSTOM_SIMPLE_EDGE] vectorized
+                  PARTITION_ONLY_SHUFFLE [RS_162]
+                    Group By Operator [GBY_161] (rows=1 width=128)
+                      Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
+                      Group By Operator [GBY_160] (rows=7618989 width=226)
+                        Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                      <-Map 1 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_129]
+                          PartitionCols:_col0
+                          Group By Operator [GBY_123] (rows=7618989 width=226)
+                            Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
+                            Select Operator [SEL_117] (rows=15237978 width=324)
+                              Output:["ss_list_price"]
+                              Filter Operator [FIL_111] (rows=15237978 width=324)
+                                predicate:((ss_list_price BETWEEN 91 AND 101 or ss_coupon_amt BETWEEN 1430 AND 2430 or ss_wholesale_cost BETWEEN 32 AND 52) and ss_quantity BETWEEN 6 AND 10)
+                                TableScan [TS_0] (rows=575995635 width=324)
+                                  default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_quantity","ss_wholesale_cost","ss_list_price","ss_coupon_amt"]
+          <-Reducer 7 [CUSTOM_SIMPLE_EDGE]
+            PARTITION_ONLY_SHUFFLE [RS_54]
+              Merge Join Operator [MERGEJOIN_104] (rows=1 width=640)
+                Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"]
+              <-Reducer 16 [CUSTOM_SIMPLE_EDGE] vectorized
+                PARTITION_ONLY_SHUFFLE [RS_159]
+                  Select Operator [SEL_158] (rows=1 width=128)
+                    Output:["_col0","_col1","_col2"]
+                    Group By Operator [GBY_157] (rows=1 width=128)
+                      Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+                    <-Reducer 15 [CUSTOM_SIMPLE_EDGE] vectorized
+                      PARTITION_ONLY_SHUFFLE [RS_156]
+                        Group By Operator [GBY_155] (rows=1 width=128)
+                          Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
+                          Group By Operator [GBY_154] (rows=7618989 width=226)
+                            Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                          <-Map 1 [SIMPLE_EDGE] vectorized
+                            SHUFFLE [RS_128]
+                              PartitionCols:_col0
+                              Group By Operator [GBY_122] (rows=7618989 width=226)
+                                Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
+                                Select Operator [SEL_116] (rows=15237978 width=324)
+                                  Output:["ss_list_price"]
+                                  Filter Operator [FIL_110] (rows=15237978 width=324)
+                                    predicate:((ss_list_price BETWEEN 66 AND 76 or ss_coupon_amt BETWEEN 920 AND 1920 or ss_wholesale_cost BETWEEN 4 AND 24) and ss_quantity BETWEEN 11 AND 15)
+                                     Please refer to the previous TableScan [TS_0]
+              <-Reducer 6 [CUSTOM_SIMPLE_EDGE]
+                PARTITION_ONLY_SHUFFLE [RS_51]
+                  Merge Join Operator [MERGEJOIN_103] (rows=1 width=512)
+                    Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
+                  <-Reducer 14 [CUSTOM_SIMPLE_EDGE] vectorized
+                    PARTITION_ONLY_SHUFFLE [RS_153]
+                      Select Operator [SEL_152] (rows=1 width=128)
+                        Output:["_col0","_col1","_col2"]
+                        Group By Operator [GBY_151] (rows=1 width=128)
+                          Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+                        <-Reducer 13 [CUSTOM_SIMPLE_EDGE] vectorized
+                          PARTITION_ONLY_SHUFFLE [RS_150]
+                            Group By Operator [GBY_149] (rows=1 width=128)
+                              Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
+                              Group By Operator [GBY_148] (rows=7618989 width=226)
+                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                              <-Map 1 [SIMPLE_EDGE] vectorized
+                                SHUFFLE [RS_127]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_121] (rows=7618989 width=226)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
+                                    Select Operator [SEL_115] (rows=15237978 width=324)
+                                      Output:["ss_list_price"]
+                                      Filter Operator [FIL_109] (rows=15237978 width=324)
+                                        predicate:((ss_list_price BETWEEN 142 AND 152 or ss_coupon_amt BETWEEN 3054 AND 4054 or ss_wholesale_cost BETWEEN 80 AND 100) and ss_quantity BETWEEN 16 AND 20)
+                                         Please refer to the previous TableScan [TS_0]
+                  <-Reducer 5 [CUSTOM_SIMPLE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_48]
+                      Merge Join Operator [MERGEJOIN_102] (rows=1 width=384)
+                        Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
+                      <-Reducer 12 [CUSTOM_SIMPLE_EDGE] vectorized
+                        PARTITION_ONLY_SHUFFLE [RS_147]
+                          Select Operator [SEL_146] (rows=1 width=128)
+                            Output:["_col0","_col1","_col2"]
+                            Group By Operator [GBY_145] (rows=1 width=128)
+                              Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+                            <-Reducer 11 [CUSTOM_SIMPLE_EDGE] vectorized
+                              PARTITION_ONLY_SHUFFLE [RS_144]
+                                Group By Operator [GBY_143] (rows=1 width=128)
+                                  Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
+                                  Group By Operator [GBY_142] (rows=7618989 width=226)
+                                    Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                                  <-Map 1 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_126]
+                                      PartitionCols:_col0
+                                      Group By Operator [GBY_120] (rows=7618989 width=226)
+                                        Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
+                                        Select Operator [SEL_114] (rows=15237978 width=324)
+                                          Output:["ss_list_price"]
+                                          Filter Operator [FIL_108] (rows=15237978 width=324)
+                                            predicate:((ss_list_price BETWEEN 135 AND 145 or ss_coupon_amt BETWEEN 14180 AND 15180 or ss_wholesale_cost BETWEEN 38 AND 58) and ss_quantity BETWEEN 21 AND 25)
+                                             Please refer to the previous TableScan [TS_0]
+                      <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
+                        PARTITION_ONLY_SHUFFLE [RS_45]
+                          Merge Join Operator [MERGEJOIN_101] (rows=1 width=256)
+                            Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                          <-Reducer 10 [CUSTOM_SIMPLE_EDGE] vectorized
+                            PARTITION_ONLY_SHUFFLE [RS_141]
+                              Select Operator [SEL_140] (rows=1 width=128)
+                                Output:["_col0","_col1","_col2"]
+                                Group By Operator [GBY_139] (rows=1 width=128)
+                                  Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+                                <-Reducer 9 [CUSTOM_SIMPLE_EDGE] vectorized
+                                  PARTITION_ONLY_SHUFFLE [RS_138]
+                                    Group By Operator [GBY_137] (rows=1 width=128)
+                                      Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
+                                      Group By Operator [GBY_136] (rows=7618989 width=226)
+                                        Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                                      <-Map 1 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_125]
+                                          PartitionCols:_col0
+                                          Group By Operator [GBY_119] (rows=7618989 width=226)
+                                            Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
+                                            Select Operator [SEL_113] (rows=15237978 width=324)
+                                              Output:["ss_list_price"]
+                                              Filter Operator [FIL_107] (rows=15237978 width=324)
+                                                predicate:((ss_list_price BETWEEN 28 AND 38 or ss_coupon_amt BETWEEN 2513 AND 3513 or ss_wholesale_cost BETWEEN 42 AND 62) and ss_quantity BETWEEN 26 AND 30)
+                                                 Please refer to the previous TableScan [TS_0]
+                          <-Reducer 3 [CUSTOM_SIMPLE_EDGE] vectorized
+                            PARTITION_ONLY_SHUFFLE [RS_135]
+                              Select Operator [SEL_134] (rows=1 width=128)
+                                Output:["_col0","_col1","_col2"]
+                                Group By Operator [GBY_133] (rows=1 width=128)
+                                  Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+                                <-Reducer 2 [CUSTOM_SIMPLE_EDGE] vectorized
+                                  PARTITION_ONLY_SHUFFLE [RS_132]
+                                    Group By Operator [GBY_131] (rows=1 width=128)
+                                      Output:["_col0","_col1","_col2"],aggregations:["sum(_col1)","count(_col2)","count(_col0)"]
+                                      Group By Operator [GBY_130] (rows=7618989 width=226)
+                                        Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                                      <-Map 1 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_124]
+                                          PartitionCols:_col0
+                                          Group By Operator [GBY_118] (rows=7618989 width=226)
+                                            Output:["_col0","_col1","_col2"],aggregations:["sum(ss_list_price)","count(ss_list_price)"],keys:ss_list_price
+                                            Select Operator [SEL_112] (rows=15237978 width=324)
+                                              Output:["ss_list_price"]
+                                              Filter Operator [FIL_106] (rows=15237978 width=324)
+                                                predicate:((ss_list_price BETWEEN 11 AND 21 or ss_coupon_amt BETWEEN 460 AND 1460 or ss_wholesale_cost BETWEEN 14 AND 34) and ss_quantity BETWEEN 0 AND 5)
+                                                 Please refer to the previous TableScan [TS_0]
 
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 2156b9c..a0cd1d2 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
@@ -76,108 +76,106 @@ Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 4 vectorized
-      File Output Operator [FS_130]
-        Limit [LIM_129] (rows=1 width=112)
-          Number of rows:100
-          Group By Operator [GBY_128] (rows=1 width=112)
-            Output:["_col0"],aggregations:["sum(VALUE._col0)"]
-          <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
-            PARTITION_ONLY_SHUFFLE [RS_36]
-              Group By Operator [GBY_35] (rows=1 width=112)
-                Output:["_col0"],aggregations:["sum(_col2)"]
-                Select Operator [SEL_34] (rows=2478 width=112)
-                  Output:["_col2"]
-                  Filter Operator [FIL_33] (rows=2478 width=112)
-                    predicate:(_col2 > _col5)
-                    Merge Join Operator [MERGEJOIN_104] (rows=7434 width=112)
-                      Conds:RS_30._col1=RS_31._col2(Inner),Output:["_col2","_col5"]
-                    <-Reducer 8 [ONE_TO_ONE_EDGE]
-                      PARTITION_ONLY_SHUFFLE [RS_31]
-                        PartitionCols:_col2
-                        Merge Join Operator [MERGEJOIN_103] (rows=97 width=116)
-                          Conds:RS_122._col0=RS_111._col0(Inner),Output:["_col1","_col2"]
-                        <-Map 11 [SIMPLE_EDGE] vectorized
-                          SHUFFLE [RS_111]
-                            PartitionCols:_col0
-                            Select Operator [SEL_110] (rows=669 width=4)
-                              Output:["_col0"]
-                              Filter Operator [FIL_109] (rows=669 width=7)
-                                predicate:(i_manufact_id = 269)
-                                TableScan [TS_20] (rows=462000 width=7)
-                                  default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_manufact_id"]
-                        <-Reducer 7 [ONE_TO_ONE_EDGE] vectorized
-                          FORWARD [RS_122]
-                            PartitionCols:_col0
-                            Select Operator [SEL_121] (rows=6951 width=116)
-                              Output:["_col0","_col1"]
-                              Group By Operator [GBY_120] (rows=6951 width=124)
-                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                              <-Reducer 6 [SIMPLE_EDGE]
-                                SHUFFLE [RS_17]
-                                  PartitionCols:_col0
-                                  Group By Operator [GBY_16] (rows=97314 width=124)
-                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
-                                    Merge Join Operator [MERGEJOIN_102] (rows=31836679 width=110)
-                                      Conds:RS_119._col0=RS_108._col0(Inner),Output:["_col1","_col2"]
-                                    <-Map 5 [SIMPLE_EDGE] vectorized
-                                      SHUFFLE [RS_108]
-                                        PartitionCols:_col0
-                                        Select Operator [SEL_106] (rows=8116 width=4)
+      File Output Operator [FS_128]
+        Group By Operator [GBY_127] (rows=1 width=112)
+          Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+        <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
+          PARTITION_ONLY_SHUFFLE [RS_36]
+            Group By Operator [GBY_35] (rows=1 width=112)
+              Output:["_col0"],aggregations:["sum(_col2)"]
+              Select Operator [SEL_34] (rows=2478 width=112)
+                Output:["_col2"]
+                Filter Operator [FIL_33] (rows=2478 width=112)
+                  predicate:(_col2 > _col5)
+                  Merge Join Operator [MERGEJOIN_103] (rows=7434 width=112)
+                    Conds:RS_30._col1=RS_31._col2(Inner),Output:["_col2","_col5"]
+                  <-Reducer 8 [ONE_TO_ONE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_31]
+                      PartitionCols:_col2
+                      Merge Join Operator [MERGEJOIN_102] (rows=97 width=116)
+                        Conds:RS_121._col0=RS_110._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 11 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_110]
+                          PartitionCols:_col0
+                          Select Operator [SEL_109] (rows=669 width=4)
+                            Output:["_col0"]
+                            Filter Operator [FIL_108] (rows=669 width=7)
+                              predicate:(i_manufact_id = 269)
+                              TableScan [TS_20] (rows=462000 width=7)
+                                default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_manufact_id"]
+                      <-Reducer 7 [ONE_TO_ONE_EDGE] vectorized
+                        FORWARD [RS_121]
+                          PartitionCols:_col0
+                          Select Operator [SEL_120] (rows=6951 width=116)
+                            Output:["_col0","_col1"]
+                            Group By Operator [GBY_119] (rows=6951 width=124)
+                              Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                            <-Reducer 6 [SIMPLE_EDGE]
+                              SHUFFLE [RS_17]
+                                PartitionCols:_col0
+                                Group By Operator [GBY_16] (rows=97314 width=124)
+                                  Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
+                                  Merge Join Operator [MERGEJOIN_101] (rows=31836679 width=110)
+                                    Conds:RS_118._col0=RS_107._col0(Inner),Output:["_col1","_col2"]
+                                  <-Map 5 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_107]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_105] (rows=8116 width=4)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_104] (rows=8116 width=98)
+                                          predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'1998-03-18 00:00:00' AND TIMESTAMP'1998-06-16 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 10 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_118]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_117] (rows=286549727 width=119)
+                                        Output:["_col0","_col1","_col2"]
+                                        Filter Operator [FIL_116] (rows=286549727 width=119)
+                                          predicate:((cs_item_sk BETWEEN DynamicValue(RS_24_item_i_item_sk_min) AND DynamicValue(RS_24_item_i_item_sk_max) and in_bloom_filter(cs_item_sk, DynamicValue(RS_24_item_i_item_sk_bloom_filter))) and cs_sold_date_sk is not null)
+                                          TableScan [TS_6] (rows=287989836 width=119)
+                                            default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_item_sk","cs_ext_discount_amt"]
+                                          <-Reducer 12 [BROADCAST_EDGE] vectorized
+                                            BROADCAST [RS_115]
+                                              Group By Operator [GBY_114] (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
+                                                SHUFFLE [RS_113]
+                                                  Group By Operator [GBY_112] (rows=1 width=12)
+                                                    Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                    Select Operator [SEL_111] (rows=669 width=4)
+                                                      Output:["_col0"]
+                                                       Please refer to the previous Select Operator [SEL_109]
+                  <-Reducer 2 [SIMPLE_EDGE]
+                    SHUFFLE [RS_30]
+                      PartitionCols:_col1
+                      Merge Join Operator [MERGEJOIN_100] (rows=31836679 width=110)
+                        Conds:RS_126._col0=RS_106._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 5 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_106]
+                          PartitionCols:_col0
+                           Please refer to the previous Select Operator [SEL_105]
+                      <-Map 1 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_126]
+                          PartitionCols:_col0
+                          Select Operator [SEL_125] (rows=286549727 width=119)
+                            Output:["_col0","_col1","_col2"]
+                            Filter Operator [FIL_124] (rows=286549727 width=119)
+                              predicate:((cs_item_sk BETWEEN DynamicValue(RS_31_item_i_item_sk_min) AND DynamicValue(RS_31_item_i_item_sk_max) and in_bloom_filter(cs_item_sk, DynamicValue(RS_31_item_i_item_sk_bloom_filter))) and cs_sold_date_sk is not null)
+                              TableScan [TS_0] (rows=287989836 width=119)
+                                default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_item_sk","cs_ext_discount_amt"]
+                              <-Reducer 9 [BROADCAST_EDGE] vectorized
+                                BROADCAST [RS_123]
+                                  Group By Operator [GBY_122] (rows=1 width=12)
+                                    Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                  <-Reducer 8 [CUSTOM_SIMPLE_EDGE]
+                                    PARTITION_ONLY_SHUFFLE [RS_64]
+                                      Group By Operator [GBY_63] (rows=1 width=12)
+                                        Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                        Select Operator [SEL_62] (rows=97 width=8)
                                           Output:["_col0"]
-                                          Filter Operator [FIL_105] (rows=8116 width=98)
-                                            predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'1998-03-18 00:00:00' AND TIMESTAMP'1998-06-16 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 10 [SIMPLE_EDGE] vectorized
-                                      SHUFFLE [RS_119]
-                                        PartitionCols:_col0
-                                        Select Operator [SEL_118] (rows=286549727 width=119)
-                                          Output:["_col0","_col1","_col2"]
-                                          Filter Operator [FIL_117] (rows=286549727 width=119)
-                                            predicate:((cs_item_sk BETWEEN DynamicValue(RS_24_item_i_item_sk_min) AND DynamicValue(RS_24_item_i_item_sk_max) and in_bloom_filter(cs_item_sk, DynamicValue(RS_24_item_i_item_sk_bloom_filter))) and cs_sold_date_sk is not null)
-                                            TableScan [TS_6] (rows=287989836 width=119)
-                                              default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_item_sk","cs_ext_discount_amt"]
-                                            <-Reducer 12 [BROADCAST_EDGE] vectorized
-                                              BROADCAST [RS_116]
-                                                Group By Operator [GBY_115] (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
-                                                  SHUFFLE [RS_114]
-                                                    Group By Operator [GBY_113] (rows=1 width=12)
-                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                      Select Operator [SEL_112] (rows=669 width=4)
-                                                        Output:["_col0"]
-                                                         Please refer to the previous Select Operator [SEL_110]
-                    <-Reducer 2 [SIMPLE_EDGE]
-                      SHUFFLE [RS_30]
-                        PartitionCols:_col1
-                        Merge Join Operator [MERGEJOIN_101] (rows=31836679 width=110)
-                          Conds:RS_127._col0=RS_107._col0(Inner),Output:["_col1","_col2"]
-                        <-Map 5 [SIMPLE_EDGE] vectorized
-                          SHUFFLE [RS_107]
-                            PartitionCols:_col0
-                             Please refer to the previous Select Operator [SEL_106]
-                        <-Map 1 [SIMPLE_EDGE] vectorized
-                          SHUFFLE [RS_127]
-                            PartitionCols:_col0
-                            Select Operator [SEL_126] (rows=286549727 width=119)
-                              Output:["_col0","_col1","_col2"]
-                              Filter Operator [FIL_125] (rows=286549727 width=119)
-                                predicate:((cs_item_sk BETWEEN DynamicValue(RS_31_item_i_item_sk_min) AND DynamicValue(RS_31_item_i_item_sk_max) and in_bloom_filter(cs_item_sk, DynamicValue(RS_31_item_i_item_sk_bloom_filter))) and cs_sold_date_sk is not null)
-                                TableScan [TS_0] (rows=287989836 width=119)
-                                  default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_item_sk","cs_ext_discount_amt"]
-                                <-Reducer 9 [BROADCAST_EDGE] vectorized
-                                  BROADCAST [RS_124]
-                                    Group By Operator [GBY_123] (rows=1 width=12)
-                                      Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                    <-Reducer 8 [CUSTOM_SIMPLE_EDGE]
-                                      PARTITION_ONLY_SHUFFLE [RS_65]
-                                        Group By Operator [GBY_64] (rows=1 width=12)
-                                          Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                          Select Operator [SEL_63] (rows=97 width=8)
-                                            Output:["_col0"]
-                                             Please refer to the previous Merge Join Operator [MERGEJOIN_103]
+                                           Please refer to the previous Merge Join Operator [MERGEJOIN_102]
 
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 bc22cfb..1a09fc4 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
@@ -77,185 +77,183 @@ Reducer 9 <- Map 8 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
       Reducer 7 vectorized
-      File Output Operator [FS_232]
-        Limit [LIM_231] (rows=1 width=8)
-          Number of rows:100
-          Group By Operator [GBY_230] (rows=1 width=8)
-            Output:["_col0"],aggregations:["count(VALUE._col0)"]
-          <-Reducer 6 [CUSTOM_SIMPLE_EDGE] vectorized
-            PARTITION_ONLY_SHUFFLE [RS_229]
-              Group By Operator [GBY_228] (rows=1 width=8)
-                Output:["_col0"],aggregations:["count()"]
-                Select Operator [SEL_227] (rows=1 width=8)
-                  Filter Operator [FIL_226] (rows=1 width=8)
-                    predicate:(_col3 = 3L)
-                    Select Operator [SEL_225] (rows=165330890 width=8)
-                      Output:["_col3"]
-                      Group By Operator [GBY_224] (rows=165330890 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_242]
-                            PartitionCols:_col0, _col1, _col2
-                            Group By Operator [GBY_241] (rows=165330890 width=282)
-                              Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                              Group By Operator [GBY_240] (rows=49146883 width=282)
-                                Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
-                                Select Operator [SEL_239] (rows=49146883 width=274)
-                                  Output:["_col0","_col1","_col2"]
-                                  Group By Operator [GBY_238] (rows=49146883 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)
-                                        Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
-                                        Merge Join Operator [MERGEJOIN_175] (rows=49146883 width=274)
-                                          Conds:RS_35._col1=RS_217._col0(Inner),Output:["_col3","_col5","_col6"]
-                                        <-Map 18 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_217]
-                                            PartitionCols:_col0
-                                            Select Operator [SEL_215] (rows=80000000 width=184)
-                                              Output:["_col0","_col1","_col2"]
-                                              TableScan [TS_6] (rows=80000000 width=184)
-                                                default@customer,customer,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_first_name","c_last_name"]
-                                        <-Reducer 10 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_35]
-                                            PartitionCols:_col1
-                                            Merge Join Operator [MERGEJOIN_174] (rows=49146883 width=97)
-                                              Conds:RS_237._col0=RS_200._col0(Inner),Output:["_col1","_col3"]
-                                            <-Map 8 [SIMPLE_EDGE] vectorized
-                                              PARTITION_ONLY_SHUFFLE [RS_200]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_197] (rows=317 width=98)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_196] (rows=317 width=102)
-                                                    predicate:d_month_seq BETWEEN 1212 AND 1223
-                                                    TableScan [TS_3] (rows=73049 width=102)
-                                                      default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_date","d_month_seq"]
-                                            <-Map 19 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_237]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_236] (rows=285117831 width=7)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_235] (rows=285117831 width=7)
-                                                    predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_33_date_dim_d_date_sk_min) AND DynamicValue(RS_33_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_33_date_dim_d_date_sk_bloom_filter))) and cs_bill_customer_sk is not null and cs_sold_date_sk is not null)
-                                                    TableScan [TS_24] (rows=287989836 width=7)
-                                                      default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_bill_customer_sk"]
-                                                    <-Reducer 13 [BROADCAST_EDGE] vectorized
-                                                      BROADCAST [RS_234]
-                                                        Group By Operator [GBY_233] (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_208]
-                                                            Group By Operator [GBY_205] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                              Select Operator [SEL_201] (rows=317 width=4)
-                                                                Output:["_col0"]
-                                                                 Please refer to the previous Select Operator [SEL_197]
-                        <-Reducer 16 [CONTAINS] vectorized
-                          Reduce Output Operator [RS_252]
-                            PartitionCols:_col0, _col1, _col2
-                            Group By Operator [GBY_251] (rows=165330890 width=282)
-                              Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                              Group By Operator [GBY_250] (rows=24986582 width=282)
-                                Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
-                                Select Operator [SEL_249] (rows=24986582 width=274)
-                                  Output:["_col0","_col1","_col2"]
-                                  Group By Operator [GBY_248] (rows=24986582 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)
-                                        Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
-                                        Merge Join Operator [MERGEJOIN_177] (rows=24986582 width=274)
-                                          Conds:RS_60._col1=RS_218._col0(Inner),Output:["_col3","_col5","_col6"]
-                                        <-Map 18 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_218]
-                                            PartitionCols:_col0
-                                             Please refer to the previous Select Operator [SEL_215]
-                                        <-Reducer 14 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_60]
-                                            PartitionCols:_col1
-                                            Merge Join Operator [MERGEJOIN_176] (rows=24986582 width=97)
-                                              Conds:RS_247._col0=RS_202._col0(Inner),Output:["_col1","_col3"]
-                                            <-Map 8 [SIMPLE_EDGE] vectorized
-                                              PARTITION_ONLY_SHUFFLE [RS_202]
-                                                PartitionCols:_col0
-                                                 Please refer to the previous Select Operator [SEL_197]
-                                            <-Map 20 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_247]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_246] (rows=143930993 width=7)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_245] (rows=143930993 width=7)
-                                                    predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_58_date_dim_d_date_sk_min) AND DynamicValue(RS_58_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_58_date_dim_d_date_sk_bloom_filter))) and ws_bill_customer_sk is not null and ws_sold_date_sk is not null)
-                                                    TableScan [TS_49] (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_244]
-                                                        Group By Operator [GBY_243] (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_209]
-                                                            Group By Operator [GBY_206] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                              Select Operator [SEL_203] (rows=317 width=4)
-                                                                Output:["_col0"]
-                                                                 Please refer to the previous Select Operator [SEL_197]
-                        <-Reducer 4 [CONTAINS] vectorized
-                          Reduce Output Operator [RS_223]
-                            PartitionCols:_col0, _col1, _col2
-                            Group By Operator [GBY_222] (rows=165330890 width=282)
-                              Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
-                              Group By Operator [GBY_221] (rows=91197425 width=282)
-                                Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
-                                Select Operator [SEL_220] (rows=91197425 width=274)
-                                  Output:["_col0","_col1","_col2"]
-                                  Group By Operator [GBY_219] (rows=91197425 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)
-                                        Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
-                                        Merge Join Operator [MERGEJOIN_173] (rows=91197425 width=274)
-                                          Conds:RS_11._col1=RS_216._col0(Inner),Output:["_col3","_col5","_col6"]
-                                        <-Map 18 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_216]
-                                            PartitionCols:_col0
-                                             Please refer to the previous Select Operator [SEL_215]
-                                        <-Reducer 2 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_11]
-                                            PartitionCols:_col1
-                                            Merge Join Operator [MERGEJOIN_172] (rows=91197425 width=96)
-                                              Conds:RS_214._col0=RS_198._col0(Inner),Output:["_col1","_col3"]
-                                            <-Map 8 [SIMPLE_EDGE] vectorized
-                                              PARTITION_ONLY_SHUFFLE [RS_198]
-                                                PartitionCols:_col0
-                                                 Please refer to the previous Select Operator [SEL_197]
-                                            <-Map 1 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_214]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_213] (rows=525327388 width=7)
-                                                  Output:["_col0","_col1"]
-                                                  Filter Operator [FIL_212] (rows=525327388 width=7)
-                                                    predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_9_date_dim_d_date_sk_min) AND DynamicValue(RS_9_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_9_date_dim_d_date_sk_bloom_filter))) and ss_customer_sk is not null and ss_sold_date_sk is not null)
-                                                    TableScan [TS_0] (rows=575995635 width=7)
-                                                      default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_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)"]
-                                                        <-Map 8 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                          PARTITION_ONLY_SHUFFLE [RS_207]
-                                                            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_199] (rows=317 width=4)
-                                                                Output:["_col0"]
-                                                                 Please refer to the previous Select Operator [SEL_197]
+      File Output Operator [FS_230]
+        Group By Operator [GBY_229] (rows=1 width=8)
+          Output:["_col0"],aggregations:["count(VALUE._col0)"]
+        <-Reducer 6 [CUSTOM_SIMPLE_EDGE] vectorized
+          PARTITION_ONLY_SHUFFLE [RS_228]
+            Group By Operator [GBY_227] (rows=1 width=8)
+              Output:["_col0"],aggregations:["count()"]
+              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)
+                    Output:["_col3"]
+                    Group By Operator [GBY_223] (rows=165330890 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)
+                            Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                            Group By Operator [GBY_238] (rows=49146883 width=282)
+                              Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
+                              Select Operator [SEL_237] (rows=49146883 width=274)
+                                Output:["_col0","_col1","_col2"]
+                                Group By Operator [GBY_236] (rows=49146883 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)
+                                      Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
+                                      Merge Join Operator [MERGEJOIN_174] (rows=49146883 width=274)
+                                        Conds:RS_35._col1=RS_216._col0(Inner),Output:["_col3","_col5","_col6"]
+                                      <-Map 18 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_216]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_214] (rows=80000000 width=184)
+                                            Output:["_col0","_col1","_col2"]
+                                            TableScan [TS_6] (rows=80000000 width=184)
+                                              default@customer,customer,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_first_name","c_last_name"]
+                                      <-Reducer 10 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_35]
+                                          PartitionCols:_col1
+                                          Merge Join Operator [MERGEJOIN_173] (rows=49146883 width=97)
+                                            Conds:RS_235._col0=RS_199._col0(Inner),Output:["_col1","_col3"]
+                                          <-Map 8 [SIMPLE_EDGE] vectorized
+                                            PARTITION_ONLY_SHUFFLE [RS_199]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_196] (rows=317 width=98)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_195] (rows=317 width=102)
+                                                  predicate:d_month_seq BETWEEN 1212 AND 1223
+                                                  TableScan [TS_3] (rows=73049 width=102)
+                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_date","d_month_seq"]
+                                          <-Map 19 [SIMPLE_EDGE] vectorized
+                                            SHUFFLE [RS_235]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_234] (rows=285117831 width=7)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_233] (rows=285117831 width=7)
+                                                  predicate:((cs_sold_date_sk BETWEEN DynamicValue(RS_33_date_dim_d_date_sk_min) AND DynamicValue(RS_33_date_dim_d_date_sk_max) and in_bloom_filter(cs_sold_date_sk, DynamicValue(RS_33_date_dim_d_date_sk_bloom_filter))) and cs_bill_customer_sk is not null and cs_sold_date_sk is not null)
+                                                  TableScan [TS_24] (rows=287989836 width=7)
+                                                    default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_sold_date_sk","cs_bill_customer_sk"]
+                                                  <-Reducer 13 [BROADCAST_EDGE] vectorized
+                                                    BROADCAST [RS_232]
+                                                      Group By Operator [GBY_231] (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_207]
+                                                          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_200] (rows=317 width=4)
+                                                              Output:["_col0"]
+                                                               Please refer to the previous Select Operator [SEL_196]
+                      <-Reducer 16 [CONTAINS] vectorized
+                        Reduce Output Operator [RS_250]
+                          PartitionCols:_col0, _col1, _col2
+                          Group By Operator [GBY_249] (rows=165330890 width=282)
+                            Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                            Group By Operator [GBY_248] (rows=24986582 width=282)
+                              Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
+                              Select Operator [SEL_247] (rows=24986582 width=274)
+                                Output:["_col0","_col1","_col2"]
+                                Group By Operator [GBY_246] (rows=24986582 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)
+                                      Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
+                                      Merge Join Operator [MERGEJOIN_176] (rows=24986582 width=274)
+                                        Conds:RS_60._col1=RS_217._col0(Inner),Output:["_col3","_col5","_col6"]
+                                      <-Map 18 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_217]
+                                          PartitionCols:_col0
+                                           Please refer to the previous Select Operator [SEL_214]
+                                      <-Reducer 14 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_60]
+                                          PartitionCols:_col1
+                                          Merge Join Operator [MERGEJOIN_175] (rows=24986582 width=97)
+                                            Conds:RS_245._col0=RS_201._col0(Inner),Output:["_col1","_col3"]
+                                          <-Map 8 [SIMPLE_EDGE] vectorized
+                                            PARTITION_ONLY_SHUFFLE [RS_201]
+                                              PartitionCols:_col0
+                                               Please refer to the previous Select Operator [SEL_196]
+                                          <-Map 20 [SIMPLE_EDGE] vectorized
+                                            SHUFFLE [RS_245]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_244] (rows=143930993 width=7)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_243] (rows=143930993 width=7)
+                                                  predicate:((ws_sold_date_sk BETWEEN DynamicValue(RS_58_date_dim_d_date_sk_min) AND DynamicValue(RS_58_date_dim_d_date_sk_max) and in_bloom_filter(ws_sold_date_sk, DynamicValue(RS_58_date_dim_d_date_sk_bloom_filter))) and ws_bill_customer_sk is not null and ws_sold_date_sk is not null)
+                                                  TableScan [TS_49] (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_242]
+                                                      Group By Operator [GBY_241] (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_208]
+                                                          Group By Operator [GBY_205] (rows=1 width=12)
+                                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                            Select Operator [SEL_202] (rows=317 width=4)
+                                                              Output:["_col0"]
+                                                               Please refer to the previous Select Operator [SEL_196]
+                      <-Reducer 4 [CONTAINS] vectorized
+                        Reduce Output Operator [RS_222]
+                          PartitionCols:_col0, _col1, _col2
+                          Group By Operator [GBY_221] (rows=165330890 width=282)
+                            Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2
+                            Group By Operator [GBY_220] (rows=91197425 width=282)
+                              Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col1, _col0, _col2
+                              Select Operator [SEL_219] (rows=91197425 width=274)
+                                Output:["_col0","_col1","_col2"]
+                                Group By Operator [GBY_218] (rows=91197425 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)
+                                      Output:["_col0","_col1","_col2"],keys:_col6, _col5, _col3
+                                      Merge Join Operator [MERGEJOIN_172] (rows=91197425 width=274)
+                                        Conds:RS_11._col1=RS_215._col0(Inner),Output:["_col3","_col5","_col6"]
+                                      <-Map 18 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_215]
+                                          PartitionCols:_col0
+                                           Please refer to the previous Select Operator [SEL_214]
+                                      <-Reducer 2 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_11]
+                                          PartitionCols:_col1
+                                          Merge Join Operator [MERGEJOIN_171] (rows=91197425 width=96)
+                                            Conds:RS_213._col0=RS_197._col0(Inner),Output:["_col1","_col3"]
+                                          <-Map 8 [SIMPLE_EDGE] vectorized
+                                            PARTITION_ONLY_SHUFFLE [RS_197]
+                                              PartitionCols:_col0
+                                               Please refer to the previous Select Operator [SEL_196]
+                                          <-Map 1 [SIMPLE_EDGE] vectorized
+                                            SHUFFLE [RS_213]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_212] (rows=525327388 width=7)
+                                                Output:["_col0","_col1"]
+                                                Filter Operator [FIL_211] (rows=525327388 width=7)
+                                                  predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_9_date_dim_d_date_sk_min) AND DynamicValue(RS_9_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_9_date_dim_d_date_sk_bloom_filter))) and ss_customer_sk is not null and ss_sold_date_sk is not null)
+                                                  TableScan [TS_0] (rows=575995635 width=7)
+                                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_customer_sk"]
+                                                  <-Reducer 9 [BROADCAST_EDGE] vectorized
+                                                    BROADCAST [RS_210]
+                                                      Group By Operator [GBY_209] (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_206]
+                                                          Group By Operator [GBY_203] (rows=1 width=12)
+                                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                            Select Operator [SEL_198] (rows=317 width=4)
+                                                              Output:["_col0"]
+                                                               Please refer to the previous Select Operator [SEL_196]
 
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 5625306..4c4851c 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
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[266][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[263][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain
 select  promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100
 from
@@ -104,209 +104,202 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 10 <- Reducer 16 (BROADCAST_EDGE)
-Map 24 <- Reducer 20 (BROADCAST_EDGE)
-Reducer 11 <- Map 10 (SIMPLE_EDGE), Map 15 (SIMPLE_EDGE)
+Map 23 <- Reducer 19 (BROADCAST_EDGE)
+Map 9 <- Reducer 15 (BROADCAST_EDGE)
+Reducer 10 <- Map 14 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
+Reducer 11 <- Map 20 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE)
 Reducer 12 <- Map 21 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE)
 Reducer 13 <- Map 22 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE)
-Reducer 14 <- Map 23 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
-Reducer 16 <- Map 15 (CUSTOM_SIMPLE_EDGE)
-Reducer 17 <- Map 15 (SIMPLE_EDGE), Map 24 (SIMPLE_EDGE)
+Reducer 15 <- Map 14 (CUSTOM_SIMPLE_EDGE)
+Reducer 16 <- Map 14 (SIMPLE_EDGE), Map 23 (SIMPLE_EDGE)
+Reducer 17 <- Map 20 (SIMPLE_EDGE), Reducer 16 (SIMPLE_EDGE)
 Reducer 18 <- Map 21 (SIMPLE_EDGE), Reducer 17 (SIMPLE_EDGE)
-Reducer 19 <- Map 22 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE)
-Reducer 20 <- Map 15 (CUSTOM_SIMPLE_EDGE)
-Reducer 3 <- Reducer 14 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 19 <- Map 14 (CUSTOM_SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 13 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE)
-Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
-Reducer 7 <- Reducer 19 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE), Reducer 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 6 <- Reducer 18 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
-      Reducer 6 vectorized
-      File Output Operator [FS_310]
-        Limit [LIM_309] (rows=1 width=336)
-          Number of rows:100
-          Select Operator [SEL_308] (rows=1 width=336)
-            Output:["_col0","_col1","_col2"]
-          <-Reducer 5 [SIMPLE_EDGE]
-            SHUFFLE [RS_88]
-              Select Operator [SEL_87] (rows=1 width=336)
-                Output:["_col0","_col1","_col2"]
-                Merge Join Operator [MERGEJOIN_266] (rows=1 width=224)
-                  Conds:(Inner),Output:["_col0","_col1"]
-                <-Reducer 4 [CUSTOM_SIMPLE_EDGE] vectorized
-                  PARTITION_ONLY_SHUFFLE [RS_300]
-                    Group By Operator [GBY_299] (rows=1 width=112)
-                      Output:["_col0"],aggregations:["sum(VALUE._col0)"]
-                    <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
-                      PARTITION_ONLY_SHUFFLE [RS_42]
-                        Group By Operator [GBY_41] (rows=1 width=112)
-                          Output:["_col0"],aggregations:["sum(_col8)"]
-                          Merge Join Operator [MERGEJOIN_264] (rows=505397 width=0)
-                            Conds:RS_37._col0=RS_38._col2(Inner),Output:["_col8"]
-                          <-Reducer 2 [SIMPLE_EDGE]
-                            SHUFFLE [RS_37]
-                              PartitionCols:_col0
-                              Merge Join Operator [MERGEJOIN_255] (rows=16000001 width=4)
-                                Conds:RS_269._col1=RS_272._col0(Inner),Output:["_col0"]
-                              <-Map 1 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_269]
-                                  PartitionCols:_col1
-                                  Select Operator [SEL_268] (rows=80000000 width=8)
-                                    Output:["_col0","_col1"]
-                                    Filter Operator [FIL_267] (rows=80000000 width=8)
-                                      predicate:c_current_addr_sk is not null
-                                      TableScan [TS_0] (rows=80000000 width=8)
-                                        default@customer,customer,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_current_addr_sk"]
-                              <-Map 9 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_272]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_271] (rows=8000000 width=4)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_270] (rows=8000000 width=112)
-                                      predicate:(ca_gmt_offset = -7)
-                                      TableScan [TS_3] (rows=40000000 width=112)
-                                        default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_gmt_offset"]
-                          <-Reducer 14 [SIMPLE_EDGE]
-                            SHUFFLE [RS_38]
-                              PartitionCols:_col2
-                              Merge Join Operator [MERGEJOIN_259] (rows=2526982 width=0)
-                                Conds:RS_30._col4=RS_298._col0(Inner),Output:["_col2","_col5"]
-                              <-Map 23 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_298]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_297] (rows=2300 width=4)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_296] (rows=2300 width=259)
-                                      predicate:((p_channel_dmail = 'Y') or (p_channel_email = 'Y') or (p_channel_tv = 'Y'))
-                                      TableScan [TS_18] (rows=2300 width=259)
-                                        default@promotion,promotion,Tbl:COMPLETE,Col:COMPLETE,Output:["p_promo_sk","p_channel_dmail","p_channel_email","p_channel_tv"]
-                              <-Reducer 13 [SIMPLE_EDGE]
-                                SHUFFLE [RS_30]
-                                  PartitionCols:_col4
-                                  Merge Join Operator [MERGEJOIN_258] (rows=2526982 width=0)
-                                    Conds:RS_27._col3=RS_294._col0(Inner),Output:["_col2","_col4","_col5"]
-                                  <-Map 22 [SIMPLE_EDGE] vectorized
-                                    SHUFFLE [RS_294]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_293] (rows=341 width=4)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_292] (rows=341 width=115)
-                                          predicate:(s_gmt_offset = -7)
-                                          TableScan [TS_15] (rows=1704 width=115)
-                                            default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_store_sk","s_gmt_offset"]
-                                  <-Reducer 12 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_27]
-                                      PartitionCols:_col3
-                                      Merge Join Operator [MERGEJOIN_257] (rows=12627499 width=0)
-                                        Conds:RS_24._col1=RS_290._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
-                                      <-Map 21 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_290]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_289] (rows=46200 width=4)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_288] (rows=46200 width=94)
-                                              predicate:(i_category = 'Electronics')
-                                              TableScan [TS_12] (rows=462000 width=94)
-                                                default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_category"]
-                                      <-Reducer 11 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_24]
-                                          PartitionCols:_col1
-                                          Merge Join Operator [MERGEJOIN_256] (rows=13119234 width=4)
-                                            Conds:RS_287._col0=RS_275._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
-                                          <-Map 15 [SIMPLE_EDGE] vectorized
-                                            PARTITION_ONLY_SHUFFLE [RS_275]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_274] (rows=50 width=4)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_273] (rows=50 width=12)
-                                                  predicate:((d_moy = 11) and (d_year = 1999))
-                                                  TableScan [TS_9] (rows=73049 width=12)
-                                                    default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
-                                          <-Map 10 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_287]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_286] (rows=479120970 width=126)
-                                                Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                                Filter Operator [FIL_285] (rows=479120970 width=126)
-                                                  predicate:((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))) and ss_customer_sk is not null and ss_promo_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null)
-                                                  TableScan [TS_6] (rows=575995635 width=126)
-                                                    default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_store_sk","ss_promo_sk","ss_ext_sales_price"]
-                                                  <-Reducer 16 [BROADCAST_EDGE] vectorized
-                                                    BROADCAST [RS_284]
-                                                      Group By Operator [GBY_283] (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
-                                                        PARTITION_ONLY_SHUFFLE [RS_281]
-                                                          Group By Operator [GBY_279] (rows=1 width=12)
-                                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                            Select Operator [SEL_276] (rows=50 width=4)
-                                                              Output:["_col0"]
-                                                               Please refer to the previous Select Operator [SEL_274]
-                <-Reducer 8 [CUSTOM_SIMPLE_EDGE] vectorized
-                  PARTITION_ONLY_SHUFFLE [RS_307]
-                    Group By Operator [GBY_306] (rows=1 width=112)
-                      Output:["_col0"],aggregations:["sum(VALUE._col0)"]
-                    <-Reducer 7 [CUSTOM_SIMPLE_EDGE]
-                      PARTITION_ONLY_SHUFFLE [RS_81]
-                        Group By Operator [GBY_80] (rows=1 width=112)
-                          Output:["_col0"],aggregations:["sum(_col7)"]
-                          Merge Join Operator [MERGEJOIN_265] (rows=529208 width=0)
-                            Conds:RS_76._col0=RS_77._col2(Inner),Output:["_col7"]
-                          <-Reducer 2 [SIMPLE_EDGE]
-                            SHUFFLE [RS_76]
-                              PartitionCols:_col0
-                               Please refer to the previous Merge Join Operator [MERGEJOIN_255]
-                          <-Reducer 19 [SIMPLE_EDGE]
-                            SHUFFLE [RS_77]
-                              PartitionCols:_col2
-                              Merge Join Operator [MERGEJOIN_263] (rows=2646038 width=0)
-                                Conds:RS_69._col3=RS_295._col0(Inner),Output:["_col2","_col4"]
-                              <-Map 22 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_295]
-                                  PartitionCols:_col0
-                                   Please refer to the previous Select Operator [SEL_293]
-                              <-Reducer 18 [SIMPLE_EDGE]
-                                SHUFFLE [RS_69]
-                                  PartitionCols:_col3
-                                  Merge Join Operator [MERGEJOIN_262] (rows=13222427 width=0)
-                                    Conds:RS_66._col1=RS_291._col0(Inner),Output:["_col2","_col3","_col4"]
-                                  <-Map 21 [SIMPLE_EDGE] vectorized
-                                    SHUFFLE [RS_291]
-                                      PartitionCols:_col0
-                                       Please refer to the previous Select Operator [SEL_289]
-                                  <-Reducer 17 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_66]
-                                      PartitionCols:_col1
-                                      Merge Join Operator [MERGEJOIN_261] (rows=13737330 width=4)
-                                        Conds:RS_305._col0=RS_277._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
-                                      <-Map 15 [SIMPLE_EDGE] vectorized
-                                        PARTITION_ONLY_SHUFFLE [RS_277]
-                                          PartitionCols:_col0
-                                           Please refer to the previous Select Operator [SEL_274]
-                                      <-Map 24 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_305]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_304] (rows=501694138 width=122)
-                                            Output:["_col0","_col1","_col2","_col3","_col4"]
-                                            Filter Operator [FIL_303] (rows=501694138 width=122)
-                                              predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_64_date_dim_d_date_sk_min) AND DynamicValue(RS_64_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_64_date_dim_d_date_sk_bloom_filter))) and ss_customer_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null)
-                                              TableScan [TS_51] (rows=575995635 width=122)
-                                                default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_store_sk","ss_ext_sales_price"]
-                                              <-Reducer 20 [BROADCAST_EDGE] vectorized
-                                                BROADCAST [RS_302]
-                                                  Group By Operator [GBY_301] (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
-                                                    PARTITION_ONLY_SHUFFLE [RS_282]
-                                                      Group By Operator [GBY_280] (rows=1 width=12)
-                                                        Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                        Select Operator [SEL_278] (rows=50 width=4)
-                                                          Output:["_col0"]
-                                                           Please refer to the previous Select Operator [SEL_274]
+      Reducer 5
+      File Output Operator [FS_88]
+        Select Operator [SEL_87] (rows=1 width=336)
+          Output:["_col0","_col1","_col2"]
+          Merge Join Operator [MERGEJOIN_263] (rows=1 width=224)
+            Conds:(Inner),Output:["_col0","_col1"]
+          <-Reducer 4 [CUSTOM_SIMPLE_EDGE] vectorized
+            PARTITION_ONLY_SHUFFLE [RS_297]
+              Group By Operator [GBY_296] (rows=1 width=112)
+                Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+              <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
+                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)
+                      Conds:RS_37._col0=RS_38._col2(Inner),Output:["_col8"]
+                    <-Reducer 2 [SIMPLE_EDGE]
+                      SHUFFLE [RS_37]
+                        PartitionCols:_col0
+                        Merge Join Operator [MERGEJOIN_252] (rows=16000001 width=4)
+                          Conds:RS_266._col1=RS_269._col0(Inner),Output:["_col0"]
+                        <-Map 1 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_266]
+                            PartitionCols:_col1
+                            Select Operator [SEL_265] (rows=80000000 width=8)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_264] (rows=80000000 width=8)
+                                predicate:c_current_addr_sk is not null
+                                TableScan [TS_0] (rows=80000000 width=8)
+                                  default@customer,customer,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_current_addr_sk"]
+                        <-Map 8 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_269]
+                            PartitionCols:_col0
+                            Select Operator [SEL_268] (rows=8000000 width=4)
+                              Output:["_col0"]
+                              Filter Operator [FIL_267] (rows=8000000 width=112)
+                                predicate:(ca_gmt_offset = -7)
+                                TableScan [TS_3] (rows=40000000 width=112)
+                                  default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_gmt_offset"]
+                    <-Reducer 13 [SIMPLE_EDGE]
+                      SHUFFLE [RS_38]
+                        PartitionCols:_col2
+                        Merge Join Operator [MERGEJOIN_256] (rows=2526982 width=0)
+                          Conds:RS_30._col4=RS_295._col0(Inner),Output:["_col2","_col5"]
+                        <-Map 22 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_295]
+                            PartitionCols:_col0
+                            Select Operator [SEL_294] (rows=2300 width=4)
+                              Output:["_col0"]
+                              Filter Operator [FIL_293] (rows=2300 width=259)
+                                predicate:((p_channel_dmail = 'Y') or (p_channel_email = 'Y') or (p_channel_tv = 'Y'))
+                                TableScan [TS_18] (rows=2300 width=259)
+                                  default@promotion,promotion,Tbl:COMPLETE,Col:COMPLETE,Output:["p_promo_sk","p_channel_dmail","p_channel_email","p_channel_tv"]
+                        <-Reducer 12 [SIMPLE_EDGE]
+                          SHUFFLE [RS_30]
+                            PartitionCols:_col4
+                            Merge Join Operator [MERGEJOIN_255] (rows=2526982 width=0)
+                              Conds:RS_27._col3=RS_291._col0(Inner),Output:["_col2","_col4","_col5"]
+                            <-Map 21 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_291]
+                                PartitionCols:_col0
+                                Select Operator [SEL_290] (rows=341 width=4)
+                                  Output:["_col0"]
+                                  Filter Operator [FIL_289] (rows=341 width=115)
+                                    predicate:(s_gmt_offset = -7)
+                                    TableScan [TS_15] (rows=1704 width=115)
+                                      default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_store_sk","s_gmt_offset"]
+                            <-Reducer 11 [SIMPLE_EDGE]
+                              SHUFFLE [RS_27]
+                                PartitionCols:_col3
+                                Merge Join Operator [MERGEJOIN_254] (rows=12627499 width=0)
+                                  Conds:RS_24._col1=RS_287._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
+                                <-Map 20 [SIMPLE_EDGE] vectorized
+                                  SHUFFLE [RS_287]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_286] (rows=46200 width=4)
+                                      Output:["_col0"]
+                                      Filter Operator [FIL_285] (rows=46200 width=94)
+                                        predicate:(i_category = 'Electronics')
+                                        TableScan [TS_12] (rows=462000 width=94)
+                                          default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_category"]
+                                <-Reducer 10 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_24]
+                                    PartitionCols:_col1
+                                    Merge Join Operator [MERGEJOIN_253] (rows=13119234 width=4)
+                                      Conds:RS_284._col0=RS_272._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
+                                    <-Map 14 [SIMPLE_EDGE] vectorized
+                                      PARTITION_ONLY_SHUFFLE [RS_272]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_271] (rows=50 width=4)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_270] (rows=50 width=12)
+                                            predicate:((d_moy = 11) and (d_year = 1999))
+                                            TableScan [TS_9] (rows=73049 width=12)
+                                              default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
+                                    <-Map 9 [SIMPLE_EDGE] vectorized
+                                      SHUFFLE [RS_284]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_283] (rows=479120970 width=126)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                          Filter Operator [FIL_282] (rows=479120970 width=126)
+                                            predicate:((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))) and ss_customer_sk is not null and ss_promo_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null)
+                                            TableScan [TS_6] (rows=575995635 width=126)
+                                              default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_store_sk","ss_promo_sk","ss_ext_sales_price"]
+                                            <-Reducer 15 [BROADCAST_EDGE] vectorized
+                                              BROADCAST [RS_281]
+                                                Group By Operator [GBY_280] (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
+                                                  PARTITION_ONLY_SHUFFLE [RS_278]
+                                                    Group By Operator [GBY_276] (rows=1 width=12)
+                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                      Select Operator [SEL_273] (rows=50 width=4)
+                                                        Output:["_col0"]
+                                                         Please refer to the previous Select Operator [SEL_271]
+          <-Reducer 7 [CUSTOM_SIMPLE_EDGE] vectorized
+            PARTITION_ONLY_SHUFFLE [RS_304]
+              Group By Operator [GBY_303] (rows=1 width=112)
+                Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+              <-Reducer 6 [CUSTOM_SIMPLE_EDGE]
+                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)
+                      Conds:RS_76._col0=RS_77._col2(Inner),Output:["_col7"]
+                    <-Reducer 2 [SIMPLE_EDGE]
+                      SHUFFLE [RS_76]
+                        PartitionCols:_col0
+                         Please refer to the previous Merge Join Operator [MERGEJOIN_252]
+                    <-Reducer 18 [SIMPLE_EDGE]
+                      SHUFFLE [RS_77]
+                        PartitionCols:_col2
+                        Merge Join Operator [MERGEJOIN_260] (rows=2646038 width=0)
+                          Conds:RS_69._col3=RS_292._col0(Inner),Output:["_col2","_col4"]
+                        <-Map 21 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_292]
+                            PartitionCols:_col0
+                             Please refer to the previous Select Operator [SEL_290]
+                        <-Reducer 17 [SIMPLE_EDGE]
+                          SHUFFLE [RS_69]
+                            PartitionCols:_col3
+                            Merge Join Operator [MERGEJOIN_259] (rows=13222427 width=0)
+                              Conds:RS_66._col1=RS_288._col0(Inner),Output:["_col2","_col3","_col4"]
+                            <-Map 20 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_288]
+                                PartitionCols:_col0
+                                 Please refer to the previous Select Operator [SEL_286]
+                            <-Reducer 16 [SIMPLE_EDGE]
+                              SHUFFLE [RS_66]
+                                PartitionCols:_col1
+                                Merge Join Operator [MERGEJOIN_258] (rows=13737330 width=4)
+                                  Conds:RS_302._col0=RS_274._col0(Inner),Output:["_col1","_col2","_col3","_col4"]
+                                <-Map 14 [SIMPLE_EDGE] vectorized
+                                  PARTITION_ONLY_SHUFFLE [RS_274]
+                                    PartitionCols:_col0
+                                     Please refer to the previous Select Operator [SEL_271]
+                                <-Map 23 [SIMPLE_EDGE] vectorized
+                                  SHUFFLE [RS_302]
+                                    PartitionCols:_col0
+                                    Select Operator [SEL_301] (rows=501694138 width=122)
+                                      Output:["_col0","_col1","_col2","_col3","_col4"]
+                                      Filter Operator [FIL_300] (rows=501694138 width=122)
+                                        predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_64_date_dim_d_date_sk_min) AND DynamicValue(RS_64_date_dim_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_64_date_dim_d_date_sk_bloom_filter))) and ss_customer_sk is not null and ss_sold_date_sk is not null and ss_store_sk is not null)
+                                        TableScan [TS_51] (rows=575995635 width=122)
+                                          default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_store_sk","ss_ext_sales_price"]
+                                        <-Reducer 19 [BROADCAST_EDGE] vectorized
+                                          BROADCAST [RS_299]
+                                            Group By Operator [GBY_298] (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
+                                              PARTITION_ONLY_SHUFFLE [RS_279]
+                                                Group By Operator [GBY_277] (rows=1 width=12)
+                                                  Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                  Select Operator [SEL_275] (rows=50 width=4)
+                                                    Output:["_col0"]
+                                                     Please refer to the previous Select Operator [SEL_271]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query90.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query90.q.out
index 7be103d..33e9427 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query90.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query90.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[152][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[149][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 6' is a cross product
 PREHOOK: query: explain
 select  cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio
  from ( select count(*) amc
@@ -54,130 +54,123 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 14 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 9 (CUSTOM_SIMPLE_EDGE)
-Reducer 14 <- Map 13 (CUSTOM_SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
-Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Map 1 <- Reducer 13 (BROADCAST_EDGE)
+Reducer 13 <- Map 12 (CUSTOM_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 12 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 6 <- Reducer 10 (CUSTOM_SIMPLE_EDGE), Reducer 5 (CUSTOM_SIMPLE_EDGE)
-Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
-Reducer 8 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 9 <- Map 13 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (CUSTOM_SIMPLE_EDGE), Reducer 9 (CUSTOM_SIMPLE_EDGE)
+Reducer 7 <- Map 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 8 <- Map 12 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:100
+    limit:-1
     Stage-1
-      Reducer 7 vectorized
-      File Output Operator [FS_180]
-        Limit [LIM_179] (rows=1 width=112)
-          Number of rows:100
-          Select Operator [SEL_178] (rows=1 width=112)
-            Output:["_col0"]
-          <-Reducer 6 [SIMPLE_EDGE]
-            SHUFFLE [RS_56]
-              Select Operator [SEL_55] (rows=1 width=112)
-                Output:["_col0"]
-                Merge Join Operator [MERGEJOIN_152] (rows=1 width=16)
-                  Conds:(Inner),Output:["_col0","_col1"]
-                <-Reducer 10 [CUSTOM_SIMPLE_EDGE] vectorized
-                  PARTITION_ONLY_SHUFFLE [RS_177]
-                    Group By Operator [GBY_176] (rows=1 width=8)
-                      Output:["_col0"],aggregations:["count(VALUE._col0)"]
-                    <-Reducer 9 [CUSTOM_SIMPLE_EDGE]
-                      PARTITION_ONLY_SHUFFLE [RS_49]
-                        Group By Operator [GBY_48] (rows=1 width=8)
-                          Output:["_col0"],aggregations:["count()"]
-                          Merge Join Operator [MERGEJOIN_151] (rows=153010 width=8)
-                            Conds:RS_44._col1=RS_157._col0(Inner)
-                          <-Map 13 [SIMPLE_EDGE] vectorized
-                            SHUFFLE [RS_157]
-                              PartitionCols:_col0
-                              Select Operator [SEL_154] (rows=655 width=4)
-                                Output:["_col0"]
-                                Filter Operator [FIL_153] (rows=655 width=8)
-                                  predicate:(hd_dep_count = 8)
-                                  TableScan [TS_9] (rows=7200 width=8)
-                                    default@household_demographics,household_demographics,Tbl:COMPLETE,Col:COMPLETE,Output:["hd_demo_sk","hd_dep_count"]
-                          <-Reducer 8 [SIMPLE_EDGE]
-                            SHUFFLE [RS_44]
-                              PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_150] (rows=1681936 width=3)
-                                Conds:RS_41._col0=RS_173._col0(Inner),Output:["_col1"]
-                              <-Map 12 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_173]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_171] (rows=9095 width=4)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_169] (rows=9095 width=8)
-                                      predicate:t_hour BETWEEN 14 AND 15
-                                      TableScan [TS_6] (rows=86400 width=8)
-                                        default@time_dim,time_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["t_time_sk","t_hour"]
-                              <-Reducer 2 [SIMPLE_EDGE]
-                                SHUFFLE [RS_41]
-                                  PartitionCols:_col0
-                                  Merge Join Operator [MERGEJOIN_146] (rows=15977923 width=7)
-                                    Conds:RS_164._col2=RS_167._col0(Inner),Output:["_col0","_col1"]
-                                  <-Map 1 [SIMPLE_EDGE] vectorized
-                                    SHUFFLE [RS_164]
-                                      PartitionCols:_col2
-                                      Select Operator [SEL_163] (rows=143895111 width=11)
-                                        Output:["_col0","_col1","_col2"]
-                                        Filter Operator [FIL_162] (rows=143895111 width=11)
-                                          predicate:((ws_ship_hdemo_sk BETWEEN DynamicValue(RS_19_household_demographics_hd_demo_sk_min) AND DynamicValue(RS_19_household_demographics_hd_demo_sk_max) and in_bloom_filter(ws_ship_hdemo_sk, DynamicValue(RS_19_household_demographics_hd_demo_sk_bloom_filter))) and ws_ship_hdemo_sk is not null and ws_sold_time_sk is not null and ws_web_page_sk is not null)
-                                          TableScan [TS_0] (rows=144002668 width=11)
-                                            default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_time_sk","ws_ship_hdemo_sk","ws_web_page_sk"]
-                                          <-Reducer 14 [BROADCAST_EDGE] vectorized
-                                            BROADCAST [RS_161]
-                                              Group By Operator [GBY_160] (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
-                                                SHUFFLE [RS_159]
-                                                  Group By Operator [GBY_158] (rows=1 width=12)
-                                                    Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                    Select Operator [SEL_156] (rows=655 width=4)
-                                                      Output:["_col0"]
-                                                       Please refer to the previous Select Operator [SEL_154]
-                                  <-Map 11 [SIMPLE_EDGE] vectorized
-                                    SHUFFLE [RS_167]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_166] (rows=511 width=4)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_165] (rows=511 width=7)
-                                          predicate:wp_char_count BETWEEN 5000 AND 5200
-                                          TableScan [TS_3] (rows=4602 width=7)
-                                            default@web_page,web_page,Tbl:COMPLETE,Col:COMPLETE,Output:["wp_web_page_sk","wp_char_count"]
-                <-Reducer 5 [CUSTOM_SIMPLE_EDGE] vectorized
-                  PARTITION_ONLY_SHUFFLE [RS_175]
-                    Group By Operator [GBY_174] (rows=1 width=8)
-                      Output:["_col0"],aggregations:["count(VALUE._col0)"]
-                    <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-                      PARTITION_ONLY_SHUFFLE [RS_23]
-                        Group By Operator [GBY_22] (rows=1 width=8)
-                          Output:["_col0"],aggregations:["count()"]
-                          Merge Join Operator [MERGEJOIN_148] (rows=153010 width=8)
-                            Conds:RS_18._col1=RS_155._col0(Inner)
-                          <-Map 13 [SIMPLE_EDGE] vectorized
-                            SHUFFLE [RS_155]
-                              PartitionCols:_col0
-                               Please refer to the previous Select Operator [SEL_154]
-                          <-Reducer 3 [SIMPLE_EDGE]
-                            SHUFFLE [RS_18]
-                              PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_147] (rows=1681936 width=3)
-                                Conds:RS_15._col0=RS_172._col0(Inner),Output:["_col1"]
-                              <-Map 12 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_172]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_170] (rows=9095 width=4)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_168] (rows=9095 width=8)
-                                      predicate:t_hour BETWEEN 6 AND 7
-                                       Please refer to the previous TableScan [TS_6]
-                              <-Reducer 2 [SIMPLE_EDGE]
-                                SHUFFLE [RS_15]
-                                  PartitionCols:_col0
-                                   Please refer to the previous Merge Join Operator [MERGEJOIN_146]
+      Reducer 6
+      File Output Operator [FS_56]
+        Select Operator [SEL_55] (rows=1 width=112)
+          Output:["_col0"]
+          Merge Join Operator [MERGEJOIN_149] (rows=1 width=16)
+            Conds:(Inner),Output:["_col0","_col1"]
+          <-Reducer 5 [CUSTOM_SIMPLE_EDGE] vectorized
+            PARTITION_ONLY_SHUFFLE [RS_172]
+              Group By Operator [GBY_171] (rows=1 width=8)
+                Output:["_col0"],aggregations:["count(VALUE._col0)"]
+              <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
+                PARTITION_ONLY_SHUFFLE [RS_23]
+                  Group By Operator [GBY_22] (rows=1 width=8)
+                    Output:["_col0"],aggregations:["count()"]
+                    Merge Join Operator [MERGEJOIN_145] (rows=153010 width=8)
+                      Conds:RS_18._col1=RS_152._col0(Inner)
+                    <-Map 12 [SIMPLE_EDGE] vectorized
+                      SHUFFLE [RS_152]
+                        PartitionCols:_col0
+                        Select Operator [SEL_151] (rows=655 width=4)
+                          Output:["_col0"]
+                          Filter Operator [FIL_150] (rows=655 width=8)
+                            predicate:(hd_dep_count = 8)
+                            TableScan [TS_9] (rows=7200 width=8)
+                              default@household_demographics,household_demographics,Tbl:COMPLETE,Col:COMPLETE,Output:["hd_demo_sk","hd_dep_count"]
+                    <-Reducer 3 [SIMPLE_EDGE]
+                      SHUFFLE [RS_18]
+                        PartitionCols:_col1
+                        Merge Join Operator [MERGEJOIN_144] (rows=1681936 width=3)
+                          Conds:RS_15._col0=RS_169._col0(Inner),Output:["_col1"]
+                        <-Map 11 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_169]
+                            PartitionCols:_col0
+                            Select Operator [SEL_167] (rows=9095 width=4)
+                              Output:["_col0"]
+                              Filter Operator [FIL_165] (rows=9095 width=8)
+                                predicate:t_hour BETWEEN 6 AND 7
+                                TableScan [TS_6] (rows=86400 width=8)
+                                  default@time_dim,time_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["t_time_sk","t_hour"]
+                        <-Reducer 2 [SIMPLE_EDGE]
+                          SHUFFLE [RS_15]
+                            PartitionCols:_col0
+                            Merge Join Operator [MERGEJOIN_143] (rows=15977923 width=7)
+                              Conds:RS_161._col2=RS_164._col0(Inner),Output:["_col0","_col1"]
+                            <-Map 1 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_161]
+                                PartitionCols:_col2
+                                Select Operator [SEL_160] (rows=143895111 width=11)
+                                  Output:["_col0","_col1","_col2"]
+                                  Filter Operator [FIL_159] (rows=143895111 width=11)
+                                    predicate:((ws_ship_hdemo_sk BETWEEN DynamicValue(RS_19_household_demographics_hd_demo_sk_min) AND DynamicValue(RS_19_household_demographics_hd_demo_sk_max) and in_bloom_filter(ws_ship_hdemo_sk, DynamicValue(RS_19_household_demographics_hd_demo_sk_bloom_filter))) and ws_ship_hdemo_sk is not null and ws_sold_time_sk is not null and ws_web_page_sk is not null)
+                                    TableScan [TS_0] (rows=144002668 width=11)
+                                      default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_time_sk","ws_ship_hdemo_sk","ws_web_page_sk"]
+                                    <-Reducer 13 [BROADCAST_EDGE] vectorized
+                                      BROADCAST [RS_158]
+                                        Group By Operator [GBY_157] (rows=1 width=12)
+                                          Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                        <-Map 12 [CUSTOM_SIMPLE_EDGE] vectorized
+                                          SHUFFLE [RS_156]
+                                            Group By Operator [GBY_155] (rows=1 width=12)
+                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                              Select Operator [SEL_153] (rows=655 width=4)
+                                                Output:["_col0"]
+                                                 Please refer to the previous Select Operator [SEL_151]
+                            <-Map 10 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_164]
+                                PartitionCols:_col0
+                                Select Operator [SEL_163] (rows=511 width=4)
+                                  Output:["_col0"]
+                                  Filter Operator [FIL_162] (rows=511 width=7)
+                                    predicate:wp_char_count BETWEEN 5000 AND 5200
+                                    TableScan [TS_3] (rows=4602 width=7)
+                                      default@web_page,web_page,Tbl:COMPLETE,Col:COMPLETE,Output:["wp_web_page_sk","wp_char_count"]
+          <-Reducer 9 [CUSTOM_SIMPLE_EDGE] vectorized
+            PARTITION_ONLY_SHUFFLE [RS_174]
+              Group By Operator [GBY_173] (rows=1 width=8)
+                Output:["_col0"],aggregations:["count(VALUE._col0)"]
+              <-Reducer 8 [CUSTOM_SIMPLE_EDGE]
+                PARTITION_ONLY_SHUFFLE [RS_49]
+                  Group By Operator [GBY_48] (rows=1 width=8)
+                    Output:["_col0"],aggregations:["count()"]
+                    Merge Join Operator [MERGEJOIN_148] (rows=153010 width=8)
+                      Conds:RS_44._col1=RS_154._col0(Inner)
+                    <-Map 12 [SIMPLE_EDGE] vectorized
+                      SHUFFLE [RS_154]
+                        PartitionCols:_col0
+                         Please refer to the previous Select Operator [SEL_151]
+                    <-Reducer 7 [SIMPLE_EDGE]
+                      SHUFFLE [RS_44]
+                        PartitionCols:_col1
+                        Merge Join Operator [MERGEJOIN_147] (rows=1681936 width=3)
+                          Conds:RS_41._col0=RS_170._col0(Inner),Output:["_col1"]
+                        <-Map 11 [SIMPLE_EDGE] vectorized
+                          SHUFFLE [RS_170]
+                            PartitionCols:_col0
+                            Select Operator [SEL_168] (rows=9095 width=4)
+                              Output:["_col0"]
+                              Filter Operator [FIL_166] (rows=9095 width=8)
+                                predicate:t_hour BETWEEN 14 AND 15
+                                 Please refer to the previous TableScan [TS_6]
+                        <-Reducer 2 [SIMPLE_EDGE]
+                          SHUFFLE [RS_41]
+                            PartitionCols:_col0
+                             Please refer to the previous Merge Join Operator [MERGEJOIN_143]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query92.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query92.q.out
index 4cf1a7f..f60d789 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query92.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query92.q.out
@@ -67,128 +67,119 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 10 (BROADCAST_EDGE)
-Map 11 <- Reducer 13 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 9 (CUSTOM_SIMPLE_EDGE)
-Reducer 13 <- Map 12 (CUSTOM_SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE)
+Map 1 <- Reducer 9 (BROADCAST_EDGE)
+Map 10 <- Reducer 12 (BROADCAST_EDGE)
+Reducer 12 <- Map 11 (CUSTOM_SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE)
 Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-Reducer 7 <- Map 11 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
-Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
-Reducer 9 <- Map 12 (SIMPLE_EDGE), Reducer 8 (ONE_TO_ONE_EDGE)
+Reducer 6 <- Map 10 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+Reducer 8 <- Map 11 (SIMPLE_EDGE), Reducer 7 (ONE_TO_ONE_EDGE)
+Reducer 9 <- Reducer 8 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 5 vectorized
-      File Output Operator [FS_136]
-        Limit [LIM_135] (rows=1 width=224)
-          Number of rows:100
-          Select Operator [SEL_134] (rows=1 width=224)
-            Output:["_col0"]
-          <-Reducer 4 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_133]
-              Select Operator [SEL_132] (rows=1 width=224)
-                Output:["_col1"]
-                Group By Operator [GBY_131] (rows=1 width=112)
-                  Output:["_col0"],aggregations:["sum(VALUE._col0)"]
-                <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
-                  PARTITION_ONLY_SHUFFLE [RS_36]
-                    Group By Operator [GBY_35] (rows=1 width=112)
-                      Output:["_col0"],aggregations:["sum(_col2)"]
-                      Select Operator [SEL_34] (rows=2478 width=112)
-                        Output:["_col2"]
-                        Filter Operator [FIL_33] (rows=2478 width=112)
-                          predicate:(_col2 > _col5)
-                          Merge Join Operator [MERGEJOIN_107] (rows=7434 width=112)
-                            Conds:RS_30._col1=RS_31._col2(Inner),Output:["_col2","_col5"]
-                          <-Reducer 9 [ONE_TO_ONE_EDGE]
-                            FORWARD [RS_31]
-                              PartitionCols:_col2
-                              Merge Join Operator [MERGEJOIN_106] (rows=97 width=116)
-                                Conds:RS_125._col0=RS_114._col0(Inner),Output:["_col1","_col2"]
-                              <-Map 12 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_114]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_113] (rows=669 width=4)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_112] (rows=669 width=7)
-                                      predicate:(i_manufact_id = 269)
-                                      TableScan [TS_20] (rows=462000 width=7)
-                                        default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_manufact_id"]
-                              <-Reducer 8 [ONE_TO_ONE_EDGE] vectorized
-                                FORWARD [RS_125]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_124] (rows=6951 width=116)
-                                    Output:["_col0","_col1"]
-                                    Group By Operator [GBY_123] (rows=6951 width=124)
-                                      Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
-                                    <-Reducer 7 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_17]
-                                        PartitionCols:_col0
-                                        Group By Operator [GBY_16] (rows=55608 width=124)
-                                          Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
-                                          Merge Join Operator [MERGEJOIN_105] (rows=15995224 width=115)
-                                            Conds:RS_122._col0=RS_111._col0(Inner),Output:["_col1","_col2"]
-                                          <-Map 6 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_111]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_109] (rows=8116 width=4)
-                                                Output:["_col0"]
-                                                Filter Operator [FIL_108] (rows=8116 width=98)
-                                                  predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'1998-03-18 00:00:00' AND TIMESTAMP'1998-06-16 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 11 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_122]
-                                              PartitionCols:_col0
-                                              Select Operator [SEL_121] (rows=143966864 width=119)
-                                                Output:["_col0","_col1","_col2"]
-                                                Filter Operator [FIL_120] (rows=143966864 width=119)
-                                                  predicate:((ws_item_sk BETWEEN DynamicValue(RS_24_item_i_item_sk_min) AND DynamicValue(RS_24_item_i_item_sk_max) and in_bloom_filter(ws_item_sk, DynamicValue(RS_24_item_i_item_sk_bloom_filter))) and ws_sold_date_sk is not null)
-                                                  TableScan [TS_6] (rows=144002668 width=119)
-                                                    default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk","ws_ext_discount_amt"]
-                                                  <-Reducer 13 [BROADCAST_EDGE] vectorized
-                                                    BROADCAST [RS_119]
-                                                      Group By Operator [GBY_118] (rows=1 width=12)
-                                                        Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                      <-Map 12 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                        SHUFFLE [RS_117]
-                                                          Group By Operator [GBY_116] (rows=1 width=12)
-                                                            Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                            Select Operator [SEL_115] (rows=669 width=4)
-                                                              Output:["_col0"]
-                                                               Please refer to the previous Select Operator [SEL_113]
-                          <-Reducer 2 [SIMPLE_EDGE]
-                            SHUFFLE [RS_30]
-                              PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_104] (rows=15995224 width=115)
-                                Conds:RS_130._col0=RS_110._col0(Inner),Output:["_col1","_col2"]
-                              <-Map 6 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_110]
-                                  PartitionCols:_col0
-                                   Please refer to the previous Select Operator [SEL_109]
-                              <-Map 1 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_130]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_129] (rows=143966864 width=119)
-                                    Output:["_col0","_col1","_col2"]
-                                    Filter Operator [FIL_128] (rows=143966864 width=119)
-                                      predicate:((ws_item_sk BETWEEN DynamicValue(RS_31_item_i_item_sk_min) AND DynamicValue(RS_31_item_i_item_sk_max) and in_bloom_filter(ws_item_sk, DynamicValue(RS_31_item_i_item_sk_bloom_filter))) and ws_sold_date_sk is not null)
-                                      TableScan [TS_0] (rows=144002668 width=119)
-                                        default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk","ws_ext_discount_amt"]
-                                      <-Reducer 10 [BROADCAST_EDGE] vectorized
-                                        BROADCAST [RS_127]
-                                          Group By Operator [GBY_126] (rows=1 width=12)
-                                            Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                          <-Reducer 9 [CUSTOM_SIMPLE_EDGE]
-                                            FORWARD [RS_68]
-                                              Group By Operator [GBY_67] (rows=1 width=12)
-                                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                Select Operator [SEL_66] (rows=97 width=8)
-                                                  Output:["_col0"]
-                                                   Please refer to the previous Merge Join Operator [MERGEJOIN_106]
+      Reducer 4 vectorized
+      File Output Operator [FS_128]
+        Group By Operator [GBY_127] (rows=1 width=112)
+          Output:["_col0"],aggregations:["sum(VALUE._col0)"]
+        <-Reducer 3 [CUSTOM_SIMPLE_EDGE]
+          PARTITION_ONLY_SHUFFLE [RS_36]
+            Group By Operator [GBY_35] (rows=1 width=112)
+              Output:["_col0"],aggregations:["sum(_col2)"]
+              Select Operator [SEL_34] (rows=2478 width=112)
+                Output:["_col2"]
+                Filter Operator [FIL_33] (rows=2478 width=112)
+                  predicate:(_col2 > _col5)
+                  Merge Join Operator [MERGEJOIN_103] (rows=7434 width=112)
+                    Conds:RS_30._col1=RS_31._col2(Inner),Output:["_col2","_col5"]
+                  <-Reducer 8 [ONE_TO_ONE_EDGE]
+                    PARTITION_ONLY_SHUFFLE [RS_31]
+                      PartitionCols:_col2
+                      Merge Join Operator [MERGEJOIN_102] (rows=97 width=116)
+                        Conds:RS_121._col0=RS_110._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 11 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_110]
+                          PartitionCols:_col0
+                          Select Operator [SEL_109] (rows=669 width=4)
+                            Output:["_col0"]
+                            Filter Operator [FIL_108] (rows=669 width=7)
+                              predicate:(i_manufact_id = 269)
+                              TableScan [TS_20] (rows=462000 width=7)
+                                default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_manufact_id"]
+                      <-Reducer 7 [ONE_TO_ONE_EDGE] vectorized
+                        FORWARD [RS_121]
+                          PartitionCols:_col0
+                          Select Operator [SEL_120] (rows=6951 width=116)
+                            Output:["_col0","_col1"]
+                            Group By Operator [GBY_119] (rows=6951 width=124)
+                              Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0
+                            <-Reducer 6 [SIMPLE_EDGE]
+                              SHUFFLE [RS_17]
+                                PartitionCols:_col0
+                                Group By Operator [GBY_16] (rows=55608 width=124)
+                                  Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","count(_col2)"],keys:_col1
+                                  Merge Join Operator [MERGEJOIN_101] (rows=15995224 width=115)
+                                    Conds:RS_118._col0=RS_107._col0(Inner),Output:["_col1","_col2"]
+                                  <-Map 5 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_107]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_105] (rows=8116 width=4)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_104] (rows=8116 width=98)
+                                          predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'1998-03-18 00:00:00' AND TIMESTAMP'1998-06-16 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 10 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_118]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_117] (rows=143966864 width=119)
+                                        Output:["_col0","_col1","_col2"]
+                                        Filter Operator [FIL_116] (rows=143966864 width=119)
+                                          predicate:((ws_item_sk BETWEEN DynamicValue(RS_24_item_i_item_sk_min) AND DynamicValue(RS_24_item_i_item_sk_max) and in_bloom_filter(ws_item_sk, DynamicValue(RS_24_item_i_item_sk_bloom_filter))) and ws_sold_date_sk is not null)
+                                          TableScan [TS_6] (rows=144002668 width=119)
+                                            default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk","ws_ext_discount_amt"]
+                                          <-Reducer 12 [BROADCAST_EDGE] vectorized
+                                            BROADCAST [RS_115]
+                                              Group By Operator [GBY_114] (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
+                                                SHUFFLE [RS_113]
+                                                  Group By Operator [GBY_112] (rows=1 width=12)
+                                                    Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                    Select Operator [SEL_111] (rows=669 width=4)
+                                                      Output:["_col0"]
+                                                       Please refer to the previous Select Operator [SEL_109]
+                  <-Reducer 2 [SIMPLE_EDGE]
+                    SHUFFLE [RS_30]
+                      PartitionCols:_col1
+                      Merge Join Operator [MERGEJOIN_100] (rows=15995224 width=115)
+                        Conds:RS_126._col0=RS_106._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 5 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_106]
+                          PartitionCols:_col0
+                           Please refer to the previous Select Operator [SEL_105]
+                      <-Map 1 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_126]
+                          PartitionCols:_col0
+                          Select Operator [SEL_125] (rows=143966864 width=119)
+                            Output:["_col0","_col1","_col2"]
+                            Filter Operator [FIL_124] (rows=143966864 width=119)
+                              predicate:((ws_item_sk BETWEEN DynamicValue(RS_31_item_i_item_sk_min) AND DynamicValue(RS_31_item_i_item_sk_max) and in_bloom_filter(ws_item_sk, DynamicValue(RS_31_item_i_item_sk_bloom_filter))) and ws_sold_date_sk is not null)
+                              TableScan [TS_0] (rows=144002668 width=119)
+                                default@web_sales,web_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_sold_date_sk","ws_item_sk","ws_ext_discount_amt"]
+                              <-Reducer 9 [BROADCAST_EDGE] vectorized
+                                BROADCAST [RS_123]
+                                  Group By Operator [GBY_122] (rows=1 width=12)
+                                    Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                  <-Reducer 8 [CUSTOM_SIMPLE_EDGE]
+                                    PARTITION_ONLY_SHUFFLE [RS_64]
+                                      Group By Operator [GBY_63] (rows=1 width=12)
+                                        Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                        Select Operator [SEL_62] (rows=97 width=8)
+                                          Output:["_col0"]
+                                           Please refer to the previous Merge Join Operator [MERGEJOIN_102]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query94.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query94.q.out
index ab688b2..3b3db96 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query94.q.out
@@ -69,156 +69,147 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 13 (BROADCAST_EDGE)
-Map 15 <- Reducer 10 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 13 <- Map 12 (CUSTOM_SIMPLE_EDGE)
-Reducer 17 <- Map 16 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
-Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 14 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 15 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-Reducer 6 <- Reducer 17 (ONE_TO_ONE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE)
+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 (ONE_TO_ONE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
-Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 9 vectorized
-      File Output Operator [FS_166]
-        Limit [LIM_165] (rows=1 width=240)
-          Number of rows:100
-          Select Operator [SEL_164] (rows=1 width=240)
-            Output:["_col0","_col1","_col2"]
-          <-Reducer 8 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_163]
-              Select Operator [SEL_162] (rows=1 width=240)
-                Output:["_col1","_col2","_col3"]
-                Group By Operator [GBY_161] (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_160]
-                    Group By Operator [GBY_159] (rows=1 width=232)
-                      Output:["_col0","_col1","_col2"],aggregations:["count(_col0)","sum(_col1)","sum(_col2)"]
-                      Group By Operator [GBY_158] (rows=5022875 width=228)
-                        Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
-                      <-Reducer 6 [SIMPLE_EDGE]
-                        SHUFFLE [RS_73]
-                          PartitionCols:_col0
-                          Group By Operator [GBY_72] (rows=5022875 width=228)
-                            Output:["_col0","_col2","_col3"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col4
-                            Select Operator [SEL_41] (rows=5022875 width=229)
+      Reducer 8 vectorized
+      File Output Operator [FS_158]
+        Group By Operator [GBY_157] (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)
+              Output:["_col0","_col1","_col2"],aggregations:["count(_col0)","sum(_col1)","sum(_col2)"]
+              Group By Operator [GBY_154] (rows=5022875 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=5022875 width=228)
+                    Output:["_col0","_col2","_col3"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col4
+                    Select Operator [SEL_41] (rows=5022875 width=229)
+                      Output:["_col4","_col5","_col6"]
+                      Filter Operator [FIL_40] (rows=5022875 width=229)
+                        predicate:_col14 is null
+                        Merge Join Operator [MERGEJOIN_125] (rows=10045750 width=229)
+                          Conds:RS_37._col4=RS_153._col0(Left Outer),Output:["_col4","_col5","_col6","_col14"]
+                        <-Reducer 16 [ONE_TO_ONE_EDGE] vectorized
+                          FORWARD [RS_153]
+                            PartitionCols:_col0
+                            Select Operator [SEL_152] (rows=8007986 width=8)
+                              Output:["_col0","_col1"]
+                              Group By Operator [GBY_151] (rows=8007986 width=4)
+                                Output:["_col0"],keys:KEY._col0
+                              <-Map 15 [SIMPLE_EDGE] vectorized
+                                SHUFFLE [RS_150]
+                                  PartitionCols:_col0
+                                  Group By Operator [GBY_149] (rows=14398467 width=4)
+                                    Output:["_col0"],keys:wr_order_number
+                                    TableScan [TS_25] (rows=14398467 width=4)
+                                      default@web_returns,wr1,Tbl:COMPLETE,Col:COMPLETE,Output:["wr_order_number"]
+                        <-Reducer 5 [ONE_TO_ONE_EDGE]
+                          FORWARD [RS_37]
+                            PartitionCols:_col4
+                            Select Operator [SEL_36] (rows=5022875 width=231)
                               Output:["_col4","_col5","_col6"]
-                              Filter Operator [FIL_40] (rows=5022875 width=229)
-                                predicate:_col14 is null
-                                Merge Join Operator [MERGEJOIN_129] (rows=10045750 width=229)
-                                  Conds:RS_37._col4=RS_157._col0(Left Outer),Output:["_col4","_col5","_col6","_col14"]
-                                <-Reducer 17 [ONE_TO_ONE_EDGE] vectorized
-                                  FORWARD [RS_157]
-                                    PartitionCols:_col0
-                                    Select Operator [SEL_156] (rows=8007986 width=8)
-                                      Output:["_col0","_col1"]
-                                      Group By Operator [GBY_155] (rows=8007986 width=4)
-                                        Output:["_col0"],keys:KEY._col0
-                                      <-Map 16 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_154]
+                              Merge Join Operator [MERGEJOIN_124] (rows=5022875 width=235)
+                                Conds:RS_33._col4=RS_148._col0(Left Semi),Output:["_col3","_col4","_col5","_col6","_col14"],residual filter predicates:{(_col3 <> _col14)}
+                              <-Reducer 4 [SIMPLE_EDGE]
+                                PARTITION_ONLY_SHUFFLE [RS_33]
+                                  PartitionCols:_col4
+                                  Merge Join Operator [MERGEJOIN_123] (rows=5022875 width=231)
+                                    Conds:RS_18._col2=RS_142._col0(Inner),Output:["_col3","_col4","_col5","_col6"]
+                                  <-Map 13 [SIMPLE_EDGE] vectorized
+                                    SHUFFLE [RS_142]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_141] (rows=12 width=91)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_140] (rows=12 width=92)
+                                          predicate:(web_company_name = 'pri')
+                                          TableScan [TS_9] (rows=84 width=92)
+                                            default@web_site,web_site,Tbl:COMPLETE,Col:COMPLETE,Output:["web_site_sk","web_company_name"]
+                                  <-Reducer 3 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_18]
+                                      PartitionCols:_col2
+                                      Merge Join Operator [MERGEJOIN_122] (rows=15673790 width=235)
+                                        Conds:RS_15._col1=RS_128._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
+                                      <-Map 11 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_128]
                                           PartitionCols:_col0
-                                          Group By Operator [GBY_153] (rows=14398467 width=4)
-                                            Output:["_col0"],keys:wr_order_number
-                                            TableScan [TS_25] (rows=14398467 width=4)
-                                              default@web_returns,wr1,Tbl:COMPLETE,Col:COMPLETE,Output:["wr_order_number"]
-                                <-Reducer 5 [ONE_TO_ONE_EDGE]
-                                  FORWARD [RS_37]
-                                    PartitionCols:_col4
-                                    Select Operator [SEL_36] (rows=5022875 width=231)
-                                      Output:["_col4","_col5","_col6"]
-                                      Merge Join Operator [MERGEJOIN_128] (rows=5022875 width=235)
-                                        Conds:RS_33._col4=RS_152._col0(Left Semi),Output:["_col3","_col4","_col5","_col6","_col14"],residual filter predicates:{(_col3 <> _col14)}
-                                      <-Reducer 4 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_33]
-                                          PartitionCols:_col4
-                                          Merge Join Operator [MERGEJOIN_127] (rows=5022875 width=231)
-                                            Conds:RS_18._col2=RS_146._col0(Inner),Output:["_col3","_col4","_col5","_col6"]
-                                          <-Map 14 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_146]
+                                          Select Operator [SEL_127] (rows=784314 width=90)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_126] (rows=784314 width=90)
+                                              predicate:(ca_state = 'TX')
+                                              TableScan [TS_6] (rows=40000000 width=90)
+                                                default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_state"]
+                                      <-Reducer 2 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_15]
+                                          PartitionCols:_col1
+                                          Merge Join Operator [MERGEJOIN_121] (rows=15987241 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]
+                                              PartitionCols:_col0
+                                              Select Operator [SEL_135] (rows=143895019 width=243)
+                                                Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+                                                Filter Operator [FIL_134] (rows=143895019 width=243)
+                                                  predicate:((ws_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(ws_ship_addr_sk, DynamicValue(RS_16_customer_address_ca_address_sk_bloom_filter))) and ws_ship_addr_sk is not null and ws_ship_date_sk is not null and ws_web_site_sk is not null)
+                                                  TableScan [TS_0] (rows=144002668 width=243)
+                                                    default@web_sales,ws1,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_ship_date_sk","ws_ship_addr_sk","ws_web_site_sk","ws_warehouse_sk","ws_order_number","ws_ext_ship_cost","ws_net_profit"]
+                                                  <-Reducer 12 [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
+                                                        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
+                                            SHUFFLE [RS_139]
                                               PartitionCols:_col0
-                                              Select Operator [SEL_145] (rows=12 width=91)
+                                              Select Operator [SEL_138] (rows=8116 width=98)
                                                 Output:["_col0"]
-                                                Filter Operator [FIL_144] (rows=12 width=92)
-                                                  predicate:(web_company_name = 'pri')
-                                                  TableScan [TS_9] (rows=84 width=92)
-                                                    default@web_site,web_site,Tbl:COMPLETE,Col:COMPLETE,Output:["web_site_sk","web_company_name"]
-                                          <-Reducer 3 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_18]
-                                              PartitionCols:_col2
-                                              Merge Join Operator [MERGEJOIN_126] (rows=15673790 width=235)
-                                                Conds:RS_15._col1=RS_132._col0(Inner),Output:["_col2","_col3","_col4","_col5","_col6"]
-                                              <-Map 12 [SIMPLE_EDGE] vectorized
-                                                SHUFFLE [RS_132]
-                                                  PartitionCols:_col0
-                                                  Select Operator [SEL_131] (rows=784314 width=90)
+                                                Filter Operator [FIL_137] (rows=8116 width=98)
+                                                  predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'1999-05-01 00:00:00' AND TIMESTAMP'1999-06-30 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=143966743 width=7)
+                                    Output:["_col0","_col1"],keys:_col0, _col1
+                                    Select Operator [SEL_146] (rows=143966743 width=7)
+                                      Output:["_col0","_col1"]
+                                      Filter Operator [FIL_145] (rows=143966743 width=7)
+                                        predicate:((ws_order_number BETWEEN DynamicValue(RS_33_ws1_ws_order_number_min) AND DynamicValue(RS_33_ws1_ws_order_number_max) and in_bloom_filter(ws_order_number, DynamicValue(RS_33_ws1_ws_order_number_bloom_filter))) and ws_warehouse_sk is not null)
+                                        TableScan [TS_22] (rows=144002668 width=7)
+                                          default@web_sales,ws2,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_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=5022875 width=8)
                                                     Output:["_col0"]
-                                                    Filter Operator [FIL_130] (rows=784314 width=90)
-                                                      predicate:(ca_state = 'TX')
-                                                      TableScan [TS_6] (rows=40000000 width=90)
-                                                        default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_state"]
-                                              <-Reducer 2 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_15]
-                                                  PartitionCols:_col1
-                                                  Merge Join Operator [MERGEJOIN_125] (rows=15987241 width=239)
-                                                    Conds:RS_140._col0=RS_143._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                  <-Map 1 [SIMPLE_EDGE] vectorized
-                                                    SHUFFLE [RS_140]
-                                                      PartitionCols:_col0
-                                                      Select Operator [SEL_139] (rows=143895019 width=243)
-                                                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                                                        Filter Operator [FIL_138] (rows=143895019 width=243)
-                                                          predicate:((ws_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(ws_ship_addr_sk, DynamicValue(RS_16_customer_address_ca_address_sk_bloom_filter))) and ws_ship_addr_sk is not null and ws_ship_date_sk is not null and ws_web_site_sk is not null)
-                                                          TableScan [TS_0] (rows=144002668 width=243)
-                                                            default@web_sales,ws1,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_ship_date_sk","ws_ship_addr_sk","ws_web_site_sk","ws_warehouse_sk","ws_order_number","ws_ext_ship_cost","ws_net_profit"]
-                                                          <-Reducer 13 [BROADCAST_EDGE] vectorized
-                                                            BROADCAST [RS_137]
-                                                              Group By Operator [GBY_136] (rows=1 width=12)
-                                                                Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                              <-Map 12 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                SHUFFLE [RS_135]
-                                                                  Group By Operator [GBY_134] (rows=1 width=12)
-                                                                    Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                    Select Operator [SEL_133] (rows=784314 width=4)
-                                                                      Output:["_col0"]
-                                                                       Please refer to the previous Select Operator [SEL_131]
-                                                  <-Map 11 [SIMPLE_EDGE] vectorized
-                                                    SHUFFLE [RS_143]
-                                                      PartitionCols:_col0
-                                                      Select Operator [SEL_142] (rows=8116 width=98)
-                                                        Output:["_col0"]
-                                                        Filter Operator [FIL_141] (rows=8116 width=98)
-                                                          predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'1999-05-01 00:00:00' AND TIMESTAMP'1999-06-30 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 15 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_152]
-                                          PartitionCols:_col0
-                                          Group By Operator [GBY_151] (rows=143966743 width=7)
-                                            Output:["_col0","_col1"],keys:_col0, _col1
-                                            Select Operator [SEL_150] (rows=143966743 width=7)
-                                              Output:["_col0","_col1"]
-                                              Filter Operator [FIL_149] (rows=143966743 width=7)
-                                                predicate:((ws_order_number BETWEEN DynamicValue(RS_33_ws1_ws_order_number_min) AND DynamicValue(RS_33_ws1_ws_order_number_max) and in_bloom_filter(ws_order_number, DynamicValue(RS_33_ws1_ws_order_number_bloom_filter))) and ws_warehouse_sk is not null)
-                                                TableScan [TS_22] (rows=144002668 width=7)
-                                                  default@web_sales,ws2,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
-                                                <-Reducer 10 [BROADCAST_EDGE] vectorized
-                                                  BROADCAST [RS_148]
-                                                    Group By Operator [GBY_147] (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_115]
-                                                        Group By Operator [GBY_114] (rows=1 width=12)
-                                                          Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                          Select Operator [SEL_113] (rows=5022875 width=8)
-                                                            Output:["_col0"]
-                                                             Please refer to the previous Merge Join Operator [MERGEJOIN_127]
+                                                     Please refer to the previous Merge Join Operator [MERGEJOIN_123]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query95.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query95.q.out
index 420cd78..90114cc 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query95.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query95.q.out
@@ -75,228 +75,219 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 14 (BROADCAST_EDGE)
-Map 16 <- Reducer 11 (BROADCAST_EDGE)
-Map 19 <- Reducer 11 (BROADCAST_EDGE)
-Map 20 <- Reducer 10 (BROADCAST_EDGE)
-Map 24 <- Reducer 10 (BROADCAST_EDGE)
-Reducer 10 <- Reducer 5 (CUSTOM_SIMPLE_EDGE)
-Reducer 11 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
-Reducer 14 <- Map 13 (CUSTOM_SIMPLE_EDGE)
-Reducer 17 <- Map 16 (SIMPLE_EDGE), Map 19 (SIMPLE_EDGE)
-Reducer 18 <- Reducer 17 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
-Reducer 21 <- Map 20 (SIMPLE_EDGE), Map 24 (SIMPLE_EDGE)
-Reducer 22 <- Map 25 (SIMPLE_EDGE), Reducer 21 (ONE_TO_ONE_EDGE)
-Reducer 23 <- Reducer 22 (SIMPLE_EDGE)
-Reducer 3 <- Map 13 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 15 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 18 (ONE_TO_ONE_EDGE), Reducer 4 (SIMPLE_EDGE)
-Reducer 6 <- Reducer 23 (ONE_TO_ONE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE)
+Map 1 <- Reducer 13 (BROADCAST_EDGE)
+Map 15 <- Reducer 10 (BROADCAST_EDGE)
+Map 18 <- Reducer 10 (BROADCAST_EDGE)
+Map 19 <- Reducer 9 (BROADCAST_EDGE)
+Map 23 <- Reducer 9 (BROADCAST_EDGE)
+Reducer 10 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
+Reducer 13 <- Map 12 (CUSTOM_SIMPLE_EDGE)
+Reducer 16 <- Map 15 (SIMPLE_EDGE), Map 18 (SIMPLE_EDGE)
+Reducer 17 <- Reducer 16 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
+Reducer 20 <- Map 19 (SIMPLE_EDGE), Map 23 (SIMPLE_EDGE)
+Reducer 21 <- Map 24 (SIMPLE_EDGE), Reducer 20 (ONE_TO_ONE_EDGE)
+Reducer 22 <- Reducer 21 (SIMPLE_EDGE)
+Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 14 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 17 (ONE_TO_ONE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 22 (ONE_TO_ONE_EDGE), Reducer 5 (ONE_TO_ONE_EDGE)
 Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
 Reducer 8 <- Reducer 7 (CUSTOM_SIMPLE_EDGE)
-Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 5 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 9 vectorized
-      File Output Operator [FS_284]
-        Limit [LIM_283] (rows=1 width=240)
-          Number of rows:100
-          Select Operator [SEL_282] (rows=1 width=240)
-            Output:["_col0","_col1","_col2"]
-          <-Reducer 8 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_281]
-              Select Operator [SEL_280] (rows=1 width=240)
-                Output:["_col1","_col2","_col3"]
-                Group By Operator [GBY_279] (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_278]
-                    Group By Operator [GBY_277] (rows=1 width=232)
-                      Output:["_col0","_col1","_col2"],aggregations:["count(_col0)","sum(_col1)","sum(_col2)"]
-                      Group By Operator [GBY_276] (rows=5022875 width=228)
-                        Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
-                      <-Reducer 6 [SIMPLE_EDGE]
-                        SHUFFLE [RS_109]
-                          PartitionCols:_col0
-                          Group By Operator [GBY_108] (rows=5022875 width=228)
-                            Output:["_col0","_col2","_col3"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col3
-                            Merge Join Operator [MERGEJOIN_235] (rows=5022875 width=227)
-                              Conds:RS_55._col3=RS_275._col0(Inner),Output:["_col3","_col4","_col5"]
-                            <-Reducer 5 [ONE_TO_ONE_EDGE]
-                              FORWARD [RS_55]
-                                PartitionCols:_col3
-                                Merge Join Operator [MERGEJOIN_234] (rows=5022875 width=227)
-                                  Conds:RS_52._col3=RS_263._col0(Inner),Output:["_col3","_col4","_col5"]
-                                <-Reducer 4 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_52]
-                                    PartitionCols:_col3
-                                    Merge Join Operator [MERGEJOIN_232] (rows=5022875 width=227)
-                                      Conds:RS_49._col2=RS_252._col0(Inner),Output:["_col3","_col4","_col5"]
-                                    <-Map 15 [SIMPLE_EDGE] vectorized
-                                      SHUFFLE [RS_252]
-                                        PartitionCols:_col0
-                                        Select Operator [SEL_251] (rows=12 width=4)
-                                          Output:["_col0"]
-                                          Filter Operator [FIL_250] (rows=12 width=92)
-                                            predicate:(web_company_name = 'pri')
-                                            TableScan [TS_9] (rows=84 width=92)
-                                              default@web_site,web_site,Tbl:COMPLETE,Col:COMPLETE,Output:["web_site_sk","web_company_name"]
-                                    <-Reducer 3 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_49]
-                                        PartitionCols:_col2
-                                        Merge Join Operator [MERGEJOIN_231] (rows=15673790 width=231)
-                                          Conds:RS_46._col1=RS_238._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
-                                        <-Map 13 [SIMPLE_EDGE] vectorized
-                                          SHUFFLE [RS_238]
-                                            PartitionCols:_col0
-                                            Select Operator [SEL_237] (rows=784314 width=4)
-                                              Output:["_col0"]
-                                              Filter Operator [FIL_236] (rows=784314 width=90)
-                                                predicate:(ca_state = 'TX')
-                                                TableScan [TS_6] (rows=40000000 width=90)
-                                                  default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_state"]
-                                        <-Reducer 2 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_46]
-                                            PartitionCols:_col1
-                                            Merge Join Operator [MERGEJOIN_230] (rows=15987241 width=235)
-                                              Conds:RS_246._col0=RS_249._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
-                                            <-Map 1 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_246]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_245] (rows=143895019 width=239)
-                                                  Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                                  Filter Operator [FIL_244] (rows=143895019 width=239)
-                                                    predicate:((ws_ship_addr_sk BETWEEN DynamicValue(RS_47_customer_address_ca_address_sk_min) AND DynamicValue(RS_47_customer_address_ca_address_sk_max) and in_bloom_filter(ws_ship_addr_sk, DynamicValue(RS_47_customer_address_ca_address_sk_bloom_filter))) and ws_ship_addr_sk is not null and ws_ship_date_sk is not null and ws_web_site_sk is not null)
-                                                    TableScan [TS_0] (rows=144002668 width=239)
-                                                      default@web_sales,ws1,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_ship_date_sk","ws_ship_addr_sk","ws_web_site_sk","ws_order_number","ws_ext_ship_cost","ws_net_profit"]
-                                                    <-Reducer 14 [BROADCAST_EDGE] vectorized
-                                                      BROADCAST [RS_243]
-                                                        Group By Operator [GBY_242] (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
-                                                          SHUFFLE [RS_241]
-                                                            Group By Operator [GBY_240] (rows=1 width=12)
-                                                              Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                              Select Operator [SEL_239] (rows=784314 width=4)
-                                                                Output:["_col0"]
-                                                                 Please refer to the previous Select Operator [SEL_237]
-                                            <-Map 12 [SIMPLE_EDGE] vectorized
-                                              SHUFFLE [RS_249]
-                                                PartitionCols:_col0
-                                                Select Operator [SEL_248] (rows=8116 width=98)
-                                                  Output:["_col0"]
-                                                  Filter Operator [FIL_247] (rows=8116 width=98)
-                                                    predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'1999-05-01 00:00:00' AND TIMESTAMP'1999-06-30 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"]
-                                <-Reducer 18 [ONE_TO_ONE_EDGE] vectorized
-                                  FORWARD [RS_263]
+      Reducer 8 vectorized
+      File Output Operator [FS_276]
+        Group By Operator [GBY_275] (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_274]
+            Group By Operator [GBY_273] (rows=1 width=232)
+              Output:["_col0","_col1","_col2"],aggregations:["count(_col0)","sum(_col1)","sum(_col2)"]
+              Group By Operator [GBY_272] (rows=5022875 width=228)
+                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
+              <-Reducer 6 [SIMPLE_EDGE]
+                SHUFFLE [RS_105]
+                  PartitionCols:_col0
+                  Group By Operator [GBY_104] (rows=5022875 width=228)
+                    Output:["_col0","_col2","_col3"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col3
+                    Merge Join Operator [MERGEJOIN_231] (rows=5022875 width=227)
+                      Conds:RS_55._col3=RS_271._col0(Inner),Output:["_col3","_col4","_col5"]
+                    <-Reducer 5 [ONE_TO_ONE_EDGE]
+                      PARTITION_ONLY_SHUFFLE [RS_55]
+                        PartitionCols:_col3
+                        Merge Join Operator [MERGEJOIN_230] (rows=5022875 width=227)
+                          Conds:RS_52._col3=RS_259._col0(Inner),Output:["_col3","_col4","_col5"]
+                        <-Reducer 4 [SIMPLE_EDGE]
+                          SHUFFLE [RS_52]
+                            PartitionCols:_col3
+                            Merge Join Operator [MERGEJOIN_228] (rows=5022875 width=227)
+                              Conds:RS_49._col2=RS_248._col0(Inner),Output:["_col3","_col4","_col5"]
+                            <-Map 14 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_248]
+                                PartitionCols:_col0
+                                Select Operator [SEL_247] (rows=12 width=4)
+                                  Output:["_col0"]
+                                  Filter Operator [FIL_246] (rows=12 width=92)
+                                    predicate:(web_company_name = 'pri')
+                                    TableScan [TS_9] (rows=84 width=92)
+                                      default@web_site,web_site,Tbl:COMPLETE,Col:COMPLETE,Output:["web_site_sk","web_company_name"]
+                            <-Reducer 3 [SIMPLE_EDGE]
+                              SHUFFLE [RS_49]
+                                PartitionCols:_col2
+                                Merge Join Operator [MERGEJOIN_227] (rows=15673790 width=231)
+                                  Conds:RS_46._col1=RS_234._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
+                                <-Map 12 [SIMPLE_EDGE] vectorized
+                                  SHUFFLE [RS_234]
                                     PartitionCols:_col0
-                                    Group By Operator [GBY_262] (rows=14686712 width=4)
-                                      Output:["_col0"],keys:KEY._col0
-                                    <-Reducer 17 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_22]
+                                    Select Operator [SEL_233] (rows=784314 width=4)
+                                      Output:["_col0"]
+                                      Filter Operator [FIL_232] (rows=784314 width=90)
+                                        predicate:(ca_state = 'TX')
+                                        TableScan [TS_6] (rows=40000000 width=90)
+                                          default@customer_address,customer_address,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_state"]
+                                <-Reducer 2 [SIMPLE_EDGE]
+                                  SHUFFLE [RS_46]
+                                    PartitionCols:_col1
+                                    Merge Join Operator [MERGEJOIN_226] (rows=15987241 width=235)
+                                      Conds:RS_242._col0=RS_245._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5"]
+                                    <-Map 1 [SIMPLE_EDGE] vectorized
+                                      SHUFFLE [RS_242]
                                         PartitionCols:_col0
-                                        Group By Operator [GBY_21] (rows=144002668 width=4)
-                                          Output:["_col0"],keys:_col1
-                                          Select Operator [SEL_20] (rows=1411940834 width=11)
-                                            Output:["_col1"]
-                                            Filter Operator [FIL_19] (rows=1411940834 width=11)
-                                              predicate:(_col0 <> _col2)
-                                              Merge Join Operator [MERGEJOIN_233] (rows=1411940834 width=11)
-                                                Conds:RS_258._col1=RS_261._col1(Inner),Output:["_col0","_col1","_col2"]
-                                              <-Map 16 [SIMPLE_EDGE] vectorized
-                                                SHUFFLE [RS_258]
-                                                  PartitionCols:_col1
-                                                  Select Operator [SEL_257] (rows=144002668 width=7)
-                                                    Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_256] (rows=144002668 width=7)
-                                                      predicate:(in_bloom_filter(ws_order_number, DynamicValue(RS_52_ws1_ws_order_number_bloom_filter)) and ws_order_number BETWEEN DynamicValue(RS_52_ws1_ws_order_number_min) AND DynamicValue(RS_52_ws1_ws_order_number_max))
-                                                      TableScan [TS_12] (rows=144002668 width=7)
-                                                        default@web_sales,ws1,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
-                                                      <-Reducer 11 [BROADCAST_EDGE] vectorized
-                                                        BROADCAST [RS_254]
-                                                          Group By Operator [GBY_253] (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_187]
-                                                              Group By Operator [GBY_186] (rows=1 width=12)
-                                                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                Select Operator [SEL_185] (rows=5022875 width=8)
-                                                                  Output:["_col0"]
-                                                                   Please refer to the previous Merge Join Operator [MERGEJOIN_232]
-                                              <-Map 19 [SIMPLE_EDGE] vectorized
-                                                SHUFFLE [RS_261]
-                                                  PartitionCols:_col1
-                                                  Select Operator [SEL_260] (rows=144002668 width=7)
-                                                    Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_259] (rows=144002668 width=7)
-                                                      predicate:(in_bloom_filter(ws_order_number, DynamicValue(RS_52_ws1_ws_order_number_bloom_filter)) and ws_order_number BETWEEN DynamicValue(RS_52_ws1_ws_order_number_min) AND DynamicValue(RS_52_ws1_ws_order_number_max))
-                                                      TableScan [TS_14] (rows=144002668 width=7)
-                                                        default@web_sales,ws2,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
-                                                      <-Reducer 11 [BROADCAST_EDGE] vectorized
-                                                        BROADCAST [RS_255]
-                                                           Please refer to the previous Group By Operator [GBY_253]
-                            <-Reducer 23 [ONE_TO_ONE_EDGE] vectorized
-                              FORWARD [RS_275]
+                                        Select Operator [SEL_241] (rows=143895019 width=239)
+                                          Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+                                          Filter Operator [FIL_240] (rows=143895019 width=239)
+                                            predicate:((ws_ship_addr_sk BETWEEN DynamicValue(RS_47_customer_address_ca_address_sk_min) AND DynamicValue(RS_47_customer_address_ca_address_sk_max) and in_bloom_filter(ws_ship_addr_sk, DynamicValue(RS_47_customer_address_ca_address_sk_bloom_filter))) and ws_ship_addr_sk is not null and ws_ship_date_sk is not null and ws_web_site_sk is not null)
+                                            TableScan [TS_0] (rows=144002668 width=239)
+                                              default@web_sales,ws1,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_ship_date_sk","ws_ship_addr_sk","ws_web_site_sk","ws_order_number","ws_ext_ship_cost","ws_net_profit"]
+                                            <-Reducer 13 [BROADCAST_EDGE] vectorized
+                                              BROADCAST [RS_239]
+                                                Group By Operator [GBY_238] (rows=1 width=12)
+                                                  Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                <-Map 12 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                  SHUFFLE [RS_237]
+                                                    Group By Operator [GBY_236] (rows=1 width=12)
+                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                      Select Operator [SEL_235] (rows=784314 width=4)
+                                                        Output:["_col0"]
+                                                         Please refer to the previous Select Operator [SEL_233]
+                                    <-Map 11 [SIMPLE_EDGE] vectorized
+                                      SHUFFLE [RS_245]
+                                        PartitionCols:_col0
+                                        Select Operator [SEL_244] (rows=8116 width=98)
+                                          Output:["_col0"]
+                                          Filter Operator [FIL_243] (rows=8116 width=98)
+                                            predicate:CAST( d_date AS TIMESTAMP) BETWEEN TIMESTAMP'1999-05-01 00:00:00' AND TIMESTAMP'1999-06-30 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"]
+                        <-Reducer 17 [ONE_TO_ONE_EDGE] vectorized
+                          FORWARD [RS_259]
+                            PartitionCols:_col0
+                            Group By Operator [GBY_258] (rows=14686712 width=4)
+                              Output:["_col0"],keys:KEY._col0
+                            <-Reducer 16 [SIMPLE_EDGE]
+                              SHUFFLE [RS_22]
                                 PartitionCols:_col0
-                                Group By Operator [GBY_274] (rows=8007986 width=4)
-                                  Output:["_col0"],keys:KEY._col0
-                                <-Reducer 22 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_40]
-                                    PartitionCols:_col0
-                                    Group By Operator [GBY_39] (rows=14398467 width=4)
-                                      Output:["_col0"],keys:_col14
-                                      Merge Join Operator [MERGEJOIN_229] (rows=1384229738 width=4)
-                                        Conds:RS_35._col0=RS_273.wr_order_number(Inner),Output:["_col14"]
-                                      <-Map 25 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_273]
-                                          PartitionCols:wr_order_number
-                                          TableScan [TS_34] (rows=14398467 width=4)
-                                            default@web_returns,web_returns,Tbl:COMPLETE,Col:COMPLETE,Output:["wr_order_number"]
-                                      <-Reducer 21 [ONE_TO_ONE_EDGE]
-                                        FORWARD [RS_35]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_33] (rows=1411940834 width=4)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_32] (rows=1411940834 width=11)
-                                              predicate:(_col0 <> _col2)
-                                              Merge Join Operator [MERGEJOIN_228] (rows=1411940834 width=11)
-                                                Conds:RS_269._col1=RS_272._col1(Inner),Output:["_col0","_col1","_col2"]
-                                              <-Map 20 [SIMPLE_EDGE] vectorized
-                                                SHUFFLE [RS_269]
-                                                  PartitionCols:_col1
-                                                  Select Operator [SEL_268] (rows=144002668 width=7)
-                                                    Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_267] (rows=144002668 width=7)
-                                                      predicate:(in_bloom_filter(ws_order_number, DynamicValue(RS_55_ws1_ws_order_number_bloom_filter)) and ws_order_number BETWEEN DynamicValue(RS_55_ws1_ws_order_number_min) AND DynamicValue(RS_55_ws1_ws_order_number_max))
-                                                      TableScan [TS_25] (rows=144002668 width=7)
-                                                        default@web_sales,ws1,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
-                                                      <-Reducer 10 [BROADCAST_EDGE] vectorized
-                                                        BROADCAST [RS_265]
-                                                          Group By Operator [GBY_264] (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]
-                                                            FORWARD [RS_206]
-                                                              Group By Operator [GBY_205] (rows=1 width=12)
-                                                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                Select Operator [SEL_204] (rows=5022875 width=8)
-                                                                  Output:["_col0"]
-                                                                   Please refer to the previous Merge Join Operator [MERGEJOIN_234]
-                                              <-Map 24 [SIMPLE_EDGE] vectorized
-                                                SHUFFLE [RS_272]
-                                                  PartitionCols:_col1
-                                                  Select Operator [SEL_271] (rows=144002668 width=7)
-                                                    Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_270] (rows=144002668 width=7)
-                                                      predicate:(in_bloom_filter(ws_order_number, DynamicValue(RS_55_ws1_ws_order_number_bloom_filter)) and ws_order_number BETWEEN DynamicValue(RS_55_ws1_ws_order_number_min) AND DynamicValue(RS_55_ws1_ws_order_number_max))
-                                                      TableScan [TS_27] (rows=144002668 width=7)
-                                                        default@web_sales,ws2,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
-                                                      <-Reducer 10 [BROADCAST_EDGE] vectorized
-                                                        BROADCAST [RS_266]
-                                                           Please refer to the previous Group By Operator [GBY_264]
+                                Group By Operator [GBY_21] (rows=144002668 width=4)
+                                  Output:["_col0"],keys:_col1
+                                  Select Operator [SEL_20] (rows=1411940834 width=11)
+                                    Output:["_col1"]
+                                    Filter Operator [FIL_19] (rows=1411940834 width=11)
+                                      predicate:(_col0 <> _col2)
+                                      Merge Join Operator [MERGEJOIN_229] (rows=1411940834 width=11)
+                                        Conds:RS_254._col1=RS_257._col1(Inner),Output:["_col0","_col1","_col2"]
+                                      <-Map 15 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_254]
+                                          PartitionCols:_col1
+                                          Select Operator [SEL_253] (rows=144002668 width=7)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_252] (rows=144002668 width=7)
+                                              predicate:(in_bloom_filter(ws_order_number, DynamicValue(RS_52_ws1_ws_order_number_bloom_filter)) and ws_order_number BETWEEN DynamicValue(RS_52_ws1_ws_order_number_min) AND DynamicValue(RS_52_ws1_ws_order_number_max))
+                                              TableScan [TS_12] (rows=144002668 width=7)
+                                                default@web_sales,ws1,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
+                                              <-Reducer 10 [BROADCAST_EDGE] vectorized
+                                                BROADCAST [RS_250]
+                                                  Group By Operator [GBY_249] (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_183]
+                                                      Group By Operator [GBY_182] (rows=1 width=12)
+                                                        Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                        Select Operator [SEL_181] (rows=5022875 width=8)
+                                                          Output:["_col0"]
+                                                           Please refer to the previous Merge Join Operator [MERGEJOIN_228]
+                                      <-Map 18 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_257]
+                                          PartitionCols:_col1
+                                          Select Operator [SEL_256] (rows=144002668 width=7)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_255] (rows=144002668 width=7)
+                                              predicate:(in_bloom_filter(ws_order_number, DynamicValue(RS_52_ws1_ws_order_number_bloom_filter)) and ws_order_number BETWEEN DynamicValue(RS_52_ws1_ws_order_number_min) AND DynamicValue(RS_52_ws1_ws_order_number_max))
+                                              TableScan [TS_14] (rows=144002668 width=7)
+                                                default@web_sales,ws2,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
+                                              <-Reducer 10 [BROADCAST_EDGE] vectorized
+                                                BROADCAST [RS_251]
+                                                   Please refer to the previous Group By Operator [GBY_249]
+                    <-Reducer 22 [ONE_TO_ONE_EDGE] vectorized
+                      FORWARD [RS_271]
+                        PartitionCols:_col0
+                        Group By Operator [GBY_270] (rows=8007986 width=4)
+                          Output:["_col0"],keys:KEY._col0
+                        <-Reducer 21 [SIMPLE_EDGE]
+                          SHUFFLE [RS_40]
+                            PartitionCols:_col0
+                            Group By Operator [GBY_39] (rows=14398467 width=4)
+                              Output:["_col0"],keys:_col14
+                              Merge Join Operator [MERGEJOIN_225] (rows=1384229738 width=4)
+                                Conds:RS_35._col0=RS_269.wr_order_number(Inner),Output:["_col14"]
+                              <-Map 24 [SIMPLE_EDGE] vectorized
+                                SHUFFLE [RS_269]
+                                  PartitionCols:wr_order_number
+                                  TableScan [TS_34] (rows=14398467 width=4)
+                                    default@web_returns,web_returns,Tbl:COMPLETE,Col:COMPLETE,Output:["wr_order_number"]
+                              <-Reducer 20 [ONE_TO_ONE_EDGE]
+                                FORWARD [RS_35]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_33] (rows=1411940834 width=4)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_32] (rows=1411940834 width=11)
+                                      predicate:(_col0 <> _col2)
+                                      Merge Join Operator [MERGEJOIN_224] (rows=1411940834 width=11)
+                                        Conds:RS_265._col1=RS_268._col1(Inner),Output:["_col0","_col1","_col2"]
+                                      <-Map 19 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_265]
+                                          PartitionCols:_col1
+                                          Select Operator [SEL_264] (rows=144002668 width=7)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_263] (rows=144002668 width=7)
+                                              predicate:(in_bloom_filter(ws_order_number, DynamicValue(RS_55_ws1_ws_order_number_bloom_filter)) and ws_order_number BETWEEN DynamicValue(RS_55_ws1_ws_order_number_min) AND DynamicValue(RS_55_ws1_ws_order_number_max))
+                                              TableScan [TS_25] (rows=144002668 width=7)
+                                                default@web_sales,ws1,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
+                                              <-Reducer 9 [BROADCAST_EDGE] vectorized
+                                                BROADCAST [RS_261]
+                                                  Group By Operator [GBY_260] (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_202]
+                                                      Group By Operator [GBY_201] (rows=1 width=12)
+                                                        Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                        Select Operator [SEL_200] (rows=5022875 width=8)
+                                                          Output:["_col0"]
+                                                           Please refer to the previous Merge Join Operator [MERGEJOIN_230]
+                                      <-Map 23 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_268]
+                                          PartitionCols:_col1
+                                          Select Operator [SEL_267] (rows=144002668 width=7)
+                                            Output:["_col0","_col1"]
+                                            Filter Operator [FIL_266] (rows=144002668 width=7)
+                                              predicate:(in_bloom_filter(ws_order_number, DynamicValue(RS_55_ws1_ws_order_number_bloom_filter)) and ws_order_number BETWEEN DynamicValue(RS_55_ws1_ws_order_number_min) AND DynamicValue(RS_55_ws1_ws_order_number_max))
+                                              TableScan [TS_27] (rows=144002668 width=7)
+                                                default@web_sales,ws2,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_warehouse_sk","ws_order_number"]
+                                              <-Reducer 9 [BROADCAST_EDGE] vectorized
+                                                BROADCAST [RS_262]
+                                                   Please refer to the previous Group By Operator [GBY_260]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query96.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query96.q.out
index 27c26aa..7947605 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query96.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query96.q.out
@@ -41,91 +41,82 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 8 (BROADCAST_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
-Reducer 3 <- Map 9 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 10 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Map 1 <- Reducer 7 (BROADCAST_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (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 (CUSTOM_SIMPLE_EDGE)
-Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
-Reducer 8 <- Map 7 (CUSTOM_SIMPLE_EDGE)
+Reducer 7 <- Map 6 (CUSTOM_SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 6 vectorized
-      File Output Operator [FS_99]
-        Limit [LIM_98] (rows=1 width=16)
-          Number of rows:100
-          Select Operator [SEL_97] (rows=1 width=16)
-            Output:["_col0"]
-          <-Reducer 5 [SIMPLE_EDGE] vectorized
-            SHUFFLE [RS_96]
-              Select Operator [SEL_95] (rows=1 width=16)
-                Output:["_col1"]
-                Group By Operator [GBY_94] (rows=1 width=8)
-                  Output:["_col0"],aggregations:["count(VALUE._col0)"]
-                <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
-                  PARTITION_ONLY_SHUFFLE [RS_23]
-                    Group By Operator [GBY_22] (rows=1 width=8)
-                      Output:["_col0"],aggregations:["count()"]
-                      Merge Join Operator [MERGEJOIN_76] (rows=1084713 width=8)
-                        Conds:RS_18._col2=RS_93._col0(Inner)
-                      <-Map 10 [SIMPLE_EDGE] vectorized
-                        SHUFFLE [RS_93]
+      Reducer 5 vectorized
+      File Output Operator [FS_91]
+        Group By Operator [GBY_90] (rows=1 width=8)
+          Output:["_col0"],aggregations:["count(VALUE._col0)"]
+        <-Reducer 4 [CUSTOM_SIMPLE_EDGE]
+          PARTITION_ONLY_SHUFFLE [RS_23]
+            Group By Operator [GBY_22] (rows=1 width=8)
+              Output:["_col0"],aggregations:["count()"]
+              Merge Join Operator [MERGEJOIN_72] (rows=1084713 width=8)
+                Conds:RS_18._col2=RS_89._col0(Inner)
+              <-Map 9 [SIMPLE_EDGE] vectorized
+                SHUFFLE [RS_89]
+                  PartitionCols:_col0
+                  Select Operator [SEL_88] (rows=155 width=4)
+                    Output:["_col0"]
+                    Filter Operator [FIL_87] (rows=155 width=92)
+                      predicate:(s_store_name = 'ese')
+                      TableScan [TS_9] (rows=1704 width=92)
+                        default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_store_sk","s_store_name"]
+              <-Reducer 3 [SIMPLE_EDGE]
+                SHUFFLE [RS_18]
+                  PartitionCols:_col2
+                  Merge Join Operator [MERGEJOIN_71] (rows=1477476 width=0)
+                    Conds:RS_15._col1=RS_86._col0(Inner),Output:["_col2"]
+                  <-Map 8 [SIMPLE_EDGE] vectorized
+                    SHUFFLE [RS_86]
+                      PartitionCols:_col0
+                      Select Operator [SEL_85] (rows=655 width=4)
+                        Output:["_col0"]
+                        Filter Operator [FIL_84] (rows=655 width=8)
+                          predicate:(hd_dep_count = 5)
+                          TableScan [TS_6] (rows=7200 width=8)
+                            default@household_demographics,household_demographics,Tbl:COMPLETE,Col:COMPLETE,Output:["hd_demo_sk","hd_dep_count"]
+                  <-Reducer 2 [SIMPLE_EDGE]
+                    SHUFFLE [RS_15]
+                      PartitionCols:_col1
+                      Merge Join Operator [MERGEJOIN_70] (rows=16240953 width=0)
+                        Conds:RS_83._col0=RS_75._col0(Inner),Output:["_col1","_col2"]
+                      <-Map 6 [SIMPLE_EDGE] vectorized
+                        PARTITION_ONLY_SHUFFLE [RS_75]
                           PartitionCols:_col0
-                          Select Operator [SEL_92] (rows=155 width=4)
+                          Select Operator [SEL_74] (rows=1515 width=4)
                             Output:["_col0"]
-                            Filter Operator [FIL_91] (rows=155 width=92)
-                              predicate:(s_store_name = 'ese')
-                              TableScan [TS_9] (rows=1704 width=92)
-                                default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_store_sk","s_store_name"]
-                      <-Reducer 3 [SIMPLE_EDGE]
-                        SHUFFLE [RS_18]
-                          PartitionCols:_col2
-                          Merge Join Operator [MERGEJOIN_75] (rows=1477476 width=0)
-                            Conds:RS_15._col1=RS_90._col0(Inner),Output:["_col2"]
-                          <-Map 9 [SIMPLE_EDGE] vectorized
-                            SHUFFLE [RS_90]
-                              PartitionCols:_col0
-                              Select Operator [SEL_89] (rows=655 width=4)
-                                Output:["_col0"]
-                                Filter Operator [FIL_88] (rows=655 width=8)
-                                  predicate:(hd_dep_count = 5)
-                                  TableScan [TS_6] (rows=7200 width=8)
-                                    default@household_demographics,household_demographics,Tbl:COMPLETE,Col:COMPLETE,Output:["hd_demo_sk","hd_dep_count"]
-                          <-Reducer 2 [SIMPLE_EDGE]
-                            SHUFFLE [RS_15]
-                              PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_74] (rows=16240953 width=0)
-                                Conds:RS_87._col0=RS_79._col0(Inner),Output:["_col1","_col2"]
-                              <-Map 7 [SIMPLE_EDGE] vectorized
-                                PARTITION_ONLY_SHUFFLE [RS_79]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_78] (rows=1515 width=4)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_77] (rows=1515 width=12)
-                                      predicate:((t_hour = 8) and (t_minute >= 30))
-                                      TableScan [TS_3] (rows=86400 width=12)
-                                        default@time_dim,time_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["t_time_sk","t_hour","t_minute"]
-                              <-Map 1 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_87]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_86] (rows=501695814 width=11)
-                                    Output:["_col0","_col1","_col2"]
-                                    Filter Operator [FIL_85] (rows=501695814 width=11)
-                                      predicate:((ss_sold_time_sk BETWEEN DynamicValue(RS_13_time_dim_t_time_sk_min) AND DynamicValue(RS_13_time_dim_t_time_sk_max) and in_bloom_filter(ss_sold_time_sk, DynamicValue(RS_13_time_dim_t_time_sk_bloom_filter))) and ss_hdemo_sk is not null and ss_sold_time_sk is not null and ss_store_sk is not null)
-                                      TableScan [TS_0] (rows=575995635 width=11)
-                                        default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_time_sk","ss_hdemo_sk","ss_store_sk"]
-                                      <-Reducer 8 [BROADCAST_EDGE] vectorized
-                                        BROADCAST [RS_84]
-                                          Group By Operator [GBY_83] (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
-                                            PARTITION_ONLY_SHUFFLE [RS_82]
-                                              Group By Operator [GBY_81] (rows=1 width=12)
-                                                Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                Select Operator [SEL_80] (rows=1515 width=4)
-                                                  Output:["_col0"]
-                                                   Please refer to the previous Select Operator [SEL_78]
+                            Filter Operator [FIL_73] (rows=1515 width=12)
+                              predicate:((t_hour = 8) and (t_minute >= 30))
+                              TableScan [TS_3] (rows=86400 width=12)
+                                default@time_dim,time_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["t_time_sk","t_hour","t_minute"]
+                      <-Map 1 [SIMPLE_EDGE] vectorized
+                        SHUFFLE [RS_83]
+                          PartitionCols:_col0
+                          Select Operator [SEL_82] (rows=501695814 width=11)
+                            Output:["_col0","_col1","_col2"]
+                            Filter Operator [FIL_81] (rows=501695814 width=11)
+                              predicate:((ss_sold_time_sk BETWEEN DynamicValue(RS_13_time_dim_t_time_sk_min) AND DynamicValue(RS_13_time_dim_t_time_sk_max) and in_bloom_filter(ss_sold_time_sk, DynamicValue(RS_13_time_dim_t_time_sk_bloom_filter))) and ss_hdemo_sk is not null and ss_sold_time_sk is not null and ss_store_sk is not null)
+                              TableScan [TS_0] (rows=575995635 width=11)
+                                default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_time_sk","ss_hdemo_sk","ss_store_sk"]
+                              <-Reducer 7 [BROADCAST_EDGE] vectorized
+                                BROADCAST [RS_80]
+                                  Group By Operator [GBY_79] (rows=1 width=12)
+                                    Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
... 6120 lines suppressed ...