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/16 00:21:58 UTC

[hive] branch master updated: HIVE-21382: Group by keys reduction optimization - keys are not reduced in query23 (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 6a33d26  HIVE-21382: Group by keys reduction optimization - keys are not reduced in query23 (Vineet Garg, reviewed by Jesus Camacho Rodriguez)
6a33d26 is described below

commit 6a33d2606668563e65b6dd9dec588ef65021353d
Author: Vineet Garg <vg...@apache.org>
AuthorDate: Fri Mar 15 17:20:00 2019 -0700

    HIVE-21382: Group by keys reduction optimization - keys are not reduced in query23 (Vineet Garg, reviewed by Jesus Camacho Rodriguez)
    
    Closes apache/hive#567
---
 .../calcite/rules/HiveRelFieldTrimmer.java         | 114 +++--
 .../clientpositive/constraints_optimization.q      |  94 ++++
 .../llap/constraints_optimization.q.out            | 431 +++++++++++++++++
 .../perf/tez/constraints/cbo_query23.q.out         |  22 +-
 .../perf/tez/constraints/cbo_query39.q.out         |  20 +-
 .../perf/tez/constraints/cbo_query64.q.out         |   4 +-
 .../perf/tez/constraints/query23.q.out             |  46 +-
 .../perf/tez/constraints/query39.q.out             | 126 +++--
 .../perf/tez/constraints/query64.q.out             | 511 +++++++++++----------
 9 files changed, 977 insertions(+), 391 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
index 2bfd12a..3759ed6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -46,6 +47,8 @@ import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPermuteInputsShuttle;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.validate.SqlValidator;
@@ -60,7 +63,6 @@ import org.apache.calcite.util.mapping.MappingType;
 import org.apache.calcite.util.mapping.Mappings;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
@@ -320,43 +322,99 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
   // if those are columns are not being used further up
   private ImmutableBitSet generateGroupSetIfCardinalitySame(final Aggregate aggregate,
                                         final ImmutableBitSet originalGroupSet, final ImmutableBitSet fieldsUsed) {
-    Pair<RelOptTable, List<Integer>> tabToOrgCol = HiveRelOptUtil.getColumnOriginSet(aggregate.getInput(),
-                                                                                     originalGroupSet);
-    if(tabToOrgCol == null) {
-      return originalGroupSet;
-    }
-    RelOptHiveTable tbl = (RelOptHiveTable)tabToOrgCol.left;
-    List<Integer> backtrackedGBList = tabToOrgCol.right;
-    ImmutableBitSet backtrackedGBSet = ImmutableBitSet.builder().addAll(backtrackedGBList).build();
 
-    List<ImmutableBitSet> allKeys = tbl.getNonNullableKeys();
-    ImmutableBitSet currentKey = null;
-    for(ImmutableBitSet key:allKeys) {
-      if(backtrackedGBSet.contains(key)) {
-        // only if grouping sets consist of keys
-        currentKey = key;
-        break;
+    RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
+    RelMetadataQuery mq = aggregate.getCluster().getMetadataQuery();
+
+    // map from backtracked table ref to list of gb keys and list of corresponding backtracked columns
+    Map<RexTableInputRef.RelTableRef, List<Pair<Integer, Integer>>> mapGBKeysLineage= new HashMap<>();
+
+    // map from table ref to list of columns (from gb keys) which are candidate to be removed
+    Map<RexTableInputRef.RelTableRef, List<Integer>> candidateKeys = new HashMap<>();
+
+    for(int key:originalGroupSet) {
+      RexNode inputRef = rexBuilder.makeInputRef(aggregate.getInput(), key);
+      Set<RexNode> exprLineage = mq.getExpressionLineage(aggregate.getInput(), inputRef);
+      if(exprLineage != null && exprLineage.size() == 1){
+        RexNode expr = exprLineage.iterator().next();
+        if(expr instanceof RexTableInputRef) {
+          RexTableInputRef tblRef = (RexTableInputRef)expr;
+          if(mapGBKeysLineage.containsKey(tblRef.getTableRef())) {
+            mapGBKeysLineage.get(tblRef.getTableRef()).add(Pair.of(tblRef.getIndex(), key));
+          } else {
+            List<Pair<Integer, Integer>> newList = new ArrayList<>();
+            newList.add(Pair.of(tblRef.getIndex(), key));
+            mapGBKeysLineage.put(tblRef.getTableRef(), newList);
+          }
+        } else if(RexUtil.isDeterministic(expr)){
+          // even though we weren't able to backtrack this key it could still be candidate for removal
+          // if rest of the columns contain pk/unique
+          Set<RexTableInputRef.RelTableRef> tableRefs = RexUtil.gatherTableReferences(Lists.newArrayList(expr));
+          if(tableRefs.size() == 1) {
+            RexTableInputRef.RelTableRef tblRef = tableRefs.iterator().next();
+            if(candidateKeys.containsKey(tblRef)) {
+              List<Integer> candidateGBKeys = candidateKeys.get(tblRef);
+              candidateGBKeys.add(key);
+            } else {
+              List<Integer> candidateGBKeys =  new ArrayList<>();
+              candidateGBKeys.add(key);
+              candidateKeys.put(tblRef, candidateGBKeys);
+            }
+          }
+        }
       }
     }
-    if(currentKey == null || currentKey.isEmpty()) {
-      return originalGroupSet;
-    }
 
     // we want to delete all columns in original GB set except the key
     ImmutableBitSet.Builder builder = ImmutableBitSet.builder();
 
-    // we have established that this gb set contains keys and it is safe to remove rest of the columns
-    for(int i=0; i<backtrackedGBList.size(); i++) {
-      Integer backtrackedCol = backtrackedGBList.get(i);
-      int orgCol = originalGroupSet.nth(i);
-      if(fieldsUsed.get((orgCol))
-          || currentKey.get(backtrackedCol)) {
-        // keep the columns which are being used or are part of keys
-        builder.set(orgCol);
+    for(Map.Entry<RexTableInputRef.RelTableRef, List<Pair<Integer, Integer>>> entry:mapGBKeysLineage.entrySet()) {
+      RelOptHiveTable tbl = (RelOptHiveTable)entry.getKey().getTable();
+      List<Pair<Integer, Integer>> gbKeyCols = entry.getValue();
+
+      ImmutableBitSet.Builder btBuilder = ImmutableBitSet.builder();
+      gbKeyCols.forEach(pair -> btBuilder.set(pair.left));
+      ImmutableBitSet backtrackedGBSet = btBuilder.build();
+
+      List<ImmutableBitSet> allKeys = tbl.getNonNullableKeys();
+      ImmutableBitSet currentKey = null;
+      for(ImmutableBitSet key:allKeys) {
+        if(backtrackedGBSet.contains(key)) {
+          // only if grouping sets consist of keys
+          currentKey = key;
+          break;
+        }
+      }
+      if(currentKey == null || currentKey.isEmpty()) {
+        continue;
+      }
+
+      // we have established that this gb set contains keys and it is safe to remove rest of the columns
+      for(Pair<Integer, Integer> gbKeyColPair:gbKeyCols) {
+        Integer backtrackedCol = gbKeyColPair.left;
+        Integer orgCol =  gbKeyColPair.right;
+        if(!fieldsUsed.get(orgCol)
+            && !currentKey.get(backtrackedCol)) {
+          // this could could be removed
+          builder.set(orgCol);
+        }
+      }
+      // remove candidate keys if possible
+      if(candidateKeys.containsKey(entry.getKey())) {
+        List<Integer> candidateGbKeys= candidateKeys.get(entry.getKey());
+        for(Integer keyToRemove:candidateGbKeys) {
+          if(!fieldsUsed.get(keyToRemove)) {
+            builder.set(keyToRemove);
+          }
+        }
       }
     }
-    return builder.build();
+    ImmutableBitSet keysToRemove = builder.build();
+    ImmutableBitSet newGroupSet = originalGroupSet.except(keysToRemove);
+    assert(!newGroupSet.isEmpty());
+    return newGroupSet;
   }
+
   // if gby keys consist of pk/uk non-pk/non-uk columns are removed if they are not being used
   private ImmutableBitSet generateNewGroupset(Aggregate aggregate, ImmutableBitSet fieldsUsed) {
 
diff --git a/ql/src/test/queries/clientpositive/constraints_optimization.q b/ql/src/test/queries/clientpositive/constraints_optimization.q
index 4f76442..bd6e716 100644
--- a/ql/src/test/queries/clientpositive/constraints_optimization.q
+++ b/ql/src/test/queries/clientpositive/constraints_optimization.q
@@ -428,6 +428,84 @@ GROUP BY
 ,	C_LOGIN
 ;
 
+-- group by keys with columns from multiple table
+explain cbo select c_customer_sk from
+ (select c_first_name, c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by c_first_name,c_customer_sk,d_date
+  having count(*) >4) subq;
+
+-- group by keys from multiple table with expression
+explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk,d_date
+  having count(*) >4) subq;
+
+-- group by keys from same table with expression
+ explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk
+  having count(*) >4) subq;
+
+ -- group by keys from multiple table with non-deterministic expression
+explain cbo select c_customer_sk from
+ (select rand(), c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by rand(),c_customer_sk,d_date
+  having count(*) >4) subq;
+
+-- group by keys from multiple table with expression on pk itself, group by shouldn't be reduced
+explain cbo select * from
+ (select substr(c_first_name, 1,30), log2(c_customer_sk),d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),log2(c_customer_sk),d_date
+  having count(*) >4) subq;
+
+ -- group by with keys consisting of pk from multiple tables with extra columns from both side
+ explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,d_date solddate, d_date_sk, count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk,d_date, d_date_sk
+  having count(*) >4) subq;
+
+ -- group by with keys consisting of pk from multiple tables with extra expressions from both side
+  explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,log2(d_date) solddate, d_date_sk, count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk, log2(d_date), d_date_sk
+  having count(*) >4) subq;
+
+
 create table web_sales(ws_order_number int, ws_item_sk int, ws_price float,
     constraint pk1 primary key(ws_order_number, ws_item_sk) disable rely);
 insert into web_sales values(1, 1, 1.2);
@@ -435,3 +513,19 @@ insert into web_sales values(1, 1, 1.2);
  explain cbo select count(distinct ws_order_number) from web_sales;
  select count(distinct ws_order_number) from web_sales;
  drop table web_sales;
+
+create table t1(i int primary key disable rely, j int);
+insert into t1 values(1,100),(2,200);
+create table t2(i int primary key disable rely, j int);
+insert into t2 values(2,1000),(4,500);
+
+-- UNION
+explain cbo select i from (select i, j from t1 union all select i,j from t2) subq group by i,j;
+select i from (select i, j from t1 union all select i,j from t2) subq group by i,j;
+
+-- INTERSECT
+explain cbo select i from (select i, j from t1 intersect select i,j from t2) subq group by i,j;
+select i from (select i, j from t1 intersect select i,j from t2) subq group by i,j;
+
+drop table t1;
+drop table t2;
\ No newline at end of file
diff --git a/ql/src/test/results/clientpositive/llap/constraints_optimization.q.out b/ql/src/test/results/clientpositive/llap/constraints_optimization.q.out
index f7ed9f5..28bdb72 100644
--- a/ql/src/test/results/clientpositive/llap/constraints_optimization.q.out
+++ b/ql/src/test/results/clientpositive/llap/constraints_optimization.q.out
@@ -2742,6 +2742,318 @@ HiveAggregate(group=[{1}])
       HiveFilter(condition=[IS NOT NULL($3)])
         HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
 
+PREHOOK: query: explain cbo select c_customer_sk from
+ (select c_first_name, c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by c_first_name,c_customer_sk,d_date
+  having count(*) >4) subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select c_customer_sk from
+ (select c_first_name, c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by c_first_name,c_customer_sk,d_date
+  having count(*) >4) subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(c_customer_sk=[$0])
+  HiveFilter(condition=[>($2, 4)])
+    HiveProject(c_customer_sk=[$1], d_date=[$0], $f2=[$2])
+      HiveAggregate(group=[{3, 4}], 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=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(c_customer_sk=[$0], c_first_name=[$8])
+            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+
+PREHOOK: query: explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk,d_date
+  having count(*) >4) subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk,d_date
+  having count(*) >4) subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(c_customer_sk=[$0])
+  HiveFilter(condition=[>($2, 4)])
+    HiveProject(c_customer_sk=[$1], d_date=[$0], $f2=[$2])
+      HiveAggregate(group=[{3, 4}], 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=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(c_customer_sk=[$0], substr=[substr($8, 1, 30)])
+            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+
+PREHOOK: query: explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk
+  having count(*) >4) subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk
+  having count(*) >4) subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(c_customer_sk=[$0])
+  HiveFilter(condition=[>($1, 4)])
+    HiveAggregate(group=[{3}], agg#0=[count()])
+      HiveJoin(condition=[=($1, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2])
+            HiveFilter(condition=[IS NOT NULL($0)])
+              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+          HiveProject(d_date_sk=[$0])
+            HiveFilter(condition=[IS NOT NULL($0)])
+              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(c_customer_sk=[$0], substr=[substr($8, 1, 30)])
+          HiveTableScan(table=[[default, customer]], table:alias=[customer])
+
+PREHOOK: query: -- group by keys from multiple table with non-deterministic expression
+explain cbo select c_customer_sk from
+ (select rand(), c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by rand(),c_customer_sk,d_date
+  having count(*) >4) subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+POSTHOOK: query: -- group by keys from multiple table with non-deterministic expression
+explain cbo select c_customer_sk from
+ (select rand(), c_customer_sk ,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by rand(),c_customer_sk,d_date
+  having count(*) >4) subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(c_customer_sk=[$1])
+  HiveFilter(condition=[>($3, 4)])
+    HiveAggregate(group=[{0, 1, 2}], agg#0=[count()])
+      HiveProject($f0=[rand()], $f1=[$2], $f2=[$4])
+        HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($1, $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(c_customer_sk=[$0])
+              HiveTableScan(table=[[default, customer]], table:alias=[customer])
+          HiveProject(d_date_sk=[$0], d_date=[$2])
+            HiveFilter(condition=[IS NOT NULL($0)])
+              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+
+PREHOOK: query: explain cbo select * from
+ (select substr(c_first_name, 1,30), log2(c_customer_sk),d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),log2(c_customer_sk),d_date
+  having count(*) >4) subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from
+ (select substr(c_first_name, 1,30), log2(c_customer_sk),d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),log2(c_customer_sk),d_date
+  having count(*) >4) subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+CBO PLAN:
+HiveFilter(condition=[>($3, 4)])
+  HiveProject(substr=[$1], log2=[$2], d_date=[$0], $f3=[$3])
+    HiveAggregate(group=[{3, 5, 6}], 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=[IS NOT NULL($0)])
+              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(c_customer_sk=[$0], substr=[substr($8, 1, 30)], log2=[log2($0)])
+          HiveTableScan(table=[[default, customer]], table:alias=[customer])
+
+PREHOOK: query: -- group by with keys consisting of pk from multiple tables with extra columns from both side
+ explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,d_date solddate, d_date_sk, count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk,d_date, d_date_sk
+  having count(*) >4) subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+POSTHOOK: query: -- group by with keys consisting of pk from multiple tables with extra columns from both side
+ explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,d_date solddate, d_date_sk, count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk,d_date, d_date_sk
+  having count(*) >4) subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(c_customer_sk=[$0])
+  HiveFilter(condition=[>($3, 4)])
+    HiveProject(c_customer_sk=[$2], d_date=[$1], d_date_sk=[$0], $f3=[$3])
+      HiveAggregate(group=[{2, 3, 4}], 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=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(c_customer_sk=[$0], substr=[substr($8, 1, 30)])
+            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+
+PREHOOK: query: -- group by with keys consisting of pk from multiple tables with extra expressions from both side
+  explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,log2(d_date) solddate, d_date_sk, count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk, log2(d_date), d_date_sk
+  having count(*) >4) subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+POSTHOOK: query: -- group by with keys consisting of pk from multiple tables with extra expressions from both side
+  explain cbo select c_customer_sk from
+ (select substr(c_first_name, 1,30), c_customer_sk ,log2(d_date) solddate, d_date_sk, count(*) cnt
+  from store_sales
+      ,date_dim
+      ,customer
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = c_customer_sk
+  group by substr(c_first_name, 1, 30),c_customer_sk, log2(d_date), d_date_sk
+  having count(*) >4) subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(c_customer_sk=[$0])
+  HiveFilter(condition=[>($3, 4)])
+    HiveProject(c_customer_sk=[$2], log2=[$1], d_date_sk=[$0], $f3=[$3])
+      HiveAggregate(group=[{2, 3, 4}], 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], log2=[log2($2)])
+              HiveFilter(condition=[IS NOT NULL($0)])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(c_customer_sk=[$0], substr=[substr($8, 1, 30)])
+            HiveTableScan(table=[[default, customer]], table:alias=[customer])
+
 PREHOOK: query: create table web_sales(ws_order_number int, ws_item_sk int, ws_price float,
     constraint pk1 primary key(ws_order_number, ws_item_sk) disable rely)
 PREHOOK: type: CREATETABLE
@@ -2805,3 +3117,122 @@ POSTHOOK: query: drop table web_sales
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: default@web_sales
+PREHOOK: query: create table t1(i int primary key disable rely, j int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+POSTHOOK: query: create table t1(i int primary key disable rely, j int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+PREHOOK: query: insert into t1 values(1,100),(2,200)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@t1
+POSTHOOK: query: insert into t1 values(1,100),(2,200)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@t1
+POSTHOOK: Lineage: t1.i SCRIPT []
+POSTHOOK: Lineage: t1.j SCRIPT []
+PREHOOK: query: create table t2(i int primary key disable rely, j int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2
+POSTHOOK: query: create table t2(i int primary key disable rely, j int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2
+PREHOOK: query: insert into t2 values(2,1000),(4,500)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@t2
+POSTHOOK: query: insert into t2 values(2,1000),(4,500)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@t2
+POSTHOOK: Lineage: t2.i SCRIPT []
+POSTHOOK: Lineage: t2.j SCRIPT []
+PREHOOK: query: explain cbo select i from (select i, j from t1 union all select i,j from t2) subq group by i,j
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select i from (select i, j from t1 union all select i,j from t2) subq group by i,j
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(i=[$0])
+  HiveAggregate(group=[{0, 1}])
+    HiveProject(i=[$0], j=[$1])
+      HiveUnion(all=[true])
+        HiveProject(i=[$0], j=[$1])
+          HiveTableScan(table=[[default, t1]], table:alias=[t1])
+        HiveProject(i=[$0], j=[$1])
+          HiveTableScan(table=[[default, t2]], table:alias=[t2])
+
+PREHOOK: query: select i from (select i, j from t1 union all select i,j from t2) subq group by i,j
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+#### A masked pattern was here ####
+POSTHOOK: query: select i from (select i, j from t1 union all select i,j from t2) subq group by i,j
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+#### A masked pattern was here ####
+2
+2
+4
+1
+PREHOOK: query: explain cbo select i from (select i, j from t1 intersect select i,j from t2) subq group by i,j
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select i from (select i, j from t1 intersect select i,j from t2) subq group by i,j
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(i=[$0])
+  HiveFilter(condition=[=($2, 2)])
+    HiveAggregate(group=[{0, 1}], agg#0=[count($2)])
+      HiveProject(i=[$0], j=[$1], $f2=[$2])
+        HiveUnion(all=[true])
+          HiveProject(i=[$0], j=[$1], $f2=[$2])
+            HiveAggregate(group=[{0, 1}], agg#0=[count()])
+              HiveTableScan(table=[[default, t1]], table:alias=[t1])
+          HiveProject(i=[$0], j=[$1], $f2=[$2])
+            HiveAggregate(group=[{0, 1}], agg#0=[count()])
+              HiveTableScan(table=[[default, t2]], table:alias=[t2])
+
+PREHOOK: query: select i from (select i, j from t1 intersect select i,j from t2) subq group by i,j
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+#### A masked pattern was here ####
+POSTHOOK: query: select i from (select i, j from t1 intersect select i,j from t2) subq group by i,j
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+#### A masked pattern was here ####
+PREHOOK: query: drop table t1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t1
+PREHOOK: Output: default@t1
+POSTHOOK: query: drop table t1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: default@t1
+PREHOOK: query: drop table t2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t2
+PREHOOK: Output: default@t2
+POSTHOOK: query: drop table t2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t2
+POSTHOOK: Output: default@t2
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 7efcd5f..3146b77 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
@@ -120,9 +120,9 @@ POSTHOOK: Input: default@web_sales
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[sum($0)])
-  HiveProject($f0=[$0])
+  HiveProject(sales=[$0])
     HiveUnion(all=[true])
-      HiveProject($f0=[*(CAST($4):DECIMAL(10, 0), $5)])
+      HiveProject(sales=[*(CAST($4):DECIMAL(10, 0), $5)])
         HiveSemiJoin(condition=[=($3, $7)], joinType=[inner])
           HiveJoin(condition=[=($2, $0)], joinType=[inner], algorithm=[none], cost=[not available])
             HiveProject($f0=[$0])
@@ -167,10 +167,10 @@ HiveAggregate(group=[{}], agg#0=[sum($0)])
               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()])
+          HiveProject(i_item_sk=[$0])
+            HiveFilter(condition=[>($2, 4)])
+              HiveProject(i_item_sk=[$1], d_date=[$0], $f2=[$2])
+                HiveAggregate(group=[{3, 4}], 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])
@@ -181,7 +181,7 @@ HiveAggregate(group=[{}], agg#0=[sum($0)])
                           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)])
+      HiveProject(sales=[*(CAST($4):DECIMAL(10, 0), $5)])
         HiveSemiJoin(condition=[=($2, $7)], joinType=[inner])
           HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
             HiveProject($f0=[$0])
@@ -226,10 +226,10 @@ HiveAggregate(group=[{}], agg#0=[sum($0)])
               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()])
+          HiveProject(i_item_sk=[$0])
+            HiveFilter(condition=[>($2, 4)])
+              HiveProject(i_item_sk=[$1], d_date=[$0], $f2=[$2])
+                HiveAggregate(group=[{3, 4}], 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])
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query39.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query39.q.out
index 88ffbbe..272bedc 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query39.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query39.q.out
@@ -63,31 +63,31 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveProject(w_warehouse_sk=[$0], i_item_sk=[$1], d_moy=[CAST(4):INTEGER], mean=[$2], cov=[$3], w_warehouse_sk1=[$4], i_item_sk1=[$5], d_moy1=[CAST(5):INTEGER], mean1=[$6], cov1=[$7])
   HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], sort4=[$6], sort5=[$7], dir0=[ASC], dir1=[ASC], dir2=[ASC], dir3=[ASC], dir4=[ASC], dir5=[ASC])
-    HiveProject(w_warehouse_sk=[$0], i_item_sk=[$1], mean=[$2], cov=[$3], w_warehouse_sk0=[$4], i_item_sk0=[$5], mean0=[$6], cov0=[$7])
-      HiveJoin(condition=[AND(=($1, $5), =($0, $4))], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveProject(w_warehouse_sk=[$1], i_item_sk=[$2], mean=[/(CAST($6):DOUBLE, $5)], cov=[CASE(=(/(CAST($6):DOUBLE, $5), 0), null, /(POWER(/(-($3, /(*($4, $4), $5)), CASE(=($5, 1), null, -($5, 1))), 0.5), /(CAST($6):DOUBLE, $5)))])
-          HiveFilter(condition=[CASE(=(/(CAST($6):DOUBLE, $5), 0), false, >(/(POWER(/(-($3, /(*($4, $4), $5)), CASE(=($5, 1), null, -($5, 1))), 0.5), /(CAST($6):DOUBLE, $5)), 1))])
-            HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($5)], agg#1=[sum($4)], agg#2=[count($3)], agg#3=[sum($3)])
+    HiveProject(w_warehouse_sk=[$0], i_item_sk=[$1], mean=[$6], cov=[$7], w_warehouse_sk0=[$0], i_item_sk0=[$1], mean0=[$2], cov0=[$3])
+      HiveJoin(condition=[AND(=($5, $1), =($4, $0))], joinType=[inner], algorithm=[none], cost=[not available])
+        HiveProject(w_warehouse_sk=[$0], i_item_sk=[$1], mean=[/(CAST($5):DOUBLE, $4)], cov=[CASE(=(/(CAST($5):DOUBLE, $4), 0), null, /(POWER(/(-($2, /(*($3, $3), $4)), CASE(=($4, 1), null, -($4, 1))), 0.5), /(CAST($5):DOUBLE, $4)))])
+          HiveFilter(condition=[CASE(=(/(CAST($5):DOUBLE, $4), 0), false, >(/(POWER(/(-($2, /(*($3, $3), $4)), CASE(=($4, 1), null, -($4, 1))), 0.5), /(CAST($5):DOUBLE, $4)), 1))])
+            HiveAggregate(group=[{1, 2}], agg#0=[sum($5)], agg#1=[sum($4)], agg#2=[count($3)], agg#3=[sum($3)])
               HiveProject($f0=[$6], $f1=[$5], $f2=[$3], $f4=[$2], $f40=[CAST($2):DOUBLE], $f6=[*(CAST($2):DOUBLE, CAST($2):DOUBLE)])
                 HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
                   HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(inv_date_sk=[$0], inv_warehouse_sk=[$2], inv_quantity_on_hand=[$3], inv_item_sk=[$1])
                       HiveTableScan(table=[[default, inventory]], table:alias=[inventory])
                     HiveProject(d_date_sk=[$0])
-                      HiveFilter(condition=[AND(=($6, 1999), =($8, 4))])
+                      HiveFilter(condition=[AND(=($6, 1999), =($8, 5))])
                         HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                   HiveProject(w_warehouse_sk=[$0], w_warehouse_name=[$2])
                     HiveTableScan(table=[[default, warehouse]], table:alias=[warehouse])
-        HiveProject(w_warehouse_sk=[$1], i_item_sk=[$2], mean=[/(CAST($6):DOUBLE, $5)], cov=[CASE(=(/(CAST($6):DOUBLE, $5), 0), null, /(POWER(/(-($3, /(*($4, $4), $5)), CASE(=($5, 1), null, -($5, 1))), 0.5), /(CAST($6):DOUBLE, $5)))])
-          HiveFilter(condition=[CASE(=(/(CAST($6):DOUBLE, $5), 0), false, >(/(POWER(/(-($3, /(*($4, $4), $5)), CASE(=($5, 1), null, -($5, 1))), 0.5), /(CAST($6):DOUBLE, $5)), 1))])
-            HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($5)], agg#1=[sum($4)], agg#2=[count($3)], agg#3=[sum($3)])
+        HiveProject(w_warehouse_sk=[$0], i_item_sk=[$1], mean=[/(CAST($5):DOUBLE, $4)], cov=[CASE(=(/(CAST($5):DOUBLE, $4), 0), null, /(POWER(/(-($2, /(*($3, $3), $4)), CASE(=($4, 1), null, -($4, 1))), 0.5), /(CAST($5):DOUBLE, $4)))])
+          HiveFilter(condition=[CASE(=(/(CAST($5):DOUBLE, $4), 0), false, >(/(POWER(/(-($2, /(*($3, $3), $4)), CASE(=($4, 1), null, -($4, 1))), 0.5), /(CAST($5):DOUBLE, $4)), 1))])
+            HiveAggregate(group=[{1, 2}], agg#0=[sum($5)], agg#1=[sum($4)], agg#2=[count($3)], agg#3=[sum($3)])
               HiveProject($f0=[$6], $f1=[$5], $f2=[$3], $f4=[$2], $f40=[CAST($2):DOUBLE], $f6=[*(CAST($2):DOUBLE, CAST($2):DOUBLE)])
                 HiveJoin(condition=[=($1, $5)], joinType=[inner], algorithm=[none], cost=[not available])
                   HiveJoin(condition=[=($0, $4)], joinType=[inner], algorithm=[none], cost=[not available])
                     HiveProject(inv_date_sk=[$0], inv_warehouse_sk=[$2], inv_quantity_on_hand=[$3], inv_item_sk=[$1])
                       HiveTableScan(table=[[default, inventory]], table:alias=[inventory])
                     HiveProject(d_date_sk=[$0])
-                      HiveFilter(condition=[AND(=($6, 1999), =($8, 5))])
+                      HiveFilter(condition=[AND(=($6, 1999), =($8, 4))])
                         HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                   HiveProject(w_warehouse_sk=[$0], w_warehouse_name=[$2])
                     HiveTableScan(table=[[default, warehouse]], table:alias=[warehouse])
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query64.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query64.q.out
index 75e69cd..8223efc 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query64.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query64.q.out
@@ -326,8 +326,8 @@ HiveProject(product_name=[$0], store_name=[$1], store_zip=[$2], b_street_number=
                   HiveTableScan(table=[[default, customer_address]], table:alias=[ad1])
               HiveProject(ca_address_sk=[$0], ca_street_number=[$2], ca_street_name=[$3], ca_city=[$6], ca_zip=[$9])
                 HiveTableScan(table=[[default, customer_address]], table:alias=[ad2])
-        HiveProject($f1=[$0], $f2=[$2], $f3=[$3], $f15=[$14], $f16=[$15], $f17=[$16], $f18=[$17])
-          HiveAggregate(group=[{19, 20, 24, 25, 29, 31, 37, 38, 39, 40, 42, 43, 44, 45}], agg#0=[count()], agg#1=[sum($16)], agg#2=[sum($17)], agg#3=[sum($18)])
+        HiveProject($f1=[$0], $f2=[$1], $f3=[$2], $f15=[$13], $f16=[$14], $f17=[$15], $f18=[$16])
+          HiveAggregate(group=[{19, 24, 25, 29, 31, 37, 38, 39, 40, 42, 43, 44, 45}], agg#0=[count()], agg#1=[sum($16)], agg#2=[sum($17)], agg#3=[sum($18)])
             HiveJoin(condition=[=($5, $41)], joinType=[inner], algorithm=[none], cost=[not available])
               HiveJoin(condition=[=($13, $36)], joinType=[inner], algorithm=[none], cost=[not available])
                 HiveJoin(condition=[AND(=($3, $34), <>($33, $35))], joinType=[inner], algorithm=[none], cost=[not available])
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 1441e17..7ba6715 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
@@ -322,28 +322,28 @@ Stage-0
                       PartitionCols:_col0
                       Group By Operator [GBY_561] (rows=2235 width=4)
                         Output:["_col0"],keys:_col0
-                        Select Operator [SEL_560] (rows=6548799 width=4)
+                        Select Operator [SEL_560] (rows=1943705 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
+                          Filter Operator [FIL_559] (rows=1943705 width=106)
+                            predicate:(_col2 > 4L)
+                            Select Operator [SEL_558] (rows=5831115 width=106)
+                              Output:["_col0","_col2"]
+                              Group By Operator [GBY_557] (rows=5831115 width=106)
+                                Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
                               <-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"]
+                                  PartitionCols:_col0, _col1
+                                  Group By Operator [GBY_83] (rows=19646398 width=106)
+                                    Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col4, _col3
+                                    Merge Join Operator [MERGEJOIN_436] (rows=19646398 width=98)
+                                      Conds:RS_79._col1=RS_527._col0(Inner),Output:["_col3","_col4"]
                                     <-Map 37 [SIMPLE_EDGE] vectorized
                                       SHUFFLE [RS_527]
                                         PartitionCols:_col0
                                         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"]
+                                          Output:["_col0"]
+                                          TableScan [TS_74] (rows=462000 width=4)
+                                            default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk"]
                                     <-Reducer 31 [SIMPLE_EDGE]
                                       SHUFFLE [RS_79]
                                         PartitionCols:_col1
@@ -492,16 +492,16 @@ Stage-0
                       PartitionCols:_col0
                       Group By Operator [GBY_532] (rows=2235 width=4)
                         Output:["_col0"],keys:_col0
-                        Select Operator [SEL_531] (rows=6548799 width=4)
+                        Select Operator [SEL_531] (rows=1943705 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
+                          Filter Operator [FIL_530] (rows=1943705 width=106)
+                            predicate:(_col2 > 4L)
+                            Select Operator [SEL_529] (rows=5831115 width=106)
+                              Output:["_col0","_col2"]
+                              Group By Operator [GBY_528] (rows=5831115 width=106)
+                                Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
                               <-Reducer 32 [SIMPLE_EDGE]
                                 SHUFFLE [RS_84]
-                                  PartitionCols:_col0, _col1, _col2
+                                  PartitionCols:_col0, _col1
                                    Please refer to the previous Group By Operator [GBY_83]
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query39.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query39.q.out
index ca2d830..812928b 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query39.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query39.q.out
@@ -66,7 +66,7 @@ Vertex dependency in root stage
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
 Reducer 3 <- Map 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (ONE_TO_ONE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE)
 Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 Reducer 7 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
 Reducer 8 <- Map 11 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
@@ -77,101 +77,99 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 6 vectorized
-      File Output Operator [FS_170]
-        Select Operator [SEL_169] (rows=189509 width=56)
+      File Output Operator [FS_166]
+        Select Operator [SEL_165] (rows=859 width=56)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
         <-Reducer 5 [SIMPLE_EDGE]
           SHUFFLE [RS_44]
-            Merge Join Operator [MERGEJOIN_146] (rows=189509 width=48)
-              Conds:RS_163._col0, _col1=RS_168._col0, _col1(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
-            <-Reducer 4 [SIMPLE_EDGE] vectorized
-              SHUFFLE [RS_163]
-                PartitionCols:_col0, _col1
-                Select Operator [SEL_162] (rows=18049 width=24)
-                  Output:["_col0","_col1","_col2","_col3"]
-                  Filter Operator [FIL_161] (rows=18049 width=40)
-                    predicate:CASE WHEN (((UDFToDouble(_col3) / _col4) = 0)) THEN (false) ELSE (((power(((_col5 - ((_col6 * _col6) / _col4)) / CASE WHEN ((_col4 = 1L)) THEN (null) ELSE ((_col4 - 1)) END), 0.5) / (UDFToDouble(_col3) / _col4)) > 1.0D)) END
-                    Select Operator [SEL_160] (rows=36099 width=40)
-                      Output:["_col0","_col1","_col3","_col4","_col5","_col6"]
-                      Group By Operator [GBY_159] (rows=36099 width=140)
-                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2
+            Select Operator [SEL_43] (rows=859 width=48)
+              Output:["_col0","_col1","_col2","_col3","_col6","_col7"]
+              Merge Join Operator [MERGEJOIN_144] (rows=859 width=40)
+                Conds:RS_160._col0, _col1=RS_164._col0, _col1(Inner),Output:["_col0","_col1","_col2","_col3","_col6","_col7"]
+              <-Reducer 4 [ONE_TO_ONE_EDGE] vectorized
+                FORWARD [RS_160]
+                  PartitionCols:_col0, _col1
+                  Select Operator [SEL_159] (rows=859 width=24)
+                    Output:["_col0","_col1","_col2","_col3"]
+                    Filter Operator [FIL_158] (rows=859 width=40)
+                      predicate:CASE WHEN (((UDFToDouble(_col2) / _col3) = 0)) THEN (false) ELSE (((power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) / (UDFToDouble(_col2) / _col3)) > 1.0D)) END
+                      Group By Operator [GBY_157] (rows=1719 width=40)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1
                       <-Reducer 3 [SIMPLE_EDGE]
                         SHUFFLE [RS_16]
-                          PartitionCols:_col0, _col1, _col2
-                          Group By Operator [GBY_15] (rows=36099 width=140)
-                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col3)","count(_col3)","sum(_col5)","sum(_col4)"],keys:_col1, _col2, _col0
-                            Select Operator [SEL_13] (rows=1032514 width=108)
-                              Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                              Merge Join Operator [MERGEJOIN_143] (rows=1032514 width=108)
-                                Conds:RS_10._col1=RS_157._col0(Inner),Output:["_col2","_col3","_col5","_col6"]
+                          PartitionCols:_col0, _col1
+                          Group By Operator [GBY_15] (rows=1719 width=40)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col3)","count(_col3)","sum(_col5)","sum(_col4)"],keys:_col1, _col2
+                            Select Operator [SEL_13] (rows=1032514 width=8)
+                              Output:["_col1","_col2","_col3","_col4","_col5"]
+                              Merge Join Operator [MERGEJOIN_141] (rows=1032514 width=8)
+                                Conds:RS_10._col1=RS_155._col0(Inner),Output:["_col2","_col3","_col5"]
                               <-Map 11 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_157]
+                                SHUFFLE [RS_155]
                                   PartitionCols:_col0
-                                  Select Operator [SEL_156] (rows=27 width=104)
-                                    Output:["_col0","_col1"]
-                                    TableScan [TS_5] (rows=27 width=104)
-                                      default@warehouse,warehouse,Tbl:COMPLETE,Col:COMPLETE,Output:["w_warehouse_sk","w_warehouse_name"]
+                                  Select Operator [SEL_154] (rows=27 width=4)
+                                    Output:["_col0"]
+                                    TableScan [TS_5] (rows=27 width=4)
+                                      default@warehouse,warehouse,Tbl:COMPLETE,Col:COMPLETE,Output:["w_warehouse_sk"]
                               <-Reducer 2 [SIMPLE_EDGE]
                                 SHUFFLE [RS_10]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_142] (rows=1032514 width=8)
-                                    Conds:RS_148._col0=RS_154._col0(Inner),Output:["_col1","_col2","_col3"]
+                                  Merge Join Operator [MERGEJOIN_140] (rows=1032514 width=8)
+                                    Conds:RS_146._col0=RS_152._col0(Inner),Output:["_col1","_col2","_col3"]
                                   <-Map 1 [SIMPLE_EDGE] vectorized
-                                    SHUFFLE [RS_148]
+                                    SHUFFLE [RS_146]
                                       PartitionCols:_col0
-                                      Select Operator [SEL_147] (rows=37584000 width=15)
+                                      Select Operator [SEL_145] (rows=37584000 width=15)
                                         Output:["_col0","_col1","_col2","_col3"]
                                         TableScan [TS_0] (rows=37584000 width=15)
                                           default@inventory,inventory,Tbl:COMPLETE,Col:COMPLETE,Output:["inv_date_sk","inv_item_sk","inv_warehouse_sk","inv_quantity_on_hand"]
                                   <-Map 10 [SIMPLE_EDGE] vectorized
-                                    SHUFFLE [RS_154]
+                                    SHUFFLE [RS_152]
                                       PartitionCols:_col0
-                                      Select Operator [SEL_152] (rows=50 width=4)
+                                      Select Operator [SEL_150] (rows=50 width=4)
                                         Output:["_col0"]
-                                        Filter Operator [FIL_150] (rows=50 width=12)
-                                          predicate:((d_moy = 4) and (d_year = 1999))
+                                        Filter Operator [FIL_148] (rows=50 width=12)
+                                          predicate:((d_moy = 5) and (d_year = 1999))
                                           TableScan [TS_2] (rows=73049 width=12)
                                             default@date_dim,date_dim,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"]
-            <-Reducer 9 [SIMPLE_EDGE] vectorized
-              SHUFFLE [RS_168]
-                PartitionCols:_col0, _col1
-                Select Operator [SEL_167] (rows=18049 width=24)
-                  Output:["_col0","_col1","_col2","_col3"]
-                  Filter Operator [FIL_166] (rows=18049 width=40)
-                    predicate:CASE WHEN (((UDFToDouble(_col3) / _col4) = 0)) THEN (false) ELSE (((power(((_col5 - ((_col6 * _col6) / _col4)) / CASE WHEN ((_col4 = 1L)) THEN (null) ELSE ((_col4 - 1)) END), 0.5) / (UDFToDouble(_col3) / _col4)) > 1.0D)) END
-                    Select Operator [SEL_165] (rows=36099 width=40)
-                      Output:["_col0","_col1","_col3","_col4","_col5","_col6"]
-                      Group By Operator [GBY_164] (rows=36099 width=140)
-                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2
+              <-Reducer 9 [ONE_TO_ONE_EDGE] vectorized
+                FORWARD [RS_164]
+                  PartitionCols:_col0, _col1
+                  Select Operator [SEL_163] (rows=859 width=24)
+                    Output:["_col0","_col1","_col2","_col3"]
+                    Filter Operator [FIL_162] (rows=859 width=40)
+                      predicate:CASE WHEN (((UDFToDouble(_col2) / _col3) = 0)) THEN (false) ELSE (((power(((_col4 - ((_col5 * _col5) / _col3)) / CASE WHEN ((_col3 = 1L)) THEN (null) ELSE ((_col3 - 1)) END), 0.5) / (UDFToDouble(_col2) / _col3)) > 1.0D)) END
+                      Group By Operator [GBY_161] (rows=1719 width=40)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1
                       <-Reducer 8 [SIMPLE_EDGE]
                         SHUFFLE [RS_36]
-                          PartitionCols:_col0, _col1, _col2
-                          Group By Operator [GBY_35] (rows=36099 width=140)
-                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col3)","count(_col3)","sum(_col5)","sum(_col4)"],keys:_col1, _col2, _col0
-                            Select Operator [SEL_33] (rows=1032514 width=108)
-                              Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                              Merge Join Operator [MERGEJOIN_145] (rows=1032514 width=108)
-                                Conds:RS_30._col1=RS_158._col0(Inner),Output:["_col2","_col3","_col5","_col6"]
+                          PartitionCols:_col0, _col1
+                          Group By Operator [GBY_35] (rows=1719 width=40)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col3)","count(_col3)","sum(_col5)","sum(_col4)"],keys:_col1, _col2
+                            Select Operator [SEL_33] (rows=1032514 width=8)
+                              Output:["_col1","_col2","_col3","_col4","_col5"]
+                              Merge Join Operator [MERGEJOIN_143] (rows=1032514 width=8)
+                                Conds:RS_30._col1=RS_156._col0(Inner),Output:["_col2","_col3","_col5"]
                               <-Map 11 [SIMPLE_EDGE] vectorized
-                                SHUFFLE [RS_158]
+                                SHUFFLE [RS_156]
                                   PartitionCols:_col0
-                                   Please refer to the previous Select Operator [SEL_156]
+                                   Please refer to the previous Select Operator [SEL_154]
                               <-Reducer 7 [SIMPLE_EDGE]
                                 SHUFFLE [RS_30]
                                   PartitionCols:_col1
-                                  Merge Join Operator [MERGEJOIN_144] (rows=1032514 width=8)
-                                    Conds:RS_149._col0=RS_155._col0(Inner),Output:["_col1","_col2","_col3"]
+                                  Merge Join Operator [MERGEJOIN_142] (rows=1032514 width=8)
+                                    Conds:RS_147._col0=RS_153._col0(Inner),Output:["_col1","_col2","_col3"]
                                   <-Map 1 [SIMPLE_EDGE] vectorized
-                                    SHUFFLE [RS_149]
+                                    SHUFFLE [RS_147]
                                       PartitionCols:_col0
-                                       Please refer to the previous Select Operator [SEL_147]
+                                       Please refer to the previous Select Operator [SEL_145]
                                   <-Map 10 [SIMPLE_EDGE] vectorized
-                                    SHUFFLE [RS_155]
+                                    SHUFFLE [RS_153]
                                       PartitionCols:_col0
-                                      Select Operator [SEL_153] (rows=50 width=4)
+                                      Select Operator [SEL_151] (rows=50 width=4)
                                         Output:["_col0"]
-                                        Filter Operator [FIL_151] (rows=50 width=12)
-                                          predicate:((d_moy = 5) and (d_year = 1999))
+                                        Filter Operator [FIL_149] (rows=50 width=12)
+                                          predicate:((d_moy = 4) and (d_year = 1999))
                                            Please refer to the previous TableScan [TS_2]
 
 PREHOOK: query: with inv as
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query64.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query64.q.out
index 30f9b6b..3a37902 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query64.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query64.q.out
@@ -265,58 +265,58 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Map 1 <- Reducer 48 (BROADCAST_EDGE)
-Map 38 <- Reducer 20 (BROADCAST_EDGE)
-Map 52 <- Reducer 49 (BROADCAST_EDGE)
-Map 53 <- Reducer 36 (BROADCAST_EDGE)
-Reducer 10 <- Map 47 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 11 <- Map 47 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE)
-Reducer 12 <- Map 50 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE)
-Reducer 13 <- Map 50 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE)
-Reducer 14 <- Map 51 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
-Reducer 15 <- Map 51 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE)
+Map 1 <- Reducer 45 (BROADCAST_EDGE)
+Map 35 <- Reducer 20 (BROADCAST_EDGE)
+Map 50 <- Reducer 47 (BROADCAST_EDGE)
+Map 54 <- Reducer 53 (BROADCAST_EDGE)
+Reducer 10 <- Map 44 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Map 44 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE)
+Reducer 12 <- Map 48 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE)
+Reducer 13 <- Map 48 (SIMPLE_EDGE), Reducer 12 (SIMPLE_EDGE)
+Reducer 14 <- Map 49 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
+Reducer 15 <- Map 49 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE)
 Reducer 16 <- Reducer 15 (SIMPLE_EDGE)
-Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 35 (SIMPLE_EDGE)
+Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 34 (SIMPLE_EDGE)
 Reducer 18 <- Reducer 17 (SIMPLE_EDGE)
 Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 19 (SIMPLE_EDGE)
 Reducer 20 <- Map 19 (CUSTOM_SIMPLE_EDGE)
-Reducer 21 <- Map 19 (SIMPLE_EDGE), Map 52 (SIMPLE_EDGE)
-Reducer 22 <- Map 47 (SIMPLE_EDGE), Reducer 21 (SIMPLE_EDGE)
-Reducer 23 <- Map 37 (SIMPLE_EDGE), Reducer 22 (SIMPLE_EDGE)
-Reducer 24 <- Reducer 23 (SIMPLE_EDGE), Reducer 43 (ONE_TO_ONE_EDGE)
-Reducer 25 <- Map 44 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
-Reducer 26 <- Map 45 (SIMPLE_EDGE), Reducer 25 (SIMPLE_EDGE)
-Reducer 27 <- Map 46 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE)
-Reducer 28 <- Map 46 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE)
-Reducer 29 <- Map 47 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE)
-Reducer 3 <- Map 47 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 30 <- Map 47 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE)
-Reducer 31 <- Map 50 (SIMPLE_EDGE), Reducer 30 (SIMPLE_EDGE)
-Reducer 32 <- Map 50 (SIMPLE_EDGE), Reducer 31 (SIMPLE_EDGE)
-Reducer 33 <- Map 51 (SIMPLE_EDGE), Reducer 32 (SIMPLE_EDGE)
-Reducer 34 <- Map 51 (SIMPLE_EDGE), Reducer 33 (SIMPLE_EDGE)
-Reducer 35 <- Reducer 34 (SIMPLE_EDGE)
-Reducer 36 <- Map 19 (CUSTOM_SIMPLE_EDGE)
-Reducer 39 <- Map 38 (SIMPLE_EDGE), Map 41 (SIMPLE_EDGE)
-Reducer 4 <- Map 37 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 22 <- Map 21 (SIMPLE_EDGE), Reducer 46 (SIMPLE_EDGE)
+Reducer 23 <- Reducer 22 (SIMPLE_EDGE), Reducer 40 (ONE_TO_ONE_EDGE)
+Reducer 24 <- Map 41 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE)
+Reducer 25 <- Map 42 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
+Reducer 26 <- Map 43 (SIMPLE_EDGE), Reducer 25 (SIMPLE_EDGE)
+Reducer 27 <- Map 43 (SIMPLE_EDGE), Reducer 26 (SIMPLE_EDGE)
+Reducer 28 <- Map 44 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE)
+Reducer 29 <- Map 44 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE)
+Reducer 3 <- Map 44 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 30 <- Map 48 (SIMPLE_EDGE), Reducer 29 (SIMPLE_EDGE)
+Reducer 31 <- Map 48 (SIMPLE_EDGE), Reducer 30 (SIMPLE_EDGE)
+Reducer 32 <- Map 49 (SIMPLE_EDGE), Reducer 31 (SIMPLE_EDGE)
+Reducer 33 <- Map 49 (SIMPLE_EDGE), Reducer 32 (SIMPLE_EDGE)
+Reducer 34 <- Reducer 33 (SIMPLE_EDGE)
+Reducer 36 <- Map 35 (SIMPLE_EDGE), Map 38 (SIMPLE_EDGE)
+Reducer 37 <- Reducer 36 (SIMPLE_EDGE)
+Reducer 39 <- Map 38 (SIMPLE_EDGE), Map 54 (SIMPLE_EDGE)
+Reducer 4 <- Map 21 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
 Reducer 40 <- Reducer 39 (SIMPLE_EDGE)
-Reducer 42 <- Map 41 (SIMPLE_EDGE), Map 53 (SIMPLE_EDGE)
-Reducer 43 <- Reducer 42 (SIMPLE_EDGE)
-Reducer 48 <- Map 47 (CUSTOM_SIMPLE_EDGE)
-Reducer 49 <- Map 47 (CUSTOM_SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 40 (ONE_TO_ONE_EDGE)
-Reducer 6 <- Map 44 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-Reducer 7 <- Map 45 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-Reducer 8 <- Map 46 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-Reducer 9 <- Map 46 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 45 <- Map 44 (CUSTOM_SIMPLE_EDGE)
+Reducer 46 <- Map 44 (SIMPLE_EDGE), Reducer 51 (SIMPLE_EDGE)
+Reducer 47 <- Map 44 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Reducer 37 (ONE_TO_ONE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 51 <- Map 50 (SIMPLE_EDGE), Map 52 (SIMPLE_EDGE)
+Reducer 53 <- Map 52 (CUSTOM_SIMPLE_EDGE)
+Reducer 6 <- Map 41 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Map 42 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+Reducer 8 <- Map 43 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+Reducer 9 <- Map 43 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
       Reducer 18 vectorized
-      File Output Operator [FS_1069]
-        Select Operator [SEL_1068] (rows=104583667777 width=1702)
+      File Output Operator [FS_1071]
+        Select Operator [SEL_1070] (rows=104583667777 width=1702)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20"]
         <-Reducer 17 [SIMPLE_EDGE]
           SHUFFLE [RS_197]
@@ -325,13 +325,13 @@ Stage-0
               Filter Operator [FIL_195] (rows=104583667777 width=1694)
                 predicate:(_col19 <= _col12)
                 Merge Join Operator [MERGEJOIN_973] (rows=313751003333 width=1694)
-                  Conds:RS_1050._col2, _col1, _col3=RS_1067._col1, _col0, _col2(Inner),Output:["_col0","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col19","_col20","_col21","_col22"]
+                  Conds:RS_1046._col2, _col1, _col3=RS_1069._col1, _col0, _col2(Inner),Output:["_col0","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col19","_col20","_col21","_col22"]
                 <-Reducer 16 [SIMPLE_EDGE] vectorized
-                  SHUFFLE [RS_1050]
+                  SHUFFLE [RS_1046]
                     PartitionCols:_col2, _col1, _col3
-                    Select Operator [SEL_1049] (rows=21299858 width=1354)
+                    Select Operator [SEL_1045] (rows=21299858 width=1354)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15"]
-                      Group By Operator [GBY_1048] (rows=21299858 width=1362)
+                      Group By Operator [GBY_1044] (rows=21299858 width=1362)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, KEY._col12, KEY._col13
                       <-Reducer 15 [SIMPLE_EDGE]
                         SHUFFLE [RS_93]
@@ -339,11 +339,11 @@ Stage-0
                           Group By Operator [GBY_92] (rows=21299858 width=1362)
                             Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count()","sum(_col8)","sum(_col9)","sum(_col10)"],keys:_col24, _col11, _col25, _col12, _col29, _col31, _col37, _col38, _col39, _col40, _col42, _col43, _col44, _col45
                             Merge Join Operator [MERGEJOIN_957] (rows=21299858 width=1155)
-                              Conds:RS_88._col17=RS_1045._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col12","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40","_col42","_col43","_col44","_col45"]
-                            <-Map 51 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_1045]
+                              Conds:RS_88._col17=RS_1041._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col12","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40","_col42","_col43","_col44","_col45"]
+                            <-Map 49 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_1041]
                                 PartitionCols:_col0
-                                Select Operator [SEL_1043] (rows=40000000 width=365)
+                                Select Operator [SEL_1039] (rows=40000000 width=365)
                                   Output:["_col0","_col1","_col2","_col3","_col4"]
                                   TableScan [TS_44] (rows=40000000 width=365)
                                     default@customer_address,ad1,Tbl:COMPLETE,Col:COMPLETE,Output:["ca_address_sk","ca_street_number","ca_street_name","ca_city","ca_zip"]
@@ -351,22 +351,22 @@ Stage-0
                               SHUFFLE [RS_88]
                                 PartitionCols:_col17
                                 Merge Join Operator [MERGEJOIN_956] (rows=21299858 width=798)
-                                  Conds:RS_85._col5=RS_1044._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col12","_col17","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40"]
-                                <-Map 51 [SIMPLE_EDGE] vectorized
-                                  SHUFFLE [RS_1044]
+                                  Conds:RS_85._col5=RS_1040._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col12","_col17","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40"]
+                                <-Map 49 [SIMPLE_EDGE] vectorized
+                                  SHUFFLE [RS_1040]
                                     PartitionCols:_col0
-                                     Please refer to the previous Select Operator [SEL_1043]
+                                     Please refer to the previous Select Operator [SEL_1039]
                                 <-Reducer 13 [SIMPLE_EDGE]
                                   SHUFFLE [RS_85]
                                     PartitionCols:_col5
                                     Filter Operator [FIL_84] (rows=21299858 width=609)
                                       predicate:(_col33 <> _col35)
                                       Merge Join Operator [MERGEJOIN_955] (rows=21299858 width=609)
-                                        Conds:RS_81._col15=RS_1040._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col12","_col17","_col24","_col25","_col29","_col31","_col33","_col35"]
-                                      <-Map 50 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_1040]
+                                        Conds:RS_81._col15=RS_1036._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col12","_col17","_col24","_col25","_col29","_col31","_col33","_col35"]
+                                      <-Map 48 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_1036]
                                           PartitionCols:_col0
-                                          Select Operator [SEL_1038] (rows=1861800 width=89)
+                                          Select Operator [SEL_1034] (rows=1861800 width=89)
                                             Output:["_col0","_col1"]
                                             TableScan [TS_40] (rows=1861800 width=89)
                                               default@customer_demographics,cd1,Tbl:COMPLETE,Col:COMPLETE,Output:["cd_demo_sk","cd_marital_status"]
@@ -374,17 +374,17 @@ Stage-0
                                         SHUFFLE [RS_81]
                                           PartitionCols:_col15
                                           Merge Join Operator [MERGEJOIN_954] (rows=21002853 width=525)
-                                            Conds:RS_78._col3=RS_1039._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col24","_col25","_col29","_col31","_col33"]
-                                          <-Map 50 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_1039]
+                                            Conds:RS_78._col3=RS_1035._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col24","_col25","_col29","_col31","_col33"]
+                                          <-Map 48 [SIMPLE_EDGE] vectorized
+                                            SHUFFLE [RS_1035]
                                               PartitionCols:_col0
-                                               Please refer to the previous Select Operator [SEL_1038]
+                                               Please refer to the previous Select Operator [SEL_1034]
                                           <-Reducer 11 [SIMPLE_EDGE]
                                             SHUFFLE [RS_78]
                                               PartitionCols:_col3
                                               Merge Join Operator [MERGEJOIN_953] (rows=20709989 width=438)
                                                 Conds:RS_75._col18=RS_980._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col24","_col25","_col29","_col31"]
-                                              <-Map 47 [SIMPLE_EDGE] vectorized
+                                              <-Map 44 [SIMPLE_EDGE] vectorized
                                                 PARTITION_ONLY_SHUFFLE [RS_980]
                                                   PartitionCols:_col0
                                                   Select Operator [SEL_974] (rows=73049 width=8)
@@ -396,7 +396,7 @@ Stage-0
                                                   PartitionCols:_col18
                                                   Merge Join Operator [MERGEJOIN_952] (rows=20709989 width=438)
                                                     Conds:RS_72._col19=RS_982._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col18","_col24","_col25","_col29"]
-                                                  <-Map 47 [SIMPLE_EDGE] vectorized
+                                                  <-Map 44 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_982]
                                                       PartitionCols:_col0
                                                       Select Operator [SEL_976] (rows=73049 width=8)
@@ -406,13 +406,13 @@ Stage-0
                                                     SHUFFLE [RS_72]
                                                       PartitionCols:_col19
                                                       Merge Join Operator [MERGEJOIN_951] (rows=20709989 width=437)
-                                                        Conds:RS_69._col16=RS_1035._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col18","_col19","_col24","_col25"]
-                                                      <-Map 46 [SIMPLE_EDGE] vectorized
-                                                        SHUFFLE [RS_1035]
+                                                        Conds:RS_69._col16=RS_1031._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col18","_col19","_col24","_col25"]
+                                                      <-Map 43 [SIMPLE_EDGE] vectorized
+                                                        SHUFFLE [RS_1031]
                                                           PartitionCols:_col0
-                                                          Select Operator [SEL_1033] (rows=7200 width=4)
+                                                          Select Operator [SEL_1029] (rows=7200 width=4)
                                                             Output:["_col0"]
-                                                            Filter Operator [FIL_1032] (rows=7200 width=8)
+                                                            Filter Operator [FIL_1028] (rows=7200 width=8)
                                                               predicate:hd_income_band_sk is not null
                                                               TableScan [TS_30] (rows=7200 width=8)
                                                                 default@household_demographics,hd1,Tbl:COMPLETE,Col:COMPLETE,Output:["hd_demo_sk","hd_income_band_sk"]
@@ -420,22 +420,22 @@ Stage-0
                                                         SHUFFLE [RS_69]
                                                           PartitionCols:_col16
                                                           Merge Join Operator [MERGEJOIN_950] (rows=20709989 width=441)
-                                                            Conds:RS_66._col4=RS_1034._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
-                                                          <-Map 46 [SIMPLE_EDGE] vectorized
-                                                            SHUFFLE [RS_1034]
+                                                            Conds:RS_66._col4=RS_1030._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
+                                                          <-Map 43 [SIMPLE_EDGE] vectorized
+                                                            SHUFFLE [RS_1030]
                                                               PartitionCols:_col0
-                                                               Please refer to the previous Select Operator [SEL_1033]
+                                                               Please refer to the previous Select Operator [SEL_1029]
                                                           <-Reducer 7 [SIMPLE_EDGE]
                                                             SHUFFLE [RS_66]
                                                               PartitionCols:_col4
                                                               Merge Join Operator [MERGEJOIN_949] (rows=20709989 width=443)
-                                                                Conds:RS_63._col6=RS_1030._col0(Inner),Output:["_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
-                                                              <-Map 45 [SIMPLE_EDGE] vectorized
-                                                                SHUFFLE [RS_1030]
+                                                                Conds:RS_63._col6=RS_1026._col0(Inner),Output:["_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
+                                                              <-Map 42 [SIMPLE_EDGE] vectorized
+                                                                SHUFFLE [RS_1026]
                                                                   PartitionCols:_col0
-                                                                  Select Operator [SEL_1029] (rows=1704 width=181)
+                                                                  Select Operator [SEL_1025] (rows=1704 width=181)
                                                                     Output:["_col0","_col1","_col2"]
-                                                                    Filter Operator [FIL_1028] (rows=1704 width=181)
+                                                                    Filter Operator [FIL_1024] (rows=1704 width=181)
                                                                       predicate:(s_store_name is not null and s_zip is not null)
                                                                       TableScan [TS_27] (rows=1704 width=181)
                                                                         default@store,store,Tbl:COMPLETE,Col:COMPLETE,Output:["s_store_sk","s_store_name","s_zip"]
@@ -443,11 +443,11 @@ Stage-0
                                                                 SHUFFLE [RS_63]
                                                                   PartitionCols:_col6
                                                                   Merge Join Operator [MERGEJOIN_948] (rows=20709989 width=267)
-                                                                    Conds:RS_60._col1, _col7=RS_1026._col0, _col1(Inner),Output:["_col3","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
-                                                                  <-Map 44 [SIMPLE_EDGE] vectorized
-                                                                    SHUFFLE [RS_1026]
+                                                                    Conds:RS_60._col1, _col7=RS_1022._col0, _col1(Inner),Output:["_col3","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
+                                                                  <-Map 41 [SIMPLE_EDGE] vectorized
+                                                                    SHUFFLE [RS_1022]
                                                                       PartitionCols:_col0, _col1
-                                                                      Select Operator [SEL_1025] (rows=57591150 width=8)
+                                                                      Select Operator [SEL_1021] (rows=57591150 width=8)
                                                                         Output:["_col0","_col1"]
                                                                         TableScan [TS_25] (rows=57591150 width=8)
                                                                           default@store_returns,store_returns,Tbl:COMPLETE,Col:COMPLETE,Output:["sr_item_sk","sr_ticket_number"]
@@ -455,18 +455,66 @@ Stage-0
                                                                     SHUFFLE [RS_60]
                                                                       PartitionCols:_col1, _col7
                                                                       Merge Join Operator [MERGEJOIN_947] (rows=12561347 width=135)
-                                                                        Conds:RS_57._col1=RS_1024._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
+                                                                        Conds:RS_57._col1=RS_1020._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
+                                                                      <-Reducer 37 [ONE_TO_ONE_EDGE] vectorized
+                                                                        FORWARD [RS_1020]
+                                                                          PartitionCols:_col0
+                                                                          Select Operator [SEL_1019] (rows=13257 width=4)
+                                                                            Output:["_col0"]
+                                                                            Filter Operator [FIL_1018] (rows=13257 width=228)
+                                                                              predicate:(_col1 > (2 * _col2))
+                                                                              Group By Operator [GBY_1017] (rows=39773 width=228)
+                                                                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
+                                                                              <-Reducer 36 [SIMPLE_EDGE]
+                                                                                SHUFFLE [RS_21]
+                                                                                  PartitionCols:_col0
+                                                                                  Group By Operator [GBY_20] (rows=6482999 width=228)
+                                                                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","sum(_col5)"],keys:_col0
+                                                                                    Merge Join Operator [MERGEJOIN_946] (rows=183085709 width=227)
+                                                                                      Conds:RS_1013._col0, _col1=RS_1015._col0, _col1(Inner),Output:["_col0","_col2","_col5"]
+                                                                                    <-Map 38 [SIMPLE_EDGE] vectorized
+                                                                                      SHUFFLE [RS_1015]
+                                                                                        PartitionCols:_col0, _col1
+                                                                                        Select Operator [SEL_1014] (rows=28798881 width=120)
+                                                                                          Output:["_col0","_col1","_col2"]
+                                                                                          TableScan [TS_14] (rows=28798881 width=337)
+                                                                                            default@catalog_returns,catalog_returns,Tbl:COMPLETE,Col:COMPLETE,Output:["cr_item_sk","cr_order_number","cr_refunded_cash","cr_reversed_charge","cr_store_credit"]
+                                                                                    <-Map 35 [SIMPLE_EDGE] vectorized
+                                                                                      SHUFFLE [RS_1013]
+                                                                                        PartitionCols:_col0, _col1
+                                                                                        Select Operator [SEL_1012] (rows=287989836 width=119)
+                                                                                          Output:["_col0","_col1","_col2"]
+                                                                                          Filter Operator [FIL_1011] (rows=287989836 width=119)
+                                                                                            predicate:(cs_item_sk BETWEEN DynamicValue(RS_49_item_i_item_sk_min) AND DynamicValue(RS_49_item_i_item_sk_max) and in_bloom_filter(cs_item_sk, DynamicValue(RS_49_item_i_item_sk_bloom_filter)))
+                                                                                            TableScan [TS_12] (rows=287989836 width=119)
+                                                                                              default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_item_sk","cs_order_number","cs_ext_list_price"]
+                                                                                            <-Reducer 20 [BROADCAST_EDGE] vectorized
+                                                                                              BROADCAST [RS_1010]
+                                                                                                Group By Operator [GBY_1009] (rows=1 width=12)
+                                                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
+                                                                                                <-Map 19 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                                                  PARTITION_ONLY_SHUFFLE [RS_1004]
+                                                                                                    Group By Operator [GBY_1003] (rows=1 width=12)
+                                                                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
+                                                                                                      Select Operator [SEL_1002] (rows=4666 width=4)
+                                                                                                        Output:["_col0"]
+                                                                                                        Select Operator [SEL_1000] (rows=4666 width=111)
+                                                                                                          Output:["_col0","_col1"]
+                                                                                                          Filter Operator [FIL_999] (rows=4666 width=311)
+                                                                                                            predicate:((i_color) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate') and i_current_price BETWEEN 36 AND 45)
+                                                                                                            TableScan [TS_3] (rows=462000 width=311)
+                                                                                                              default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_current_price","i_color","i_product_name"]
                                                                       <-Reducer 4 [SIMPLE_EDGE]
                                                                         SHUFFLE [RS_57]
                                                                           PartitionCols:_col1
                                                                           Merge Join Operator [MERGEJOIN_945] (rows=12561347 width=135)
-                                                                            Conds:RS_54._col2=RS_1011._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
-                                                                          <-Map 37 [SIMPLE_EDGE] vectorized
-                                                                            SHUFFLE [RS_1011]
+                                                                            Conds:RS_54._col2=RS_1007._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
+                                                                          <-Map 21 [SIMPLE_EDGE] vectorized
+                                                                            SHUFFLE [RS_1007]
                                                                               PartitionCols:_col0
-                                                                              Select Operator [SEL_1010] (rows=69376329 width=23)
+                                                                              Select Operator [SEL_1006] (rows=69376329 width=23)
                                                                                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-                                                                                Filter Operator [FIL_1009] (rows=69376329 width=23)
+                                                                                Filter Operator [FIL_1005] (rows=69376329 width=23)
                                                                                   predicate:(c_current_addr_sk is not null and c_current_cdemo_sk is not null and c_current_hdemo_sk is not null and c_first_sales_date_sk is not null and c_first_shipto_date_sk is not null)
                                                                                   TableScan [TS_9] (rows=80000000 width=23)
                                                                                     default@customer,customer,Tbl:COMPLETE,Col:COMPLETE,Output:["c_customer_sk","c_current_cdemo_sk","c_current_hdemo_sk","c_current_addr_sk","c_first_shipto_date_sk","c_first_sales_date_sk"]
@@ -475,7 +523,7 @@ Stage-0
                                                                               PartitionCols:_col2
                                                                               Merge Join Operator [MERGEJOIN_944] (rows=14484878 width=119)
                                                                                 Conds:RS_51._col0=RS_986._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"]
-                                                                              <-Map 47 [SIMPLE_EDGE] vectorized
+                                                                              <-Map 44 [SIMPLE_EDGE] vectorized
                                                                                 PARTITION_ONLY_SHUFFLE [RS_986]
                                                                                   PartitionCols:_col0
                                                                                   Select Operator [SEL_981] (rows=652 width=4)
@@ -491,12 +539,7 @@ Stage-0
                                                                                   <-Map 19 [SIMPLE_EDGE] vectorized
                                                                                     PARTITION_ONLY_SHUFFLE [RS_1001]
                                                                                       PartitionCols:_col0
-                                                                                      Select Operator [SEL_1000] (rows=4666 width=111)
-                                                                                        Output:["_col0","_col1"]
-                                                                                        Filter Operator [FIL_999] (rows=4666 width=311)
-                                                                                          predicate:((i_color) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate') and i_current_price BETWEEN 36 AND 45)
-                                                                                          TableScan [TS_3] (rows=462000 width=311)
-                                                                                            default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_current_price","i_color","i_product_name"]
+                                                                                       Please refer to the previous Select Operator [SEL_1000]
                                                                                   <-Map 1 [SIMPLE_EDGE] vectorized
                                                                                     SHUFFLE [RS_998]
                                                                                       PartitionCols:_col1
@@ -506,185 +549,142 @@ Stage-0
                                                                                           predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_52_d1_d_date_sk_min) AND DynamicValue(RS_52_d1_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_52_d1_d_date_sk_bloom_filter))) and ss_addr_sk is not null and ss_cdemo_sk is not null and ss_customer_sk is not null and ss_hdemo_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_0] (rows=575995635 width=355)
                                                                                             default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_cdemo_sk","ss_hdemo_sk","ss_addr_sk","ss_store_sk","ss_promo_sk","ss_ticket_number","ss_wholesale_cost","ss_list_price","ss_coupon_amt"]
-                                                                                          <-Reducer 48 [BROADCAST_EDGE] vectorized
+                                                                                          <-Reducer 45 [BROADCAST_EDGE] vectorized
                                                                                             BROADCAST [RS_995]
                                                                                               Group By Operator [GBY_994] (rows=1 width=12)
                                                                                                 Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                                              <-Map 47 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                                              <-Map 44 [CUSTOM_SIMPLE_EDGE] vectorized
                                                                                                 PARTITION_ONLY_SHUFFLE [RS_992]
                                                                                                   Group By Operator [GBY_990] (rows=1 width=12)
                                                                                                     Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
                                                                                                     Select Operator [SEL_987] (rows=652 width=4)
                                                                                                       Output:["_col0"]
                                                                                                        Please refer to the previous Select Operator [SEL_981]
-                                                                      <-Reducer 40 [ONE_TO_ONE_EDGE] vectorized
-                                                                        FORWARD [RS_1024]
-                                                                          PartitionCols:_col0
-                                                                          Select Operator [SEL_1023] (rows=13257 width=4)
-                                                                            Output:["_col0"]
-                                                                            Filter Operator [FIL_1022] (rows=13257 width=228)
-                                                                              predicate:(_col1 > (2 * _col2))
-                                                                              Group By Operator [GBY_1021] (rows=39773 width=228)
-                                                                                Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
-                                                                              <-Reducer 39 [SIMPLE_EDGE]
-                                                                                SHUFFLE [RS_21]
-                                                                                  PartitionCols:_col0
-                                                                                  Group By Operator [GBY_20] (rows=6482999 width=228)
-                                                                                    Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","sum(_col5)"],keys:_col0
-                                                                                    Merge Join Operator [MERGEJOIN_946] (rows=183085709 width=227)
-                                                                                      Conds:RS_1017._col0, _col1=RS_1019._col0, _col1(Inner),Output:["_col0","_col2","_col5"]
-                                                                                    <-Map 41 [SIMPLE_EDGE] vectorized
-                                                                                      SHUFFLE [RS_1019]
-                                                                                        PartitionCols:_col0, _col1
-                                                                                        Select Operator [SEL_1018] (rows=28798881 width=120)
-                                                                                          Output:["_col0","_col1","_col2"]
-                                                                                          TableScan [TS_14] (rows=28798881 width=337)
-                                                                                            default@catalog_returns,catalog_returns,Tbl:COMPLETE,Col:COMPLETE,Output:["cr_item_sk","cr_order_number","cr_refunded_cash","cr_reversed_charge","cr_store_credit"]
-                                                                                    <-Map 38 [SIMPLE_EDGE] vectorized
-                                                                                      SHUFFLE [RS_1017]
-                                                                                        PartitionCols:_col0, _col1
-                                                                                        Select Operator [SEL_1016] (rows=287989836 width=119)
-                                                                                          Output:["_col0","_col1","_col2"]
-                                                                                          Filter Operator [FIL_1015] (rows=287989836 width=119)
-                                                                                            predicate:(cs_item_sk BETWEEN DynamicValue(RS_49_item_i_item_sk_min) AND DynamicValue(RS_49_item_i_item_sk_max) and in_bloom_filter(cs_item_sk, DynamicValue(RS_49_item_i_item_sk_bloom_filter)))
-                                                                                            TableScan [TS_12] (rows=287989836 width=119)
-                                                                                              default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_item_sk","cs_order_number","cs_ext_list_price"]
-                                                                                            <-Reducer 20 [BROADCAST_EDGE] vectorized
-                                                                                              BROADCAST [RS_1014]
-                                                                                                Group By Operator [GBY_1013] (rows=1 width=12)
-                                                                                                  Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                                                <-Map 19 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                                                  PARTITION_ONLY_SHUFFLE [RS_1007]
-                                                                                                    Group By Operator [GBY_1005] (rows=1 width=12)
-                                                                                                      Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                                                      Select Operator [SEL_1002] (rows=4666 width=4)
-                                                                                                        Output:["_col0"]
-                                                                                                         Please refer to the previous Select Operator [SEL_1000]
-                <-Reducer 35 [SIMPLE_EDGE] vectorized
-                  SHUFFLE [RS_1067]
+                <-Reducer 34 [SIMPLE_EDGE] vectorized
+                  SHUFFLE [RS_1069]
                     PartitionCols:_col1, _col0, _col2
-                    Select Operator [SEL_1066] (rows=21299858 width=525)
+                    Select Operator [SEL_1068] (rows=21299858 width=525)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
-                      Group By Operator [GBY_1065] (rows=21299858 width=1362)
-                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, KEY._col12, KEY._col13
-                      <-Reducer 34 [SIMPLE_EDGE]
+                      Group By Operator [GBY_1067] (rows=21299858 width=1255)
+                        Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, KEY._col12
+                      <-Reducer 33 [SIMPLE_EDGE]
                         SHUFFLE [RS_189]
-                          PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
-                          Group By Operator [GBY_188] (rows=21299858 width=1362)
-                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count()","sum(_col8)","sum(_col9)","sum(_col10)"],keys:_col24, _col11, _col25, _col12, _col29, _col31, _col37, _col38, _col39, _col40, _col42, _col43, _col44, _col45
-                            Merge Join Operator [MERGEJOIN_972] (rows=21299858 width=1155)
-                              Conds:RS_184._col17=RS_1047._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col12","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40","_col42","_col43","_col44","_col45"]
-                            <-Map 51 [SIMPLE_EDGE] vectorized
-                              SHUFFLE [RS_1047]
+                          PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+                          Group By Operator [GBY_188] (rows=21299858 width=1255)
+                            Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16"],aggregations:["count()","sum(_col8)","sum(_col9)","sum(_col10)"],keys:_col24, _col11, _col25, _col29, _col31, _col37, _col38, _col39, _col40, _col42, _col43, _col44, _col45
+                            Merge Join Operator [MERGEJOIN_972] (rows=21299858 width=1048)
+                              Conds:RS_184._col17=RS_1043._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40","_col42","_col43","_col44","_col45"]
+                            <-Map 49 [SIMPLE_EDGE] vectorized
+                              SHUFFLE [RS_1043]
                                 PartitionCols:_col0
-                                 Please refer to the previous Select Operator [SEL_1043]
-                            <-Reducer 33 [SIMPLE_EDGE]
+                                 Please refer to the previous Select Operator [SEL_1039]
+                            <-Reducer 32 [SIMPLE_EDGE]
                               SHUFFLE [RS_184]
                                 PartitionCols:_col17
-                                Merge Join Operator [MERGEJOIN_971] (rows=21299858 width=798)
-                                  Conds:RS_181._col5=RS_1046._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col12","_col17","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40"]
-                                <-Map 51 [SIMPLE_EDGE] vectorized
-                                  SHUFFLE [RS_1046]
+                                Merge Join Operator [MERGEJOIN_971] (rows=21299858 width=691)
+                                  Conds:RS_181._col5=RS_1042._col0(Inner),Output:["_col8","_col9","_col10","_col11","_col17","_col24","_col25","_col29","_col31","_col37","_col38","_col39","_col40"]
+                                <-Map 49 [SIMPLE_EDGE] vectorized
+                                  SHUFFLE [RS_1042]
                                     PartitionCols:_col0
-                                     Please refer to the previous Select Operator [SEL_1043]
-                                <-Reducer 32 [SIMPLE_EDGE]
+                                     Please refer to the previous Select Operator [SEL_1039]
+                                <-Reducer 31 [SIMPLE_EDGE]
                                   SHUFFLE [RS_181]
                                     PartitionCols:_col5
-                                    Filter Operator [FIL_180] (rows=21299858 width=609)
+                                    Filter Operator [FIL_180] (rows=21299858 width=502)
                                       predicate:(_col33 <> _col35)
-                                      Merge Join Operator [MERGEJOIN_970] (rows=21299858 width=609)
-                                        Conds:RS_177._col15=RS_1042._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col12","_col17","_col24","_col25","_col29","_col31","_col33","_col35"]
-                                      <-Map 50 [SIMPLE_EDGE] vectorized
-                                        SHUFFLE [RS_1042]
+                                      Merge Join Operator [MERGEJOIN_970] (rows=21299858 width=502)
+                                        Conds:RS_177._col15=RS_1038._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col17","_col24","_col25","_col29","_col31","_col33","_col35"]
+                                      <-Map 48 [SIMPLE_EDGE] vectorized
+                                        SHUFFLE [RS_1038]
                                           PartitionCols:_col0
-                                           Please refer to the previous Select Operator [SEL_1038]
-                                      <-Reducer 31 [SIMPLE_EDGE]
+                                           Please refer to the previous Select Operator [SEL_1034]
+                                      <-Reducer 30 [SIMPLE_EDGE]
                                         SHUFFLE [RS_177]
                                           PartitionCols:_col15
-                                          Merge Join Operator [MERGEJOIN_969] (rows=21002853 width=525)
-                                            Conds:RS_174._col3=RS_1041._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col24","_col25","_col29","_col31","_col33"]
-                                          <-Map 50 [SIMPLE_EDGE] vectorized
-                                            SHUFFLE [RS_1041]
+                                          Merge Join Operator [MERGEJOIN_969] (rows=21002853 width=418)
+                                            Conds:RS_174._col3=RS_1037._col0(Inner),Output:["_col5","_col8","_col9","_col10","_col11","_col15","_col17","_col24","_col25","_col29","_col31","_col33"]
+                                          <-Map 48 [SIMPLE_EDGE] vectorized
+                                            SHUFFLE [RS_1037]
                                               PartitionCols:_col0
-                                               Please refer to the previous Select Operator [SEL_1038]
-                                          <-Reducer 30 [SIMPLE_EDGE]
+                                               Please refer to the previous Select Operator [SEL_1034]
+                                          <-Reducer 29 [SIMPLE_EDGE]
                                             SHUFFLE [RS_174]
                                               PartitionCols:_col3
-                                              Merge Join Operator [MERGEJOIN_968] (rows=20709989 width=438)
-                                                Conds:RS_171._col18=RS_984._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col24","_col25","_col29","_col31"]
-                                              <-Map 47 [SIMPLE_EDGE] vectorized
+                                              Merge Join Operator [MERGEJOIN_968] (rows=20709989 width=331)
+                                                Conds:RS_171._col18=RS_984._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col15","_col17","_col24","_col25","_col29","_col31"]
+                                              <-Map 44 [SIMPLE_EDGE] vectorized
                                                 PARTITION_ONLY_SHUFFLE [RS_984]
                                                   PartitionCols:_col0
                                                   Select Operator [SEL_978] (rows=73049 width=8)
                                                     Output:["_col0","_col1"]
                                                      Please refer to the previous TableScan [TS_38]
-                                              <-Reducer 29 [SIMPLE_EDGE]
+                                              <-Reducer 28 [SIMPLE_EDGE]
                                                 SHUFFLE [RS_171]
                                                   PartitionCols:_col18
-                                                  Merge Join Operator [MERGEJOIN_967] (rows=20709989 width=438)
-                                                    Conds:RS_168._col19=RS_983._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col18","_col24","_col25","_col29"]
-                                                  <-Map 47 [SIMPLE_EDGE] vectorized
+                                                  Merge Join Operator [MERGEJOIN_967] (rows=20709989 width=331)
+                                                    Conds:RS_168._col19=RS_983._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col15","_col17","_col18","_col24","_col25","_col29"]
+                                                  <-Map 44 [SIMPLE_EDGE] vectorized
                                                     PARTITION_ONLY_SHUFFLE [RS_983]
                                                       PartitionCols:_col0
                                                       Select Operator [SEL_977] (rows=73049 width=8)
                                                         Output:["_col0","_col1"]
                                                          Please refer to the previous TableScan [TS_38]
-                                                  <-Reducer 28 [SIMPLE_EDGE]
+                                                  <-Reducer 27 [SIMPLE_EDGE]
                                                     SHUFFLE [RS_168]
                                                       PartitionCols:_col19
-                                                      Merge Join Operator [MERGEJOIN_966] (rows=20709989 width=437)
-                                                        Conds:RS_165._col16=RS_1037._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col17","_col18","_col19","_col24","_col25"]
-                                                      <-Map 46 [SIMPLE_EDGE] vectorized
-                                                        SHUFFLE [RS_1037]
+                                                      Merge Join Operator [MERGEJOIN_966] (rows=20709989 width=330)
+                                                        Conds:RS_165._col16=RS_1033._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col15","_col17","_col18","_col19","_col24","_col25"]
+                                                      <-Map 43 [SIMPLE_EDGE] vectorized
+                                                        SHUFFLE [RS_1033]
                                                           PartitionCols:_col0
-                                                           Please refer to the previous Select Operator [SEL_1033]
-                                                      <-Reducer 27 [SIMPLE_EDGE]
+                                                           Please refer to the previous Select Operator [SEL_1029]
+                                                      <-Reducer 26 [SIMPLE_EDGE]
                                                         SHUFFLE [RS_165]
                                                           PartitionCols:_col16
-                                                          Merge Join Operator [MERGEJOIN_965] (rows=20709989 width=441)
-                                                            Conds:RS_162._col4=RS_1036._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
-                                                          <-Map 46 [SIMPLE_EDGE] vectorized
-                                                            SHUFFLE [RS_1036]
+                                                          Merge Join Operator [MERGEJOIN_965] (rows=20709989 width=334)
+                                                            Conds:RS_162._col4=RS_1032._col0(Inner),Output:["_col3","_col5","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
+                                                          <-Map 43 [SIMPLE_EDGE] vectorized
+                                                            SHUFFLE [RS_1032]
                                                               PartitionCols:_col0
-                                                               Please refer to the previous Select Operator [SEL_1033]
-                                                          <-Reducer 26 [SIMPLE_EDGE]
+                                                               Please refer to the previous Select Operator [SEL_1029]
+                                                          <-Reducer 25 [SIMPLE_EDGE]
                                                             SHUFFLE [RS_162]
                                                               PartitionCols:_col4
-                                                              Merge Join Operator [MERGEJOIN_964] (rows=20709989 width=443)
-                                                                Conds:RS_159._col6=RS_1031._col0(Inner),Output:["_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
-                                                              <-Map 45 [SIMPLE_EDGE] vectorized
-                                                                SHUFFLE [RS_1031]
+                                                              Merge Join Operator [MERGEJOIN_964] (rows=20709989 width=336)
+                                                                Conds:RS_159._col6=RS_1027._col0(Inner),Output:["_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19","_col24","_col25"]
+                                                              <-Map 42 [SIMPLE_EDGE] vectorized
+                                                                SHUFFLE [RS_1027]
                                                                   PartitionCols:_col0
-                                                                   Please refer to the previous Select Operator [SEL_1029]
-                                                              <-Reducer 25 [SIMPLE_EDGE]
+                                                                   Please refer to the previous Select Operator [SEL_1025]
+                                                              <-Reducer 24 [SIMPLE_EDGE]
                                                                 SHUFFLE [RS_159]
                                                                   PartitionCols:_col6
-                                                                  Merge Join Operator [MERGEJOIN_963] (rows=20709989 width=267)
-                                                                    Conds:RS_156._col1, _col7=RS_1027._col0, _col1(Inner),Output:["_col3","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
-                                                                  <-Map 44 [SIMPLE_EDGE] vectorized
-                                                                    SHUFFLE [RS_1027]
+                                                                  Merge Join Operator [MERGEJOIN_963] (rows=20709989 width=160)
+                                                                    Conds:RS_156._col1, _col7=RS_1023._col0, _col1(Inner),Output:["_col3","_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19"]
+                                                                  <-Map 41 [SIMPLE_EDGE] vectorized
+                                                                    SHUFFLE [RS_1023]
                                                                       PartitionCols:_col0, _col1
-                                                                       Please refer to the previous Select Operator [SEL_1025]
-                                                                  <-Reducer 24 [SIMPLE_EDGE]
+                                                                       Please refer to the previous Select Operator [SEL_1021]
+                                                                  <-Reducer 23 [SIMPLE_EDGE]
                                                                     SHUFFLE [RS_156]
                                                                       PartitionCols:_col1, _col7
-                                                                      Merge Join Operator [MERGEJOIN_962] (rows=12561347 width=135)
-                                                                        Conds:RS_153._col1=RS_1064._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
-                                                                      <-Reducer 23 [SIMPLE_EDGE]
+                                                                      Merge Join Operator [MERGEJOIN_962] (rows=12561347 width=28)
+                                                                        Conds:RS_153._col1=RS_1066._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19"]
+                                                                      <-Reducer 22 [SIMPLE_EDGE]
                                                                         SHUFFLE [RS_153]
                                                                           PartitionCols:_col1
-                                                                          Merge Join Operator [MERGEJOIN_960] (rows=12561347 width=135)
-                                                                            Conds:RS_150._col2=RS_1012._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col15","_col16","_col17","_col18","_col19"]
-                                                                          <-Map 37 [SIMPLE_EDGE] vectorized
-                                                                            SHUFFLE [RS_1012]
+                                                                          Merge Join Operator [MERGEJOIN_960] (rows=12561347 width=28)
+                                                                            Conds:RS_150._col2=RS_1008._col0(Inner),Output:["_col1","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col19"]
+                                                                          <-Map 21 [SIMPLE_EDGE] vectorized
+                                                                            SHUFFLE [RS_1008]
                                                                               PartitionCols:_col0
-                                                                               Please refer to the previous Select Operator [SEL_1010]
-                                                                          <-Reducer 22 [SIMPLE_EDGE]
+                                                                               Please refer to the previous Select Operator [SEL_1006]
+                                                                          <-Reducer 46 [SIMPLE_EDGE]
                                                                             SHUFFLE [RS_150]
                                                                               PartitionCols:_col2
-                                                                              Merge Join Operator [MERGEJOIN_959] (rows=14484878 width=119)
-                                                                                Conds:RS_147._col0=RS_988._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"]
-                                                                              <-Map 47 [SIMPLE_EDGE] vectorized
+                                                                              Merge Join Operator [MERGEJOIN_959] (rows=14484878 width=12)
+                                                                                Conds:RS_147._col0=RS_988._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
+                                                                              <-Map 44 [SIMPLE_EDGE] vectorized
                                                                                 PARTITION_ONLY_SHUFFLE [RS_988]
                                                                                   PartitionCols:_col0
                                                                                   Select Operator [SEL_985] (rows=652 width=4)
@@ -692,73 +692,78 @@ Stage-0
                                                                                     Filter Operator [FIL_979] (rows=652 width=8)
                                                                                       predicate:(d_year = 2001)
                                                                                        Please refer to the previous TableScan [TS_38]
-                                                                              <-Reducer 21 [SIMPLE_EDGE]
+                                                                              <-Reducer 51 [SIMPLE_EDGE]
                                                                                 SHUFFLE [RS_147]
                                                                                   PartitionCols:_col0
-                                                                                  Merge Join Operator [MERGEJOIN_958] (rows=40567099 width=312)
-                                                                                    Conds:RS_1055._col1=RS_1003._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"]
-                                                                                  <-Map 19 [SIMPLE_EDGE] vectorized
-                                                                                    PARTITION_ONLY_SHUFFLE [RS_1003]
-                                                                                      PartitionCols:_col0
-                                                                                       Please refer to the previous Select Operator [SEL_1000]
+                                                                                  Merge Join Operator [MERGEJOIN_958] (rows=40567099 width=205)
+                                                                                    Conds:RS_1051._col1=RS_1054._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                                                                                   <-Map 52 [SIMPLE_EDGE] vectorized
-                                                                                    SHUFFLE [RS_1055]
+                                                                                    PARTITION_ONLY_SHUFFLE [RS_1054]
+                                                                                      PartitionCols:_col0
+                                                                                      Select Operator [SEL_1053] (rows=4666 width=4)
+                                                                                        Output:["_col0"]
+                                                                                        Filter Operator [FIL_1052] (rows=4666 width=204)
+                                                                                          predicate:((i_color) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate') and i_current_price BETWEEN 36 AND 45)
+                                                                                          TableScan [TS_99] (rows=462000 width=204)
+                                                                                            default@item,item,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_current_price","i_color"]
+                                                                                  <-Map 50 [SIMPLE_EDGE] vectorized
+                                                                                    SHUFFLE [RS_1051]
                                                                                       PartitionCols:_col1
-                                                                                      Select Operator [SEL_1054] (rows=417313408 width=351)
+                                                                                      Select Operator [SEL_1050] (rows=417313408 width=351)
                                                                                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"]
-                                                                                        Filter Operator [FIL_1053] (rows=417313408 width=355)
+                                                                                        Filter Operator [FIL_1049] (rows=417313408 width=355)
                                                                                           predicate:((ss_sold_date_sk BETWEEN DynamicValue(RS_148_d1_d_date_sk_min) AND DynamicValue(RS_148_d1_d_date_sk_max) and in_bloom_filter(ss_sold_date_sk, DynamicValue(RS_148_d1_d_date_sk_bloom_filter))) and ss_addr_sk is not null and ss_cdemo_sk is not null and ss_customer_sk is not null and ss_hdemo_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_96] (rows=575995635 width=355)
                                                                                             default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_item_sk","ss_customer_sk","ss_cdemo_sk","ss_hdemo_sk","ss_addr_sk","ss_store_sk","ss_promo_sk","ss_ticket_number","ss_wholesale_cost","ss_list_price","ss_coupon_amt"]
-                                                                                          <-Reducer 49 [BROADCAST_EDGE] vectorized
-                                                                                            BROADCAST [RS_1052]
-                                                                                              Group By Operator [GBY_1051] (rows=1 width=12)
+                                                                                          <-Reducer 47 [BROADCAST_EDGE] vectorized
+                                                                                            BROADCAST [RS_1048]
+                                                                                              Group By Operator [GBY_1047] (rows=1 width=12)
                                                                                                 Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                                              <-Map 47 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                                              <-Map 44 [CUSTOM_SIMPLE_EDGE] vectorized
                                                                                                 PARTITION_ONLY_SHUFFLE [RS_993]
                                                                                                   Group By Operator [GBY_991] (rows=1 width=12)
                                                                                                     Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
                                                                                                     Select Operator [SEL_989] (rows=652 width=4)
                                                                                                       Output:["_col0"]
                                                                                                        Please refer to the previous Select Operator [SEL_985]
-                                                                      <-Reducer 43 [ONE_TO_ONE_EDGE] vectorized
-                                                                        FORWARD [RS_1064]
+                                                                      <-Reducer 40 [ONE_TO_ONE_EDGE] vectorized
+                                                                        FORWARD [RS_1066]
                                                                           PartitionCols:_col0
-                                                                          Select Operator [SEL_1063] (rows=13257 width=4)
+                                                                          Select Operator [SEL_1065] (rows=13257 width=4)
                                                                             Output:["_col0"]
-                                                                            Filter Operator [FIL_1062] (rows=13257 width=228)
+                                                                            Filter Operator [FIL_1064] (rows=13257 width=228)
                                                                               predicate:(_col1 > (2 * _col2))
-                                                                              Group By Operator [GBY_1061] (rows=39773 width=228)
+                                                                              Group By Operator [GBY_1063] (rows=39773 width=228)
                                                                                 Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0
-                                                                              <-Reducer 42 [SIMPLE_EDGE]
+                                                                              <-Reducer 39 [SIMPLE_EDGE]
                                                                                 SHUFFLE [RS_117]
                                                                                   PartitionCols:_col0
                                                                                   Group By Operator [GBY_116] (rows=6482999 width=228)
                                                                                     Output:["_col0","_col1","_col2"],aggregations:["sum(_col2)","sum(_col5)"],keys:_col0
                                                                                     Merge Join Operator [MERGEJOIN_961] (rows=183085709 width=227)
-                                                                                      Conds:RS_1060._col0, _col1=RS_1020._col0, _col1(Inner),Output:["_col0","_col2","_col5"]
-                                                                                    <-Map 41 [SIMPLE_EDGE] vectorized
-                                                                                      SHUFFLE [RS_1020]
+                                                                                      Conds:RS_1062._col0, _col1=RS_1016._col0, _col1(Inner),Output:["_col0","_col2","_col5"]
+                                                                                    <-Map 38 [SIMPLE_EDGE] vectorized
+                                                                                      SHUFFLE [RS_1016]
                                                                                         PartitionCols:_col0, _col1
-                                                                                         Please refer to the previous Select Operator [SEL_1018]
-                                                                                    <-Map 53 [SIMPLE_EDGE] vectorized
-                                                                                      SHUFFLE [RS_1060]
+                                                                                         Please refer to the previous Select Operator [SEL_1014]
+                                                                                    <-Map 54 [SIMPLE_EDGE] vectorized
+                                                                                      SHUFFLE [RS_1062]
                                                                                         PartitionCols:_col0, _col1
-                                                                                        Select Operator [SEL_1059] (rows=287989836 width=119)
+                                                                                        Select Operator [SEL_1061] (rows=287989836 width=119)
                                                                                           Output:["_col0","_col1","_col2"]
-                                                                                          Filter Operator [FIL_1058] (rows=287989836 width=119)
+                                                                                          Filter Operator [FIL_1060] (rows=287989836 width=119)
                                                                                             predicate:(cs_item_sk BETWEEN DynamicValue(RS_145_item_i_item_sk_min) AND DynamicValue(RS_145_item_i_item_sk_max) and in_bloom_filter(cs_item_sk, DynamicValue(RS_145_item_i_item_sk_bloom_filter)))
                                                                                             TableScan [TS_108] (rows=287989836 width=119)
                                                                                               default@catalog_sales,catalog_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_item_sk","cs_order_number","cs_ext_list_price"]
-                                                                                            <-Reducer 36 [BROADCAST_EDGE] vectorized
-                                                                                              BROADCAST [RS_1057]
-                                                                                                Group By Operator [GBY_1056] (rows=1 width=12)
+                                                                                            <-Reducer 53 [BROADCAST_EDGE] vectorized
+                                                                                              BROADCAST [RS_1059]
+                                                                                                Group By Operator [GBY_1058] (rows=1 width=12)
                                                                                                   Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","bloom_filter(VALUE._col2, expectedEntries=1000000)"]
-                                                                                                <-Map 19 [CUSTOM_SIMPLE_EDGE] vectorized
-                                                                                                  PARTITION_ONLY_SHUFFLE [RS_1008]
-                                                                                                    Group By Operator [GBY_1006] (rows=1 width=12)
+                                                                                                <-Map 52 [CUSTOM_SIMPLE_EDGE] vectorized
+                                                                                                  PARTITION_ONLY_SHUFFLE [RS_1057]
+                                                                                                    Group By Operator [GBY_1056] (rows=1 width=12)
                                                                                                       Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0, expectedEntries=1000000)"]
-                                                                                                      Select Operator [SEL_1004] (rows=4666 width=4)
+                                                                                                      Select Operator [SEL_1055] (rows=4666 width=4)
                                                                                                         Output:["_col0"]
-                                                                                                         Please refer to the previous Select Operator [SEL_1000]
+                                                                                                         Please refer to the previous Select Operator [SEL_1053]