You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2021/09/27 09:51:45 UTC

[impala] 02/07: IMPALA-10681: Improve inner join cardinality estimates

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

stigahuang pushed a commit to branch branch-4.0.1
in repository https://gitbox.apache.org/repos/asf/impala.git

commit b3a31496aa7d73b402c4a87cc404b005213c6178
Author: Aman Sinha <am...@cloudera.com>
AuthorDate: Tue May 18 17:40:43 2021 -0700

    IMPALA-10681: Improve inner join cardinality estimates
    
    During cardinality estimation for inner joins, if the join
    conjunct involves a scan slot on left side and a function
    (e.g MAX) on the right, currently we determine that the NDV
    stats of either side is not useful and return the left side's
    cardinality even though it may be a significant over-estimate.
    
    In this patch, we handle join conjuncts of such types by
    keeping them in an 'other' eligible conjuncts list as long as
    the NDV for expressions on both sides of the join and the
    input row count is available. For example, in the following
    cases the NDV is available but was not being used for inner
    joins since the previous logic was only looking for scan
    slots: (a) int_col = MAX(int_col) and the right input does
    not have a group-by, so right NDV = 1 can be used. (b) if it
    has a group-by and the group-by columns already have
    associated NDV, the combined NDV is also available.
    Other such examples exist. An auxiliary struct is introduced
    to keep track of the ndv and row count.
    
    Once these 'other' eligible conjuncts are populated, we do the
    join cardinality estimation in a manner similar to the normal
    join conjuncts by fetching the stats from the auxiliary struct.
    
    Testing:
     - Added new planner tests for inner join cardinality
     - Modified expected plans for certains tests including
       TPC-DS queries and ran end-to-end TPC-DS queries
     - Since TPC-DS plans are complex, I did a check of the cardinality
       changes for some of the hash joins but not the changes in the
       shape of a plan (e.g whether the join order changed).
     - Preliminary tests with a TPC-DS 10 GB scale factor on a single
       node showed between 5-15% performance improvements for 4 of the
       6 queries whose plans changed.
    
    Change-Id: I8aa9d3b8f3c4848b3e9414fe19ad7ad348d12ecc
    Reviewed-on: http://gerrit.cloudera.org:8080/17387
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Aman Sinha <am...@cloudera.com>
    Reviewed-by: Aman Sinha <am...@cloudera.com>
---
 .../main/java/org/apache/impala/analysis/Expr.java |   15 +
 .../apache/impala/analysis/FunctionCallExpr.java   |    6 +
 .../java/org/apache/impala/planner/JoinNode.java   |  199 +-
 .../queries/PlannerTest/card-inner-join.test       |  136 ++
 .../queries/PlannerTest/join-order.test            |  294 +--
 .../queries/PlannerTest/joins.test                 |    8 +-
 .../PlannerTest/partition-key-scans-default.test   |    2 +-
 .../queries/PlannerTest/partition-key-scans.test   |    2 +-
 .../queries/PlannerTest/tpcds/tpcds-q04.test       | 2544 ++++++++++----------
 .../queries/PlannerTest/tpcds/tpcds-q05.test       |  386 +--
 .../queries/PlannerTest/tpcds/tpcds-q11.test       |  934 +++----
 .../queries/PlannerTest/tpcds/tpcds-q54.test       |  789 +++---
 .../queries/PlannerTest/tpcds/tpcds-q71.test       |  405 ++--
 .../queries/PlannerTest/tpcds/tpcds-q74.test       |  810 ++++---
 .../queries/PlannerTest/views.test                 |  188 +-
 15 files changed, 3544 insertions(+), 3174 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/Expr.java b/fe/src/main/java/org/apache/impala/analysis/Expr.java
index 3eab962..07c27c1 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Expr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Expr.java
@@ -1855,4 +1855,19 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
     }
     return hasChanges;
   }
+
+  /**
+   * A slot descriptor may be associated with more than 1 source expression.
+   * This method returns the first source expr in that case or null if there
+   * are no source exprs.
+   */
+  public Expr getSlotDescFirstSourceExpr() {
+    SlotRef slotRef = unwrapSlotRef(false);
+    if (slotRef == null) return null;
+    SlotDescriptor slotDesc = slotRef.getDesc();
+    if (slotDesc.getSourceExprs().size() >= 1) {
+      return slotDesc.getSourceExprs().get(0);
+    }
+    return null;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java b/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
index 038fa15..93f1c4c 100644
--- a/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/FunctionCallExpr.java
@@ -266,6 +266,12 @@ public class FunctionCallExpr extends Expr {
     return fn_ instanceof AggregateFunction && !isAnalyticFnCall_;
   }
 
+  /** Returns true if this is a call to an analytic aggregate function. */
+  public boolean isAnalyticFunction() {
+    Preconditions.checkNotNull(fn_);
+    return fn_ instanceof AggregateFunction && isAnalyticFnCall_;
+  }
+
   /** Returns true if this function is a call to the built-in grouping() function. */
   public boolean isGroupingBuiltin() {
     return functionNameEqualsBuiltin(fnName_, "grouping");
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinNode.java b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
index 28db19c..a758ab9 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
@@ -23,9 +23,11 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.impala.analysis.AnalyticExpr;
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.Expr;
+import org.apache.impala.analysis.FunctionCallExpr;
 import org.apache.impala.analysis.JoinOperator;
 import org.apache.impala.analysis.SlotDescriptor;
 import org.apache.impala.analysis.SlotRef;
@@ -245,16 +247,25 @@ public abstract class JoinNode extends PlanNode {
    * We estimate the cardinality based on equality join predicates of the form
    * "L.c = R.d", with L being a table from child(0) and R a table from child(1).
    * For each set of such join predicates between two tables, we try to determine whether
-   * the tables might have foreign/primary key (FK/PK) relationship, and either use a
-   * special FK/PK estimation or a generic estimation method. Once the estimation method
-   * has been determined we compute the final cardinality based on the single most
-   * selective join predicate. We do not attempt to estimate the joint selectivity of
-   * multiple join predicates to avoid underestimation.
+   * the tables might have foreign/primary key (FK/PK) relationship, and use one of 3
+   * estimators: (a) special FK/PK estimation (b) generic estimation method (c) an
+   * estimation for 'other' conjuncts that may involve functions or expressions - this
+   * estimation is very similar to the generic estimator.
+   *
+   * Once the estimation method has been determined we compute the final cardinality
+   * based on the single most selective join predicate. We do not attempt to estimate
+   * the joint selectivity of multiple join predicates to avoid underestimation.
    * The FK/PK detection logic is based on the assumption that most joins are FK/PK. We
    * only use the generic estimation method if we have high confidence that there is no
    * FK/PK relationship. In the absence of relevant stats, we assume FK/PK with a join
    * selectivity of 1.
    *
+   * In some cases where a function is involved in the join predicate - e.g c = max(d),
+   * the RHS may have relevant stats. For instance if it is s scalar subquery, the RHS
+   * NDV = 1. Whenever such stats are available, we classify them into an 'other'
+   * conjuncts list and leverage the available stats. We use the same estimation
+   * formula as the generic estimator.
+   *
    * FK/PK estimation:
    * cardinality = |child(0)| * (|child(1)| / |R|) * (NDV(R.d) / NDV(L.c))
    * - the cardinality of a FK/PK must be <= |child(0)|
@@ -288,15 +299,26 @@ public abstract class JoinNode extends PlanNode {
 
     // Collect join conjuncts that are eligible to participate in cardinality estimation.
     List<EqJoinConjunctScanSlots> eqJoinConjunctSlots = new ArrayList<>();
+    // A list of stats for 'other' join conjuncts where the slot refs on one or both
+    // sides don't directly trace back to a base table column.
+    // e.g  a = MAX(b). Here, the RHS is a function.
+    List<NdvAndRowCountStats> otherEqJoinStats = new ArrayList<>();
     for (Expr eqJoinConjunct: eqJoinConjuncts_) {
-      EqJoinConjunctScanSlots slots = EqJoinConjunctScanSlots.create(eqJoinConjunct);
-      if (slots != null) eqJoinConjunctSlots.add(slots);
+      EqJoinConjunctScanSlots slots = EqJoinConjunctScanSlots.create(eqJoinConjunct,
+          otherEqJoinStats, lhsCard, rhsCard);
+      if (slots != null) {
+        eqJoinConjunctSlots.add(slots);
+      }
     }
 
     if (eqJoinConjunctSlots.isEmpty()) {
-      // There are no eligible equi-join conjuncts. Optimistically assume FK/PK with a
-      // join selectivity of 1.
-      return lhsCard;
+      if (!otherEqJoinStats.isEmpty() && joinOp_.isInnerJoin()) {
+        return getGenericJoinCardinality2(otherEqJoinStats, lhsCard, rhsCard);
+      } else {
+        // There are no eligible equi-join conjuncts. Optimistically assume FK/PK with a
+        // join selectivity of 1.
+        return lhsCard;
+      }
     }
 
     fkPkEqJoinConjuncts_ = getFkPkEqJoinConjuncts(eqJoinConjunctSlots);
@@ -382,22 +404,58 @@ public abstract class JoinNode extends PlanNode {
       long lhsCard, long rhsCard) {
     Preconditions.checkState(joinOp_.isInnerJoin() || joinOp_.isOuterJoin());
     Preconditions.checkState(!eqJoinConjunctSlots.isEmpty());
-    Preconditions.checkState(lhsCard >= 0 && rhsCard >= 0);
 
     long result = -1;
     for (EqJoinConjunctScanSlots slots: eqJoinConjunctSlots) {
-      // Adjust the NDVs on both sides to account for predicates. Intuitively, the NDVs
-      // should only decrease. We ignore adjustments that would lead to an increase.
-      double lhsAdjNdv = slots.lhsNdv();
-      if (slots.lhsNumRows() > lhsCard) lhsAdjNdv *= lhsCard / slots.lhsNumRows();
-      double rhsAdjNdv = slots.rhsNdv();
-      if (slots.rhsNumRows() > rhsCard) rhsAdjNdv *= rhsCard / slots.rhsNumRows();
-      // A lower limit of 1 on the max Adjusted Ndv ensures we don't estimate
-      // cardinality more than the max possible. This also handles the case of
-      // null columns on both sides having an Ndv of zero (which would change
-      // after IMPALA-7310 is fixed).
-      long joinCard = Math.round((lhsCard / Math.max(1, Math.max(lhsAdjNdv, rhsAdjNdv))) *
-          rhsCard);
+      long joinCard = getGenericJoinCardinalityInternal(slots.lhsNdv(), slots.rhsNdv(),
+          slots.lhsNumRows(), slots.rhsNumRows(), lhsCard, rhsCard);
+      if (result == -1) {
+        result = joinCard;
+      } else {
+        result = Math.min(result, joinCard);
+      }
+    }
+    Preconditions.checkState(result >= 0);
+    return result;
+  }
+
+  /**
+   * An internal utility method to compute generic join cardinality as described
+   * in the comments for {@link JoinNode#getJoinCardinality}. The input
+   * cardinalities must be >= 0.
+   */
+  private long getGenericJoinCardinalityInternal(double lhsNdv, double rhsNdv,
+      double lhsNumRows, double rhsNumRows, long lhsCard, long rhsCard) {
+    Preconditions.checkState(lhsCard >= 0 && rhsCard >= 0);
+    // Adjust the NDVs on both sides to account for predicates. Intuitively, the NDVs
+    // should only decrease. We ignore adjustments that would lead to an increase.
+    double lhsAdjNdv = lhsNdv;
+    if (lhsNumRows > lhsCard) lhsAdjNdv *= lhsCard / lhsNumRows;
+    double rhsAdjNdv = rhsNdv;
+    if (rhsNumRows > rhsCard) rhsAdjNdv *= rhsCard / rhsNumRows;
+    // A lower limit of 1 on the max Adjusted Ndv ensures we don't estimate
+    // cardinality more than the max possible.
+    long joinCard = Math.round((lhsCard / Math.max(1, Math.max(lhsAdjNdv, rhsAdjNdv))) *
+        rhsCard);
+    return joinCard;
+  }
+
+  /**
+   * This function mirrors the logic for {@link JoinNode#getGenericJoinCardinality} except
+   * that instead of the EqJoinConjunctScanSlots, it uses the {@link NdvAndRowCountStats}
+   * to directly access stats that were pre-populated. Currently, this function is
+   * restricted to inner joins. In order to extend it to outer joins some more analysis is
+   * needed to ensure it works correctly for different types of outer joins.
+   */
+  private long getGenericJoinCardinality2(List<NdvAndRowCountStats> statsList,
+      long lhsCard, long rhsCard) {
+    Preconditions.checkState(joinOp_.isInnerJoin());
+    Preconditions.checkState(!statsList.isEmpty());
+
+    long result = -1;
+    for (NdvAndRowCountStats stats: statsList) {
+      long joinCard = getGenericJoinCardinalityInternal(stats.lhsNdv(), stats.rhsNdv(),
+          stats.lhsNumRows(), stats.rhsNumRows(), lhsCard, rhsCard);
       if (result == -1) {
         result = joinCard;
       } else {
@@ -440,20 +498,75 @@ public abstract class JoinNode extends PlanNode {
     /**
      * Returns a new EqJoinConjunctScanSlots for the given equi-join conjunct or null if
      * the given conjunct is not of the form <SlotRef> = <SlotRef> or if the underlying
-     * table/column of at least one side is missing stats.
+     * table/column of at least one side is missing stats. Even when the conjunct does not
+     * refer to scan slots on both sides, the NDV stats for the expr might be available
+     * and this function populates the supplied otherEqJonConjuncts list with whatever
+     * stats it can retrieve.
      */
-    public static EqJoinConjunctScanSlots create(Expr eqJoinConjunct) {
+    public static EqJoinConjunctScanSlots create(Expr eqJoinConjunct,
+      List<NdvAndRowCountStats> otherEqJoinConjuncts, long lhsCard, long rhsCard) {
       if (!Expr.IS_EQ_BINARY_PREDICATE.apply(eqJoinConjunct)) return null;
       SlotDescriptor lhsScanSlot = eqJoinConjunct.getChild(0).findSrcScanSlot();
-      if (lhsScanSlot == null || !hasNumRowsAndNdvStats(lhsScanSlot)) return null;
+      boolean hasLhs = true;
+      boolean hasRhs = true;
+      if (lhsScanSlot == null || !hasNumRowsAndNdvStats(lhsScanSlot)) hasLhs = false;
       SlotDescriptor rhsScanSlot = eqJoinConjunct.getChild(1).findSrcScanSlot();
-      if (rhsScanSlot == null || !hasNumRowsAndNdvStats(rhsScanSlot)) return null;
-      return new EqJoinConjunctScanSlots(eqJoinConjunct, lhsScanSlot, rhsScanSlot);
+      if (rhsScanSlot == null || !hasNumRowsAndNdvStats(rhsScanSlot)) hasRhs = false;
+      if (hasLhs && hasRhs) {
+        return new EqJoinConjunctScanSlots(eqJoinConjunct, lhsScanSlot, rhsScanSlot);
+      }
+
+      Expr lhsExpr = eqJoinConjunct.getChild(0);
+      Expr rhsExpr = eqJoinConjunct.getChild(1);
+      if (!hasLhs) {
+        lhsExpr = lhsExpr.getSlotDescFirstSourceExpr();
+        if (lhsExpr == null) return null;
+      }
+      if (!hasRhs) {
+        rhsExpr = rhsExpr.getSlotDescFirstSourceExpr();
+        if (rhsExpr == null) return null;
+      }
+      // For analytic exprs, the NDV is incorrect (see IMPALA-10697). Until that is
+      // fixed, we should skip assigning the stats for such conjuncts.
+      if (lhsExpr instanceof AnalyticExpr || rhsExpr instanceof AnalyticExpr) {
+        return null;
+      }
+      long lhsNdv = lhsScanSlot != null ?
+          lhsScanSlot.getStats().getNumDistinctValues() :
+          JoinNode.getNdv(eqJoinConjunct.getChild(0));
+      long rhsNdv = rhsScanSlot != null ?
+          rhsScanSlot.getStats().getNumDistinctValues() :
+          JoinNode.getNdv(eqJoinConjunct.getChild(1));
+      if (lhsNdv == -1 || rhsNdv == -1) return null;
+
+      // In the following num rows assignment, if the underlying scan slot is not
+      // available we cannot get the actual base table row count. In that case we
+      // approximate the row count as just the lhs or rhs cardinality. Since the
+      // ratio of cardinality/num_rows is used to adjust (scale down) the NDV
+      // later (when computing join cardinality), it means we would fall back to
+      // not doing the adjustment which is ok since the NDV eventually gets capped
+      // at the cardinality.
+      long lhsNumRows = lhsScanSlot != null && hasNumRowsStats(lhsScanSlot) ?
+          lhsScanSlot.getParent().getTable().getNumRows() : lhsCard;
+      long rhsNumRows = rhsScanSlot != null && hasNumRowsStats(rhsScanSlot) ?
+          rhsScanSlot.getParent().getTable().getNumRows() : rhsCard;
+      otherEqJoinConjuncts.add(new NdvAndRowCountStats(lhsNdv, rhsNdv, lhsNumRows,
+          rhsNumRows));
+
+      return null;
     }
 
     private static boolean hasNumRowsAndNdvStats(SlotDescriptor slotDesc) {
+      return (hasNdvStats(slotDesc) && hasNumRowsStats(slotDesc));
+    }
+
+    private static boolean hasNdvStats(SlotDescriptor slotDesc) {
       if (slotDesc.getColumn() == null) return false;
       if (!slotDesc.getStats().hasNumDistinctValues()) return false;
+      return true;
+    }
+
+    private static boolean hasNumRowsStats(SlotDescriptor slotDesc) {
       FeTable tbl = slotDesc.getParent().getTable();
       if (tbl == null || tbl.getNumRows() == -1) return false;
       return true;
@@ -484,6 +597,34 @@ public abstract class JoinNode extends PlanNode {
   }
 
   /**
+   * A struct to pass around ndv and num rows stats during
+   * cardinality estimations. The ndv values are upper bounded
+   * by the num rows
+   */
+  public static final class NdvAndRowCountStats {
+    private final long lhsNdv_;
+    private final long rhsNdv_;
+    private final long lhsNumRows_;
+    private final long rhsNumRows_;
+
+    public NdvAndRowCountStats(long lhsNdv, long rhsNdv,
+                               long lhsNumRows, long rhsNumRows) {
+      // upper bound the ndv values since the caller may not have done
+      // the adjustment
+      lhsNdv_ = Math.min(lhsNdv, lhsNumRows);
+      rhsNdv_ = Math.min(rhsNdv, rhsNumRows);
+      lhsNumRows_ = lhsNumRows;
+      rhsNumRows_ = rhsNumRows;
+    }
+
+    // Convenience functions. They return double to avoid excessive casts in callers.
+    public double lhsNdv() { return lhsNdv_; }
+    public double rhsNdv() { return rhsNdv_; }
+    public double lhsNumRows() { return lhsNumRows_; }
+    public double rhsNumRows() { return rhsNumRows_; }
+  }
+
+  /**
    * Returns the estimated cardinality of a semi join node.
    * For a left semi join between child(0) and child(1), we look for equality join
    * conditions "L.c = R.d" (with L being from child(0) and R from child(1)) and use as
@@ -565,7 +706,7 @@ public abstract class JoinNode extends PlanNode {
    * Unwraps the SlotRef in expr and returns the NDVs of it.
    * Returns -1 if the NDVs are unknown or if expr is not a SlotRef.
    */
-  protected long getNdv(Expr expr) {
+  public static long getNdv(Expr expr) {
     SlotRef slotRef = expr.unwrapSlotRef(false);
     if (slotRef == null) return -1;
     SlotDescriptor slotDesc = slotRef.getDesc();
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test b/testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test
index 2da3d1a..f609ec8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test
@@ -983,3 +983,139 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> b.id + b.int_col
    row-size=8B cardinality=11.00K
 ====
+# IMPALA-10681: Better estimate join cardinality
+# if one side is a function expr
+select ss_customer_sk from tpcds.store_sales
+ inner join
+(select max(s_store_sk) as max_store_sk
+ from tpcds.store) v
+on ss_store_sk = max_store_sk;
+---- PLAN
+PLAN-ROOT SINK
+|
+03:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_store_sk = max(s_store_sk)
+|  runtime filters: RF000 <- max(s_store_sk)
+|  row-size=12B cardinality=480.07K
+|
+|--02:AGGREGATE [FINALIZE]
+|  |  output: max(s_store_sk)
+|  |  row-size=4B cardinality=1
+|  |
+|  01:SCAN HDFS [tpcds.store]
+|     HDFS partitions=1/1 files=1 size=3.08KB
+|     row-size=4B cardinality=12
+|
+00:SCAN HDFS [tpcds.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
+   runtime filters: RF000 -> ss_store_sk
+   row-size=8B cardinality=2.88M
+====
+# Modified version of above with a union all
+select ss_customer_sk from tpcds.store_sales
+ inner join
+ (select max(s_store_sk) as max_store_sk from tpcds.store
+   union all
+  select min(s_store_sk) as max_store_sk from tpcds.store) v
+ on ss_store_sk = max_store_sk;
+---- PLAN
+PLAN-ROOT SINK
+|
+06:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_store_sk = max_store_sk
+|  runtime filters: RF000 <- max_store_sk
+|  row-size=12B cardinality=960.13K
+|
+|--01:UNION
+|  |  pass-through-operands: all
+|  |  row-size=4B cardinality=2
+|  |
+|  |--05:AGGREGATE [FINALIZE]
+|  |  |  output: min(s_store_sk)
+|  |  |  row-size=4B cardinality=1
+|  |  |
+|  |  04:SCAN HDFS [tpcds.store]
+|  |     HDFS partitions=1/1 files=1 size=3.08KB
+|  |     row-size=4B cardinality=12
+|  |
+|  03:AGGREGATE [FINALIZE]
+|  |  output: max(s_store_sk)
+|  |  row-size=4B cardinality=1
+|  |
+|  02:SCAN HDFS [tpcds.store]
+|     HDFS partitions=1/1 files=1 size=3.08KB
+|     row-size=4B cardinality=12
+|
+00:SCAN HDFS [tpcds.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
+   runtime filters: RF000 -> ss_store_sk
+   row-size=8B cardinality=2.88M
+====
+# Semi join version of the above (for comparison purposes)
+select ss_customer_sk from tpcds.store_sales
+ left semi join
+(select max(s_store_sk) as max_store_sk from tpcds.store
+  union all
+ select min(s_store_sk) as max_store_sk from tpcds.store) v
+ on ss_store_sk = max_store_sk;
+---- PLAN
+PLAN-ROOT SINK
+|
+06:HASH JOIN [LEFT SEMI JOIN]
+|  hash predicates: ss_store_sk = max_store_sk
+|  runtime filters: RF000 <- max_store_sk
+|  row-size=8B cardinality=960.13K
+|
+|--01:UNION
+|  |  pass-through-operands: all
+|  |  row-size=4B cardinality=2
+|  |
+|  |--05:AGGREGATE [FINALIZE]
+|  |  |  output: min(s_store_sk)
+|  |  |  row-size=4B cardinality=1
+|  |  |
+|  |  04:SCAN HDFS [tpcds.store]
+|  |     HDFS partitions=1/1 files=1 size=3.08KB
+|  |     row-size=4B cardinality=12
+|  |
+|  03:AGGREGATE [FINALIZE]
+|  |  output: max(s_store_sk)
+|  |  row-size=4B cardinality=1
+|  |
+|  02:SCAN HDFS [tpcds.store]
+|     HDFS partitions=1/1 files=1 size=3.08KB
+|     row-size=4B cardinality=12
+|
+00:SCAN HDFS [tpcds.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
+   runtime filters: RF000 -> ss_store_sk
+   row-size=8B cardinality=2.88M
+====
+# Modified version above: use a non aggregate function such
+# as concat along with group-by
+select c_first_name from tpcds.customer
+ inner join
+(select concat(s_manager, s_market_manager) as mgr
+  from tpcds.store group by mgr) v
+on c_last_name = v.mgr;
+---- PLAN
+PLAN-ROOT SINK
+|
+03:HASH JOIN [INNER JOIN]
+|  hash predicates: c_last_name = concat(s_manager, s_market_manager)
+|  runtime filters: RF000 <- concat(s_manager, s_market_manager)
+|  row-size=48B cardinality=141
+|
+|--02:AGGREGATE [FINALIZE]
+|  |  group by: concat(s_manager, s_market_manager)
+|  |  row-size=12B cardinality=7
+|  |
+|  01:SCAN HDFS [tpcds.store]
+|     HDFS partitions=1/1 files=1 size=3.08KB
+|     row-size=50B cardinality=12
+|
+00:SCAN HDFS [tpcds.customer]
+   HDFS partitions=1/1 files=1 size=12.60MB
+   runtime filters: RF000 -> c_last_name
+   row-size=36B cardinality=100.00K
+====
\ No newline at end of file
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test b/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
index 6891f90..7532e24 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
@@ -1242,65 +1242,66 @@ on (t3.id = t4.id)
 PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN]
-|  hash predicates: t3.id = t4.id
-|  runtime filters: RF000 <- t4.id
+|  hash predicates: t4.id = t3.id
+|  runtime filters: RF000 <- t3.id
 |  row-size=28B cardinality=1
 |
-|--09:SCAN HDFS [functional.alltypestiny t4]
-|     HDFS partitions=4/4 files=4 size=460B
-|     row-size=4B cardinality=8
-|
-11:HASH JOIN [RIGHT OUTER JOIN]
-|  hash predicates: t3.id = a.id
-|  runtime filters: RF002 <- a.id
-|  row-size=24B cardinality=8
-|
-|--10:HASH JOIN [INNER JOIN]
-|  |  hash predicates: a.id = count(a.id)
-|  |  runtime filters: RF004 <- count(a.id)
-|  |  row-size=20B cardinality=8
+|--11:HASH JOIN [RIGHT OUTER JOIN]
+|  |  hash predicates: t3.id = a.id
+|  |  runtime filters: RF002 <- a.id
+|  |  row-size=24B cardinality=1
 |  |
-|  |--04:AGGREGATE [FINALIZE]
-|  |  |  output: count(a.id)
-|  |  |  row-size=8B cardinality=1
-|  |  |
-|  |  03:AGGREGATE
-|  |  |  group by: a.id
-|  |  |  row-size=4B cardinality=8
-|  |  |
-|  |  02:HASH JOIN [INNER JOIN]
-|  |  |  hash predicates: a.id = b.id
-|  |  |  runtime filters: RF006 <- b.id
-|  |  |  row-size=8B cardinality=8
+|  |--10:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: a.id = count(a.id)
+|  |  |  runtime filters: RF004 <- count(a.id)
+|  |  |  row-size=20B cardinality=1
 |  |  |
-|  |  |--01:SCAN HDFS [functional.alltypestiny b]
+|  |  |--04:AGGREGATE [FINALIZE]
+|  |  |  |  output: count(a.id)
+|  |  |  |  row-size=8B cardinality=1
+|  |  |  |
+|  |  |  03:AGGREGATE
+|  |  |  |  group by: a.id
+|  |  |  |  row-size=4B cardinality=8
+|  |  |  |
+|  |  |  02:HASH JOIN [INNER JOIN]
+|  |  |  |  hash predicates: a.id = b.id
+|  |  |  |  runtime filters: RF006 <- b.id
+|  |  |  |  row-size=8B cardinality=8
+|  |  |  |
+|  |  |  |--01:SCAN HDFS [functional.alltypestiny b]
+|  |  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |  |     row-size=4B cardinality=8
+|  |  |  |
+|  |  |  00:SCAN HDFS [functional.alltypestiny a]
 |  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |     runtime filters: RF006 -> a.id
 |  |  |     row-size=4B cardinality=8
 |  |  |
-|  |  00:SCAN HDFS [functional.alltypestiny a]
+|  |  07:HASH JOIN [LEFT OUTER JOIN]
+|  |  |  hash predicates: b.id = a.id
+|  |  |  other predicates: a.`year` < 10
+|  |  |  row-size=12B cardinality=8
+|  |  |
+|  |  |--05:SCAN HDFS [functional.alltypes a]
+|  |  |     partition predicates: a.`year` < 10
+|  |  |     partitions=0/24 files=0 size=0B
+|  |  |     runtime filters: RF004 -> a.id
+|  |  |     row-size=8B cardinality=0
+|  |  |
+|  |  06:SCAN HDFS [functional.alltypestiny b]
 |  |     HDFS partitions=4/4 files=4 size=460B
-|  |     runtime filters: RF006 -> a.id
 |  |     row-size=4B cardinality=8
 |  |
-|  07:HASH JOIN [LEFT OUTER JOIN]
-|  |  hash predicates: b.id = a.id
-|  |  other predicates: a.`year` < 10
-|  |  row-size=12B cardinality=8
-|  |
-|  |--05:SCAN HDFS [functional.alltypes a]
-|  |     partition predicates: a.`year` < 10
-|  |     partitions=0/24 files=0 size=0B
-|  |     runtime filters: RF004 -> a.id
-|  |     row-size=8B cardinality=0
-|  |
-|  06:SCAN HDFS [functional.alltypestiny b]
-|     HDFS partitions=4/4 files=4 size=460B
-|     row-size=4B cardinality=8
+|  08:SCAN HDFS [functional.alltypes t3]
+|     HDFS partitions=24/24 files=24 size=478.45KB
+|     runtime filters: RF002 -> t3.id
+|     row-size=4B cardinality=7.30K
 |
-08:SCAN HDFS [functional.alltypes t3]
-   HDFS partitions=24/24 files=24 size=478.45KB
-   runtime filters: RF000 -> t3.id, RF002 -> t3.id
-   row-size=4B cardinality=7.30K
+09:SCAN HDFS [functional.alltypestiny t4]
+   HDFS partitions=4/4 files=4 size=460B
+   runtime filters: RF000 -> t4.id
+   row-size=4B cardinality=8
 ====
 # Same as above but with full outer joins.
 select 1 from
@@ -1331,12 +1332,12 @@ PLAN-ROOT SINK
 |
 11:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: t3.id = a.id
-|  row-size=24B cardinality=7.31K
+|  row-size=24B cardinality=7.30K
 |
 |--10:HASH JOIN [INNER JOIN]
 |  |  hash predicates: a.id = count(a.id)
 |  |  runtime filters: RF002 <- count(a.id)
-|  |  row-size=20B cardinality=8
+|  |  row-size=20B cardinality=1
 |  |
 |  |--04:AGGREGATE [FINALIZE]
 |  |  |  output: count(a.id)
@@ -1399,66 +1400,67 @@ where t2.month = 1
 PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN]
-|  hash predicates: b.id = t4.id
-|  runtime filters: RF000 <- t4.id
+|  hash predicates: t4.id = b.id
+|  runtime filters: RF000 <- b.id
 |  row-size=16B cardinality=1
 |
-|--09:SCAN HDFS [functional.alltypestiny t4]
-|     HDFS partitions=4/4 files=4 size=460B
-|     row-size=4B cardinality=8
-|
-11:HASH JOIN [RIGHT SEMI JOIN]
-|  hash predicates: t3.id = b.id
-|  runtime filters: RF002 <- b.id
-|  row-size=12B cardinality=8
-|
-|--10:HASH JOIN [INNER JOIN]
-|  |  hash predicates: b.id = count(a.id)
-|  |  runtime filters: RF004 <- count(a.id)
-|  |  row-size=12B cardinality=8
+|--11:HASH JOIN [RIGHT SEMI JOIN]
+|  |  hash predicates: t3.id = b.id
+|  |  runtime filters: RF002 <- b.id
+|  |  row-size=12B cardinality=1
 |  |
-|  |--04:AGGREGATE [FINALIZE]
-|  |  |  output: count(a.id)
-|  |  |  row-size=8B cardinality=1
+|  |--10:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: b.id = count(a.id)
+|  |  |  runtime filters: RF004 <- count(a.id)
+|  |  |  row-size=12B cardinality=1
 |  |  |
-|  |  03:AGGREGATE
-|  |  |  group by: a.id
-|  |  |  row-size=4B cardinality=8
+|  |  |--04:AGGREGATE [FINALIZE]
+|  |  |  |  output: count(a.id)
+|  |  |  |  row-size=8B cardinality=1
+|  |  |  |
+|  |  |  03:AGGREGATE
+|  |  |  |  group by: a.id
+|  |  |  |  row-size=4B cardinality=8
+|  |  |  |
+|  |  |  02:HASH JOIN [INNER JOIN]
+|  |  |  |  hash predicates: a.id = b.id
+|  |  |  |  runtime filters: RF008 <- b.id
+|  |  |  |  row-size=8B cardinality=8
+|  |  |  |
+|  |  |  |--01:SCAN HDFS [functional.alltypestiny b]
+|  |  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |  |     row-size=4B cardinality=8
+|  |  |  |
+|  |  |  00:SCAN HDFS [functional.alltypestiny a]
+|  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |     runtime filters: RF008 -> a.id
+|  |  |     row-size=4B cardinality=8
 |  |  |
-|  |  02:HASH JOIN [INNER JOIN]
-|  |  |  hash predicates: a.id = b.id
-|  |  |  runtime filters: RF008 <- b.id
-|  |  |  row-size=8B cardinality=8
+|  |  07:HASH JOIN [LEFT SEMI JOIN]
+|  |  |  hash predicates: b.id = a.id
+|  |  |  runtime filters: RF006 <- a.id
+|  |  |  row-size=4B cardinality=8
 |  |  |
-|  |  |--01:SCAN HDFS [functional.alltypestiny b]
+|  |  |--05:SCAN HDFS [functional.alltypestiny a]
 |  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |     runtime filters: RF004 -> a.id
 |  |  |     row-size=4B cardinality=8
 |  |  |
-|  |  00:SCAN HDFS [functional.alltypestiny a]
-|  |     HDFS partitions=4/4 files=4 size=460B
-|  |     runtime filters: RF008 -> a.id
-|  |     row-size=4B cardinality=8
-|  |
-|  07:HASH JOIN [LEFT SEMI JOIN]
-|  |  hash predicates: b.id = a.id
-|  |  runtime filters: RF006 <- a.id
-|  |  row-size=4B cardinality=8
-|  |
-|  |--05:SCAN HDFS [functional.alltypestiny a]
-|  |     HDFS partitions=4/4 files=4 size=460B
-|  |     runtime filters: RF000 -> a.id, RF004 -> a.id
-|  |     row-size=4B cardinality=8
+|  |  06:SCAN HDFS [functional.alltypes b]
+|  |     partition predicates: b.month = 1
+|  |     HDFS partitions=2/24 files=2 size=40.32KB
+|  |     runtime filters: RF004 -> b.id, RF006 -> b.id
+|  |     row-size=4B cardinality=620
 |  |
-|  06:SCAN HDFS [functional.alltypes b]
-|     partition predicates: b.month = 1
-|     HDFS partitions=2/24 files=2 size=40.32KB
-|     runtime filters: RF000 -> b.id, RF004 -> b.id, RF006 -> b.id
-|     row-size=4B cardinality=620
+|  08:SCAN HDFS [functional.alltypes t3]
+|     HDFS partitions=24/24 files=24 size=478.45KB
+|     runtime filters: RF002 -> t3.id
+|     row-size=4B cardinality=7.30K
 |
-08:SCAN HDFS [functional.alltypes t3]
-   HDFS partitions=24/24 files=24 size=478.45KB
-   runtime filters: RF000 -> t3.id, RF002 -> t3.id
-   row-size=4B cardinality=7.30K
+09:SCAN HDFS [functional.alltypestiny t4]
+   HDFS partitions=4/4 files=4 size=460B
+   runtime filters: RF000 -> t4.id
+   row-size=4B cardinality=8
 ====
 # Same as above but with anti joins.
 select 1 from
@@ -1479,64 +1481,64 @@ where t2.month = 1
 PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN]
-|  hash predicates: b.id = t4.id
-|  runtime filters: RF000 <- t4.id
+|  hash predicates: t4.id = b.id
+|  runtime filters: RF000 <- b.id
 |  row-size=16B cardinality=1
 |
-|--09:SCAN HDFS [functional.alltypestiny t4]
-|     HDFS partitions=4/4 files=4 size=460B
-|     row-size=4B cardinality=8
-|
-11:HASH JOIN [RIGHT ANTI JOIN]
-|  hash predicates: t3.id = b.id
-|  row-size=12B cardinality=620
-|
-|--10:HASH JOIN [INNER JOIN]
-|  |  hash predicates: b.id = count(a.id)
-|  |  runtime filters: RF002 <- count(a.id)
-|  |  row-size=12B cardinality=620
+|--11:HASH JOIN [RIGHT ANTI JOIN]
+|  |  hash predicates: t3.id = b.id
+|  |  row-size=12B cardinality=1
 |  |
-|  |--04:AGGREGATE [FINALIZE]
-|  |  |  output: count(a.id)
-|  |  |  row-size=8B cardinality=1
+|  |--10:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: b.id = count(a.id)
+|  |  |  runtime filters: RF002 <- count(a.id)
+|  |  |  row-size=12B cardinality=1
 |  |  |
-|  |  03:AGGREGATE
-|  |  |  group by: a.id
-|  |  |  row-size=4B cardinality=8
+|  |  |--04:AGGREGATE [FINALIZE]
+|  |  |  |  output: count(a.id)
+|  |  |  |  row-size=8B cardinality=1
+|  |  |  |
+|  |  |  03:AGGREGATE
+|  |  |  |  group by: a.id
+|  |  |  |  row-size=4B cardinality=8
+|  |  |  |
+|  |  |  02:HASH JOIN [INNER JOIN]
+|  |  |  |  hash predicates: a.id = b.id
+|  |  |  |  runtime filters: RF004 <- b.id
+|  |  |  |  row-size=8B cardinality=8
+|  |  |  |
+|  |  |  |--01:SCAN HDFS [functional.alltypestiny b]
+|  |  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |  |     row-size=4B cardinality=8
+|  |  |  |
+|  |  |  00:SCAN HDFS [functional.alltypestiny a]
+|  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |     runtime filters: RF004 -> a.id
+|  |  |     row-size=4B cardinality=8
 |  |  |
-|  |  02:HASH JOIN [INNER JOIN]
-|  |  |  hash predicates: a.id = b.id
-|  |  |  runtime filters: RF004 <- b.id
-|  |  |  row-size=8B cardinality=8
+|  |  07:HASH JOIN [LEFT ANTI JOIN]
+|  |  |  hash predicates: b.id = a.id
+|  |  |  row-size=4B cardinality=620
 |  |  |
-|  |  |--01:SCAN HDFS [functional.alltypestiny b]
+|  |  |--05:SCAN HDFS [functional.alltypestiny a]
 |  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |     runtime filters: RF002 -> a.id
 |  |  |     row-size=4B cardinality=8
 |  |  |
-|  |  00:SCAN HDFS [functional.alltypestiny a]
-|  |     HDFS partitions=4/4 files=4 size=460B
-|  |     runtime filters: RF004 -> a.id
-|  |     row-size=4B cardinality=8
-|  |
-|  07:HASH JOIN [LEFT ANTI JOIN]
-|  |  hash predicates: b.id = a.id
-|  |  row-size=4B cardinality=620
-|  |
-|  |--05:SCAN HDFS [functional.alltypestiny a]
-|  |     HDFS partitions=4/4 files=4 size=460B
-|  |     runtime filters: RF000 -> a.id, RF002 -> a.id
-|  |     row-size=4B cardinality=8
+|  |  06:SCAN HDFS [functional.alltypes b]
+|  |     partition predicates: b.month = 1
+|  |     HDFS partitions=2/24 files=2 size=40.32KB
+|  |     runtime filters: RF002 -> b.id
+|  |     row-size=4B cardinality=620
 |  |
-|  06:SCAN HDFS [functional.alltypes b]
-|     partition predicates: b.month = 1
-|     HDFS partitions=2/24 files=2 size=40.32KB
-|     runtime filters: RF000 -> b.id, RF002 -> b.id
-|     row-size=4B cardinality=620
+|  08:SCAN HDFS [functional.alltypes t3]
+|     HDFS partitions=24/24 files=24 size=478.45KB
+|     row-size=4B cardinality=7.30K
 |
-08:SCAN HDFS [functional.alltypes t3]
-   HDFS partitions=24/24 files=24 size=478.45KB
-   runtime filters: RF000 -> t3.id
-   row-size=4B cardinality=7.30K
+09:SCAN HDFS [functional.alltypestiny t4]
+   HDFS partitions=4/4 files=4 size=460B
+   runtime filters: RF000 -> t4.id
+   row-size=4B cardinality=8
 ====
 # Regression test for IMPALA-1343.
 SELECT sum(t4.tinyint_col)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
index 678054d..f2adf2b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
@@ -318,7 +318,7 @@ PLAN-ROOT SINK
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: a.int_col = int_col
 |  runtime filters: RF000 <- int_col
-|  row-size=17B cardinality=100
+|  row-size=17B cardinality=30
 |
 |--01:UNION
 |     constant-operands=3
@@ -336,7 +336,7 @@ PLAN-ROOT SINK
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: a.int_col = int_col
 |  runtime filters: RF000 <- int_col
-|  row-size=17B cardinality=100
+|  row-size=17B cardinality=30
 |
 |--03:EXCHANGE [BROADCAST]
 |  |
@@ -1431,7 +1431,7 @@ PLAN-ROOT SINK
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = x, a.int_col = y
 |  runtime filters: RF000 <- x, RF001 <- y
-|  row-size=33B cardinality=7.30K
+|  row-size=33B cardinality=11
 |
 |--01:UNION
 |  |  row-size=25B cardinality=11
@@ -1573,7 +1573,7 @@ PLAN-ROOT SINK
 04:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: b.id = id
 |  runtime filters: RF000 <- id
-|  row-size=34B cardinality=7
+|  row-size=34B cardinality=1
 |
 |--06:EXCHANGE [HASH(id)]
 |  |
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans-default.test b/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans-default.test
index 63ddfa7..c210ae3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans-default.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans-default.test
@@ -305,7 +305,7 @@ PLAN-ROOT SINK
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.int_col = min(t2.`year`)
 |  runtime filters: RF000 <- min(t2.`year`)
-|  row-size=8B cardinality=8
+|  row-size=8B cardinality=4
 |
 |--02:AGGREGATE [FINALIZE]
 |  |  output: min(t2.`year`)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test b/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test
index 4a9b6a8..232d074 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test
@@ -253,7 +253,7 @@ PLAN-ROOT SINK
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.int_col = min(t2.`year`)
 |  runtime filters: RF000 <- min(t2.`year`)
-|  row-size=8B cardinality=8
+|  row-size=8B cardinality=4
 |
 |--02:AGGREGATE [FINALIZE]
 |  |  output: min(t2.`year`)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
index 1500f1b..a1bd3d8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
@@ -119,10 +119,10 @@ ORDER BY t_s_secyear.customer_id,
          t_s_secyear.customer_preferred_cust_flag
 LIMIT 100;
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=495.00MB Threads=19
-Per-Host Resource Estimates: Memory=2.19GB
+Max Per-Host Resource Reservation: Memory=461.00MB Threads=19
+Per-Host Resource Estimates: Memory=2.16GB
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=2.19GB mem-reservation=495.00MB thread-reservation=19 runtime-filters-memory=10.00MB
+|  Per-Host Resources: mem-estimate=2.16GB mem-reservation=461.00MB thread-reservation=19 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
@@ -131,7 +131,7 @@ PLAN-ROOT SINK
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  mem-estimate=7.52KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=84 row-size=77B cardinality=100
-|  in pipelines: 47(GETNEXT), 13(OPEN)
+|  in pipelines: 47(GETNEXT), 27(OPEN)
 |
 46:HASH JOIN [INNER JOIN]
 |  hash predicates: customer_id = customer_id
@@ -139,8 +139,8 @@ PLAN-ROOT SINK
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
 |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
 |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=26,12,40,54,68,82 row-size=313B cardinality=589.03K
-|  in pipelines: 13(GETNEXT), 41(OPEN)
+|  tuple-ids=54,26,68,12,40,82 row-size=313B cardinality=29.06K
+|  in pipelines: 27(GETNEXT), 41(OPEN)
 |
 |--35:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -208,185 +208,133 @@ PLAN-ROOT SINK
 45:HASH JOIN [INNER JOIN]
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
+|  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
 |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
-|  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=26,12,40,54,68 row-size=269B cardinality=589.03K
-|  in pipelines: 13(GETNEXT), 34(OPEN)
+|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=54,26,68,12,40 row-size=269B cardinality=29.06K
+|  in pipelines: 27(GETNEXT), 13(OPEN)
 |
-|--28:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=68 row-size=44B cardinality=14.80K
-|  |  in pipelines: 34(GETNEXT)
-|  |
-|  34:AGGREGATE [FINALIZE]
-|  |  output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=67 row-size=169B cardinality=14.80K
-|  |  in pipelines: 34(GETNEXT), 29(OPEN)
-|  |
-|  33:HASH JOIN [INNER JOIN]
-|  |  hash predicates: c_customer_sk = ws_bill_customer_sk
-|  |  fk/pk conjuncts: c_customer_sk = ws_bill_customer_sk
-|  |  runtime filters: RF027[min_max] <- ws_bill_customer_sk
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=64,65,66 row-size=185B cardinality=148.00K
-|  |  in pipelines: 29(GETNEXT), 30(OPEN)
+|--44:HASH JOIN [INNER JOIN]
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF008[bloom] <- customer_id, RF009[min_max] <- customer_id
+|  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=26,68,12,40 row-size=225B cardinality=29.06K
+|  |  in pipelines: 13(GETNEXT), 20(OPEN)
 |  |
-|  |--32:HASH JOIN [INNER JOIN]
-|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
-|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
-|  |  |  runtime filters: RF028[bloom] <- d_date_sk, RF029[min_max] <- d_date_sk
-|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
-|  |  |  in pipelines: 30(GETNEXT), 31(OPEN)
+|  |--14:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=40 row-size=44B cardinality=29.46K
+|  |  |  in pipelines: 20(GETNEXT)
 |  |  |
-|  |  |--31:SCAN HDFS [tpcds_parquet.date_dim]
-|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  20:AGGREGATE [FINALIZE]
+|  |  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=52.01MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=35 row-size=169B cardinality=29.46K
+|  |  |  in pipelines: 20(GETNEXT), 15(OPEN)
+|  |  |
+|  |  19:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: c_customer_sk = cs_bill_customer_sk
+|  |  |  fk/pk conjuncts: c_customer_sk = cs_bill_customer_sk
+|  |  |  runtime filters: RF027[min_max] <- cs_bill_customer_sk
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=32,33,34 row-size=185B cardinality=294.63K
+|  |  |  in pipelines: 15(GETNEXT), 16(OPEN)
+|  |  |
+|  |  |--18:HASH JOIN [INNER JOIN]
+|  |  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  |  runtime filters: RF028[bloom] <- d_date_sk, RF029[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
+|  |  |  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |  |  |
+|  |  |  |--17:SCAN HDFS [tpcds_parquet.date_dim]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |     tuple-ids=34 row-size=8B cardinality=373
+|  |  |  |     in pipelines: 17(GETNEXT)
+|  |  |  |
+|  |  |  16:SCAN HDFS [tpcds_parquet.catalog_sales]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF029[min_max] -> cs_sold_date_sk, RF028[bloom] -> cs_sold_date_sk
 |  |  |     stored statistics:
-|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       table: rows=1.44M size=96.62MB
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=66 row-size=8B cardinality=373
-|  |  |     in pipelines: 31(GETNEXT)
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
+|  |  |     tuple-ids=33 row-size=24B cardinality=1.44M
+|  |  |     in pipelines: 16(GETNEXT)
 |  |  |
-|  |  30:SCAN HDFS [tpcds_parquet.web_sales]
-|  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF029[min_max] -> ws_sold_date_sk, RF028[bloom] -> ws_sold_date_sk
+|  |  15:SCAN HDFS [tpcds_parquet.customer]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF027[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
-|  |       table: rows=719.38K size=45.09MB
+|  |       table: rows=100.00K size=5.49MB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
-|  |     mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
-|  |     tuple-ids=65 row-size=24B cardinality=719.38K
-|  |     in pipelines: 30(GETNEXT)
-|  |
-|  29:SCAN HDFS [tpcds_parquet.customer]
-|     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF027[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
-|     stored statistics:
-|       table: rows=100.00K size=5.49MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=64 row-size=153B cardinality=100.00K
-|     in pipelines: 29(GETNEXT)
-|
-44:HASH JOIN [INNER JOIN]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
-|  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=26,12,40,54 row-size=225B cardinality=589.03K
-|  in pipelines: 13(GETNEXT), 27(OPEN)
-|
-|--21:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=54 row-size=44B cardinality=294.63K
-|  |  in pipelines: 27(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|  |     tuple-ids=32 row-size=153B cardinality=100.00K
+|  |     in pipelines: 15(GETNEXT)
 |  |
-|  27:AGGREGATE [FINALIZE]
-|  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=52.01MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=49 row-size=169B cardinality=294.63K
-|  |  in pipelines: 27(GETNEXT), 22(OPEN)
-|  |
-|  26:HASH JOIN [INNER JOIN]
-|  |  hash predicates: c_customer_sk = cs_bill_customer_sk
-|  |  fk/pk conjuncts: c_customer_sk = cs_bill_customer_sk
-|  |  runtime filters: RF023[min_max] <- cs_bill_customer_sk
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=46,47,48 row-size=185B cardinality=294.63K
-|  |  in pipelines: 22(GETNEXT), 23(OPEN)
+|  43:HASH JOIN [INNER JOIN]
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF011[min_max] <- customer_id
+|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  tuple-ids=26,68,12 row-size=181B cardinality=29.06K
+|  |  in pipelines: 13(GETNEXT), 06(OPEN)
 |  |
-|  |--25:HASH JOIN [INNER JOIN]
-|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
-|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
-|  |  |  runtime filters: RF024[bloom] <- d_date_sk, RF025[min_max] <- d_date_sk
-|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=47,48 row-size=32B cardinality=294.63K
-|  |  |  in pipelines: 23(GETNEXT), 24(OPEN)
+|  |--00:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=12 row-size=44B cardinality=58.90K
+|  |  |  in pipelines: 06(GETNEXT)
 |  |  |
-|  |  |--24:SCAN HDFS [tpcds_parquet.date_dim]
-|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
+|  |  06:AGGREGATE [FINALIZE]
+|  |  |  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=103.99MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=169B cardinality=58.90K
+|  |  |  in pipelines: 06(GETNEXT), 02(OPEN)
+|  |  |
+|  |  05:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  runtime filters: RF023[min_max] <- c_customer_sk
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=1,2,0 row-size=185B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |  |
+|  |  |--01:SCAN HDFS [tpcds_parquet.customer]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF008[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |  |     stored statistics:
-|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       table: rows=100.00K size=5.49MB
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=48 row-size=8B cardinality=373
-|  |  |     in pipelines: 24(GETNEXT)
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|  |  |     tuple-ids=0 row-size=153B cardinality=100.00K
+|  |  |     in pipelines: 01(GETNEXT)
 |  |  |
-|  |  23:SCAN HDFS [tpcds_parquet.catalog_sales]
-|  |     HDFS partitions=1/1 files=3 size=96.62MB
-|  |     runtime filters: RF025[min_max] -> cs_sold_date_sk, RF024[bloom] -> cs_sold_date_sk
-|  |     stored statistics:
-|  |       table: rows=1.44M size=96.62MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
-|  |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|  |     tuple-ids=47 row-size=24B cardinality=1.44M
-|  |     in pipelines: 23(GETNEXT)
-|  |
-|  22:SCAN HDFS [tpcds_parquet.customer]
-|     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF023[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
-|     stored statistics:
-|       table: rows=100.00K size=5.49MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=46 row-size=153B cardinality=100.00K
-|     in pipelines: 22(GETNEXT)
-|
-43:HASH JOIN [INNER JOIN]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF006[bloom] <- customer_id, RF007[min_max] <- customer_id
-|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=26,12,40 row-size=181B cardinality=589.03K
-|  in pipelines: 13(GETNEXT), 20(OPEN)
-|
-|--14:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=40 row-size=44B cardinality=29.46K
-|  |  in pipelines: 20(GETNEXT)
-|  |
-|  20:AGGREGATE [FINALIZE]
-|  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=52.01MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=35 row-size=169B cardinality=29.46K
-|  |  in pipelines: 20(GETNEXT), 15(OPEN)
-|  |
-|  19:HASH JOIN [INNER JOIN]
-|  |  hash predicates: c_customer_sk = cs_bill_customer_sk
-|  |  fk/pk conjuncts: c_customer_sk = cs_bill_customer_sk
-|  |  runtime filters: RF019[min_max] <- cs_bill_customer_sk
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=32,33,34 row-size=185B cardinality=294.63K
-|  |  in pipelines: 15(GETNEXT), 16(OPEN)
-|  |
-|  |--18:HASH JOIN [INNER JOIN]
-|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
-|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
-|  |  |  runtime filters: RF020[bloom] <- d_date_sk, RF021[min_max] <- d_date_sk
+|  |  04:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  runtime filters: RF024[bloom] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
-|  |  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |  |
-|  |  |--17:SCAN HDFS [tpcds_parquet.date_dim]
+|  |  |--03:SCAN HDFS [tpcds_parquet.date_dim]
 |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |  |     stored statistics:
@@ -396,225 +344,277 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=34 row-size=8B cardinality=373
-|  |  |     in pipelines: 17(GETNEXT)
+|  |  |     tuple-ids=2 row-size=8B cardinality=373
+|  |  |     in pipelines: 03(GETNEXT)
 |  |  |
-|  |  16:SCAN HDFS [tpcds_parquet.catalog_sales]
-|  |     HDFS partitions=1/1 files=3 size=96.62MB
-|  |     runtime filters: RF021[min_max] -> cs_sold_date_sk, RF020[bloom] -> cs_sold_date_sk
+|  |  02:SCAN HDFS [tpcds_parquet.store_sales]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     runtime filters: RF023[min_max] -> ss_customer_sk, RF024[bloom] -> ss_sold_date_sk
 |  |     stored statistics:
-|  |       table: rows=1.44M size=96.62MB
+|  |       table: rows=2.88M size=200.95MB
+|  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
-|  |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|  |     tuple-ids=33 row-size=24B cardinality=1.44M
-|  |     in pipelines: 16(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
+|  |     tuple-ids=1 row-size=24B cardinality=2.88M
+|  |     in pipelines: 02(GETNEXT)
+|  |
+|  42:HASH JOIN [INNER JOIN]
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF012[bloom] <- customer_id, RF013[min_max] <- customer_id
+|  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=26,68 row-size=137B cardinality=29.06K
+|  |  in pipelines: 13(GETNEXT), 34(OPEN)
+|  |
+|  |--28:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=68 row-size=44B cardinality=14.80K
+|  |  |  in pipelines: 34(GETNEXT)
+|  |  |
+|  |  34:AGGREGATE [FINALIZE]
+|  |  |  output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=67 row-size=169B cardinality=14.80K
+|  |  |  in pipelines: 34(GETNEXT), 29(OPEN)
+|  |  |
+|  |  33:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: c_customer_sk = ws_bill_customer_sk
+|  |  |  fk/pk conjuncts: c_customer_sk = ws_bill_customer_sk
+|  |  |  runtime filters: RF019[min_max] <- ws_bill_customer_sk
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=64,65,66 row-size=185B cardinality=148.00K
+|  |  |  in pipelines: 29(GETNEXT), 30(OPEN)
+|  |  |
+|  |  |--32:HASH JOIN [INNER JOIN]
+|  |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  |  runtime filters: RF020[bloom] <- d_date_sk, RF021[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
+|  |  |  |  in pipelines: 30(GETNEXT), 31(OPEN)
+|  |  |  |
+|  |  |  |--31:SCAN HDFS [tpcds_parquet.date_dim]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |     tuple-ids=66 row-size=8B cardinality=373
+|  |  |  |     in pipelines: 31(GETNEXT)
+|  |  |  |
+|  |  |  30:SCAN HDFS [tpcds_parquet.web_sales]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF021[min_max] -> ws_sold_date_sk, RF020[bloom] -> ws_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
+|  |  |     tuple-ids=65 row-size=24B cardinality=719.38K
+|  |  |     in pipelines: 30(GETNEXT)
+|  |  |
+|  |  29:SCAN HDFS [tpcds_parquet.customer]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF019[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF008[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|  |     tuple-ids=64 row-size=153B cardinality=100.00K
+|  |     in pipelines: 29(GETNEXT)
 |  |
-|  15:SCAN HDFS [tpcds_parquet.customer]
-|     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF019[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id
-|     stored statistics:
-|       table: rows=100.00K size=5.49MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=32 row-size=153B cardinality=100.00K
-|     in pipelines: 15(GETNEXT)
-|
-42:HASH JOIN [INNER JOIN]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF009[min_max] <- customer_id
-|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=26,12 row-size=137B cardinality=589.03K
-|  in pipelines: 13(GETNEXT), 06(OPEN)
-|
-|--00:UNION
+|  07:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=12 row-size=44B cardinality=58.90K
-|  |  in pipelines: 06(GETNEXT)
+|  |  tuple-ids=26 row-size=93B cardinality=589.03K
+|  |  in pipelines: 13(GETNEXT)
 |  |
-|  06:AGGREGATE [FINALIZE]
+|  13:AGGREGATE [FINALIZE]
 |  |  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2) > CAST(0 AS DECIMAL(3,0))
 |  |  mem-estimate=103.99MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=58.90K
-|  |  in pipelines: 06(GETNEXT), 02(OPEN)
+|  |  tuple-ids=17 row-size=169B cardinality=589.03K
+|  |  in pipelines: 13(GETNEXT), 09(OPEN)
 |  |
-|  05:HASH JOIN [INNER JOIN]
+|  12:HASH JOIN [INNER JOIN]
 |  |  hash predicates: ss_customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  |  runtime filters: RF015[min_max] <- c_customer_sk
 |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=1,2,0 row-size=185B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |  tuple-ids=15,16,14 row-size=185B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT), 08(OPEN)
 |  |
-|  |--01:SCAN HDFS [tpcds_parquet.customer]
+|  |--08:SCAN HDFS [tpcds_parquet.customer]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF007[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF006[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     runtime filters: RF013[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF012[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF008[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
 |  |       table: rows=100.00K size=5.49MB
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=153B cardinality=100.00K
-|  |     in pipelines: 01(GETNEXT)
+|  |     tuple-ids=14 row-size=153B cardinality=100.00K
+|  |     in pipelines: 08(GETNEXT)
 |  |
-|  04:HASH JOIN [INNER JOIN]
+|  11:HASH JOIN [INNER JOIN]
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  runtime filters: RF016[bloom] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |  tuple-ids=15,16 row-size=32B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT), 10(OPEN)
 |  |
-|  |--03:SCAN HDFS [tpcds_parquet.date_dim]
+|  |--10:SCAN HDFS [tpcds_parquet.date_dim]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |  |     stored statistics:
 |  |       table: rows=73.05K size=2.15MB
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=8B cardinality=373
-|  |     in pipelines: 03(GETNEXT)
+|  |     tuple-ids=16 row-size=8B cardinality=373
+|  |     in pipelines: 10(GETNEXT)
 |  |
-|  02:SCAN HDFS [tpcds_parquet.store_sales]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  09:SCAN HDFS [tpcds_parquet.store_sales]
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
 |     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
+|       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=1 row-size=24B cardinality=2.88M
-|     in pipelines: 02(GETNEXT)
+|     tuple-ids=15 row-size=24B cardinality=2.88M
+|     in pipelines: 09(GETNEXT)
 |
-07:UNION
+21:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=26 row-size=93B cardinality=589.03K
-|  in pipelines: 13(GETNEXT)
+|  tuple-ids=54 row-size=44B cardinality=294.63K
+|  in pipelines: 27(GETNEXT)
 |
-13:AGGREGATE [FINALIZE]
-|  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
+27:AGGREGATE [FINALIZE]
+|  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
 |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  mem-estimate=103.99MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=17 row-size=169B cardinality=589.03K
-|  in pipelines: 13(GETNEXT), 09(OPEN)
+|  mem-estimate=52.01MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=49 row-size=169B cardinality=294.63K
+|  in pipelines: 27(GETNEXT), 22(OPEN)
 |
-12:HASH JOIN [INNER JOIN]
-|  hash predicates: ss_customer_sk = c_customer_sk
-|  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  runtime filters: RF011[min_max] <- c_customer_sk
+26:HASH JOIN [INNER JOIN]
+|  hash predicates: c_customer_sk = cs_bill_customer_sk
+|  fk/pk conjuncts: c_customer_sk = cs_bill_customer_sk
+|  runtime filters: RF005[min_max] <- cs_bill_customer_sk
 |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=15,16,14 row-size=185B cardinality=589.03K
-|  in pipelines: 09(GETNEXT), 08(OPEN)
-|
-|--08:SCAN HDFS [tpcds_parquet.customer]
-|     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF007[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF006[bloom] -> tpcds_parqu [...]
-|     stored statistics:
-|       table: rows=100.00K size=5.49MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=14 row-size=153B cardinality=100.00K
-|     in pipelines: 08(GETNEXT)
-|
-11:HASH JOIN [INNER JOIN]
-|  hash predicates: ss_sold_date_sk = d_date_sk
-|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF012[bloom] <- d_date_sk
-|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=15,16 row-size=32B cardinality=589.03K
-|  in pipelines: 09(GETNEXT), 10(OPEN)
+|  tuple-ids=46,47,48 row-size=185B cardinality=294.63K
+|  in pipelines: 22(GETNEXT), 23(OPEN)
 |
-|--10:SCAN HDFS [tpcds_parquet.date_dim]
-|     HDFS partitions=1/1 files=1 size=2.15MB
-|     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
+|--25:HASH JOIN [INNER JOIN]
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=47,48 row-size=32B cardinality=294.63K
+|  |  in pipelines: 23(GETNEXT), 24(OPEN)
+|  |
+|  |--24:SCAN HDFS [tpcds_parquet.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
+|  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |     tuple-ids=48 row-size=8B cardinality=373
+|  |     in pipelines: 24(GETNEXT)
+|  |
+|  23:SCAN HDFS [tpcds_parquet.catalog_sales]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     runtime filters: RF007[min_max] -> cs_sold_date_sk, RF006[bloom] -> cs_sold_date_sk
 |     stored statistics:
-|       table: rows=73.05K size=2.15MB
+|       table: rows=1.44M size=96.62MB
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=16 row-size=8B cardinality=373
-|     in pipelines: 10(GETNEXT)
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
+|     tuple-ids=47 row-size=24B cardinality=1.44M
+|     in pipelines: 23(GETNEXT)
 |
-09:SCAN HDFS [tpcds_parquet.store_sales]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
-   runtime filters: RF011[min_max] -> ss_customer_sk, RF012[bloom] -> ss_sold_date_sk
+22:SCAN HDFS [tpcds_parquet.customer]
+   HDFS partitions=1/1 files=1 size=5.49MB
+   runtime filters: RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> c_customer_sk, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
    stored statistics:
-     table: rows=2.88M size=200.96MB
-     partitions: 1824/1824 rows=2.88M
+     table: rows=100.00K size=5.49MB
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
-   mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=15 row-size=24B cardinality=2.88M
-   in pipelines: 09(GETNEXT)
+   extrapolated-rows=disabled max-scan-range-rows=100.00K
+   mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+   tuple-ids=46 row-size=153B cardinality=100.00K
+   in pipelines: 22(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=603.75MB Threads=49
-Per-Host Resource Estimates: Memory=2.67GB
-F36:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+Max Per-Host Resource Reservation: Memory=547.81MB Threads=50
+Per-Host Resource Estimates: Memory=2.60GB
+F37:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-83:MERGING-EXCHANGE [UNPARTITIONED]
+84:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  limit: 100
 |  mem-estimate=26.23KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=84 row-size=77B cardinality=100
 |  in pipelines: 47(GETNEXT)
 |
-F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=137.44MB mem-reservation=105.12MB thread-reservation=1 runtime-filters-memory=4.00MB
+F24:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=25.46MB mem-reservation=16.31MB thread-reservation=1 runtime-filters-memory=2.00MB
 47:TOP-N [LIMIT=100]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  mem-estimate=7.52KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=84 row-size=77B cardinality=100
 |  in pipelines: 47(GETNEXT), 52(OPEN)
 |
-46:HASH JOIN [INNER JOIN, BROADCAST]
+46:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
 |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
-|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=26,12,40,54,68,82 row-size=313B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 81(OPEN)
+|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=54,26,68,12,40,82 row-size=313B cardinality=29.06K
+|  in pipelines: 52(GETNEXT), 82(OPEN)
 |
-|--82:EXCHANGE [BROADCAST]
-|  |  mem-estimate=6.30MB mem-reservation=0B thread-reservation=0
+|--83:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=3.20MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=82 row-size=44B cardinality=148.00K
-|  |  in pipelines: 81(GETNEXT)
+|  |  in pipelines: 82(GETNEXT)
 |  |
-|  F35:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F36:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
 |  35:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=82 row-size=44B cardinality=148.00K
-|  |  in pipelines: 81(GETNEXT)
+|  |  in pipelines: 82(GETNEXT)
 |  |
-|  81:AGGREGATE [FINALIZE]
+|  82:AGGREGATE [FINALIZE]
 |  |  output: sum:merge((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2))
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=81 row-size=169B cardinality=148.00K
-|  |  in pipelines: 81(GETNEXT), 37(OPEN)
+|  |  in pipelines: 82(GETNEXT), 37(OPEN)
 |  |
-|  80:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  81:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=81 row-size=169B cardinality=148.00K
 |  |  in pipelines: 37(GETNEXT)
 |  |
-|  F33:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  F34:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=46.49MB mem-reservation=34.00MB thread-reservation=1
 |  41:AGGREGATE [STREAMING]
 |  |  output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
@@ -631,12 +631,12 @@ Per-Host Resources: mem-estimate=137.44MB mem-reservation=105.12MB thread-reserv
 |  |  tuple-ids=79,80,78 row-size=185B cardinality=148.00K
 |  |  in pipelines: 37(GETNEXT), 36(OPEN)
 |  |
-|  |--79:EXCHANGE [HASH(c_customer_sk)]
+|  |--80:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=78 row-size=153B cardinality=100.00K
 |  |  |  in pipelines: 36(GETNEXT)
 |  |  |
-|  |  F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=2
 |  |  36:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
@@ -648,12 +648,12 @@ Per-Host Resources: mem-estimate=137.44MB mem-reservation=105.12MB thread-reserv
 |  |     tuple-ids=78 row-size=153B cardinality=100.00K
 |  |     in pipelines: 36(GETNEXT)
 |  |
-|  78:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  79:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=2.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=79,80 row-size=32B cardinality=148.00K
 |  |  in pipelines: 37(GETNEXT)
 |  |
-|  F30:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F31:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=194.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB
 |  39:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ws_sold_date_sk = d_date_sk
@@ -663,12 +663,12 @@ Per-Host Resources: mem-estimate=137.44MB mem-reservation=105.12MB thread-reserv
 |  |  tuple-ids=79,80 row-size=32B cardinality=148.00K
 |  |  in pipelines: 37(GETNEXT), 38(OPEN)
 |  |
-|  |--77:EXCHANGE [BROADCAST]
+|  |--78:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=80 row-size=8B cardinality=373
 |  |  |  in pipelines: 38(GETNEXT)
 |  |  |
-|  |  F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
 |  |  38:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -694,538 +694,546 @@ Per-Host Resources: mem-estimate=137.44MB mem-reservation=105.12MB thread-reserv
 |     tuple-ids=79 row-size=24B cardinality=719.38K
 |     in pipelines: 37(GETNEXT)
 |
-45:HASH JOIN [INNER JOIN, BROADCAST]
+45:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
+|  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
 |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
-|  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=26,12,40,54,68 row-size=269B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 75(OPEN)
+|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=54,26,68,12,40 row-size=269B cardinality=29.06K
+|  in pipelines: 52(GETNEXT), 57(OPEN)
 |
-|--76:EXCHANGE [BROADCAST]
-|  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=68 row-size=44B cardinality=14.80K
-|  |  in pipelines: 75(GETNEXT)
-|  |
-|  F29:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
-|  28:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=68 row-size=44B cardinality=14.80K
-|  |  in pipelines: 75(GETNEXT)
-|  |
-|  75:AGGREGATE [FINALIZE]
-|  |  output: sum:merge((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=67 row-size=169B cardinality=14.80K
-|  |  in pipelines: 75(GETNEXT), 30(OPEN)
-|  |
-|  74:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=67 row-size=169B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT)
-|  |
-|  F27:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=46.49MB mem-reservation=34.00MB thread-reservation=1
-|  34:AGGREGATE [STREAMING]
-|  |  output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=67 row-size=169B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT)
-|  |
-|  33:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash predicates: ws_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
-|  |  runtime filters: RF027[min_max] <- c_customer_sk
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=65,66,64 row-size=185B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT), 29(OPEN)
+|--44:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF009[min_max] <- customer_id
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=26,68,12,40 row-size=225B cardinality=29.06K
+|  |  in pipelines: 57(GETNEXT), 75(OPEN)
 |  |
-|  |--73:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=64 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 29(GETNEXT)
+|  |--76:EXCHANGE [HASH(customer_id)]
+|  |  |  mem-estimate=566.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=40 row-size=44B cardinality=29.46K
+|  |  |  in pipelines: 75(GETNEXT)
 |  |  |
-|  |  F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB
-|  |  29:SCAN HDFS [tpcds_parquet.customer, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Host Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1
+|  |  14:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=40 row-size=44B cardinality=29.46K
+|  |  |  in pipelines: 75(GETNEXT)
+|  |  |
+|  |  75:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=35 row-size=169B cardinality=29.46K
+|  |  |  in pipelines: 75(GETNEXT), 16(OPEN)
+|  |  |
+|  |  74:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  |  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=35 row-size=169B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT)
+|  |  |
+|  |  F28:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+|  |  Per-Host Resources: mem-estimate=38.77MB mem-reservation=25.50MB thread-reservation=1
+|  |  20:AGGREGATE [STREAMING]
+|  |  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=35 row-size=169B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT)
+|  |  |
+|  |  19:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash predicates: cs_bill_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
+|  |  |  runtime filters: RF027[min_max] <- c_customer_sk
+|  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  tuple-ids=33,34,32 row-size=185B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT), 15(OPEN)
+|  |  |
+|  |  |--73:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=32 row-size=153B cardinality=100.00K
+|  |  |  |  in pipelines: 15(GETNEXT)
+|  |  |  |
+|  |  |  F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB
+|  |  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|  |  |     tuple-ids=32 row-size=153B cardinality=100.00K
+|  |  |     in pipelines: 15(GETNEXT)
+|  |  |
+|  |  72:EXCHANGE [HASH(cs_bill_customer_sk)]
+|  |  |  mem-estimate=3.11MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT)
+|  |  |
+|  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Host Resources: mem-estimate=290.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB
+|  |  18:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  runtime filters: RF028[bloom] <- d_date_sk, RF029[min_max] <- d_date_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |  |
+|  |  |--71:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=34 row-size=8B cardinality=373
+|  |  |  |  in pipelines: 17(GETNEXT)
+|  |  |  |
+|  |  |  F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
+|  |  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |     tuple-ids=34 row-size=8B cardinality=373
+|  |  |     in pipelines: 17(GETNEXT)
+|  |  |
+|  |  16:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF029[min_max] -> cs_sold_date_sk, RF027[min_max] -> cs_bill_customer_sk, RF028[bloom] -> cs_sold_date_sk
 |  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
+|  |       table: rows=1.44M size=96.62MB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=64 row-size=153B cardinality=100.00K
-|  |     in pipelines: 29(GETNEXT)
-|  |
-|  72:EXCHANGE [HASH(ws_bill_customer_sk)]
-|  |  mem-estimate=2.34MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
+|  |     tuple-ids=33 row-size=24B cardinality=1.44M
+|  |     in pipelines: 16(GETNEXT)
 |  |
-|  F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=194.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB
-|  32:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: ws_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
-|  |  runtime filters: RF028[bloom] <- d_date_sk, RF029[min_max] <- d_date_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT), 31(OPEN)
-|  |
-|  |--71:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=66 row-size=8B cardinality=373
-|  |  |  in pipelines: 31(GETNEXT)
+|  43:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF011[min_max] <- customer_id
+|  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=26,68,12 row-size=181B cardinality=29.06K
+|  |  in pipelines: 57(GETNEXT), 68(OPEN)
+|  |
+|  |--70:EXCHANGE [HASH(customer_id)]
+|  |  |  mem-estimate=987.66KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=12 row-size=44B cardinality=58.90K
+|  |  |  in pipelines: 68(GETNEXT)
 |  |  |
-|  |  F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
-|  |  31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=66 row-size=8B cardinality=373
-|  |     in pipelines: 31(GETNEXT)
-|  |
-|  30:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
-|     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF029[min_max] -> ws_sold_date_sk, RF027[min_max] -> ws_bill_customer_sk, RF028[bloom] -> ws_sold_date_sk
-|     stored statistics:
-|       table: rows=719.38K size=45.09MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=644.77K
-|     mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=65 row-size=24B cardinality=719.38K
-|     in pipelines: 30(GETNEXT)
-|
-44:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
-|  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=26,12,40,54 row-size=225B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 69(OPEN)
-|
-|--70:EXCHANGE [BROADCAST]
-|  |  mem-estimate=10.14MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=54 row-size=44B cardinality=294.63K
-|  |  in pipelines: 69(GETNEXT)
-|  |
-|  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1
-|  21:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=54 row-size=44B cardinality=294.63K
-|  |  in pipelines: 69(GETNEXT)
-|  |
-|  69:AGGREGATE [FINALIZE]
-|  |  output: sum:merge((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=49 row-size=169B cardinality=294.63K
-|  |  in pipelines: 69(GETNEXT), 23(OPEN)
-|  |
-|  68:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=49 row-size=169B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT)
-|  |
-|  F21:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=38.77MB mem-reservation=25.50MB thread-reservation=1
-|  27:AGGREGATE [STREAMING]
-|  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=49 row-size=169B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT)
-|  |
-|  26:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash predicates: cs_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
-|  |  runtime filters: RF023[min_max] <- c_customer_sk
-|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=47,48,46 row-size=185B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT), 22(OPEN)
-|  |
-|  |--67:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=46 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 22(GETNEXT)
+|  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Host Resources: mem-estimate=45.17MB mem-reservation=34.00MB thread-reservation=1
+|  |  00:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=12 row-size=44B cardinality=58.90K
+|  |  |  in pipelines: 68(GETNEXT)
 |  |  |
-|  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB
-|  |  22:SCAN HDFS [tpcds_parquet.customer, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
-|  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=46 row-size=153B cardinality=100.00K
-|  |     in pipelines: 22(GETNEXT)
-|  |
-|  66:EXCHANGE [HASH(cs_bill_customer_sk)]
-|  |  mem-estimate=3.11MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=47,48 row-size=32B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT)
-|  |
-|  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=290.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB
-|  25:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: cs_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
-|  |  runtime filters: RF024[bloom] <- d_date_sk, RF025[min_max] <- d_date_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=47,48 row-size=32B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT), 24(OPEN)
-|  |
-|  |--65:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=48 row-size=8B cardinality=373
-|  |  |  in pipelines: 24(GETNEXT)
+|  |  68:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2)
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=34.66MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=169B cardinality=58.90K
+|  |  |  in pipelines: 68(GETNEXT), 02(OPEN)
 |  |  |
-|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
-|  |  24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=48 row-size=8B cardinality=373
-|  |     in pipelines: 24(GETNEXT)
-|  |
-|  23:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
-|     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF025[min_max] -> cs_sold_date_sk, RF023[min_max] -> cs_bill_customer_sk, RF024[bloom] -> cs_sold_date_sk
-|     stored statistics:
-|       table: rows=1.44M size=96.62MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=650.14K
-|     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=47 row-size=24B cardinality=1.44M
-|     in pipelines: 23(GETNEXT)
-|
-43:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF006[bloom] <- customer_id, RF007[min_max] <- customer_id
-|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=26,12,40 row-size=181B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 63(OPEN)
-|
-|--64:EXCHANGE [BROADCAST]
-|  |  mem-estimate=1.38MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=40 row-size=44B cardinality=29.46K
-|  |  in pipelines: 63(GETNEXT)
-|  |
-|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1
-|  14:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=40 row-size=44B cardinality=29.46K
-|  |  in pipelines: 63(GETNEXT)
-|  |
-|  63:AGGREGATE [FINALIZE]
-|  |  output: sum:merge((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=35 row-size=169B cardinality=29.46K
-|  |  in pipelines: 63(GETNEXT), 16(OPEN)
-|  |
-|  62:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=35 row-size=169B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT)
-|  |
-|  F15:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=38.77MB mem-reservation=25.50MB thread-reservation=1
-|  20:AGGREGATE [STREAMING]
-|  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=35 row-size=169B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT)
-|  |
-|  19:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash predicates: cs_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
-|  |  runtime filters: RF019[min_max] <- c_customer_sk
-|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=33,34,32 row-size=185B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT), 15(OPEN)
-|  |
-|  |--61:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=32 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 15(GETNEXT)
+|  |  67:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  |  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=169B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT)
 |  |  |
-|  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=131.00MB mem-reservation=11.00MB thread-reservation=2 runtime-filters-memory=3.00MB
-|  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  F21:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
+|  |  Per-Host Resources: mem-estimate=58.76MB mem-reservation=42.50MB thread-reservation=1
+|  |  06:AGGREGATE [STREAMING]
+|  |  |  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=169B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT)
+|  |  |
+|  |  05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  runtime filters: RF023[min_max] <- c_customer_sk
+|  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  tuple-ids=1,2,0 row-size=185B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |  |
+|  |  |--66:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=0 row-size=153B cardinality=100.00K
+|  |  |  |  in pipelines: 01(GETNEXT)
+|  |  |  |
+|  |  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB
+|  |  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|  |  |     tuple-ids=0 row-size=153B cardinality=100.00K
+|  |  |     in pipelines: 01(GETNEXT)
+|  |  |
+|  |  65:EXCHANGE [HASH(ss_customer_sk)]
+|  |  |  mem-estimate=6.11MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT)
+|  |  |
+|  |  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Host Resources: mem-estimate=82.95MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB
+|  |  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  runtime filters: RF024[bloom] <- d_date_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |  |
+|  |  |--64:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=2 row-size=8B cardinality=373
+|  |  |  |  in pipelines: 03(GETNEXT)
+|  |  |  |
+|  |  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
+|  |  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |     tuple-ids=2 row-size=8B cardinality=373
+|  |  |     in pipelines: 03(GETNEXT)
+|  |  |
+|  |  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     runtime filters: RF023[min_max] -> ss_customer_sk, RF024[bloom] -> ss_sold_date_sk
 |  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
+|  |       table: rows=2.88M size=200.95MB
+|  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=32 row-size=153B cardinality=100.00K
-|  |     in pipelines: 15(GETNEXT)
-|  |
-|  60:EXCHANGE [HASH(cs_bill_customer_sk)]
-|  |  mem-estimate=3.11MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT)
-|  |
-|  F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=290.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB
-|  18:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: cs_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
-|  |  runtime filters: RF020[bloom] <- d_date_sk, RF021[min_max] <- d_date_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
+|  |     tuple-ids=1 row-size=24B cardinality=2.88M
+|  |     in pipelines: 02(GETNEXT)
+|  |
+|  69:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=1.69MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=26,68 row-size=137B cardinality=29.06K
+|  |  in pipelines: 57(GETNEXT)
 |  |
-|  |--59:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=34 row-size=8B cardinality=373
-|  |  |  in pipelines: 17(GETNEXT)
+|  F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Resources: mem-estimate=49.76MB mem-reservation=37.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  42:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF012[bloom] <- customer_id, RF013[min_max] <- customer_id
+|  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=26,68 row-size=137B cardinality=29.06K
+|  |  in pipelines: 57(GETNEXT), 62(OPEN)
+|  |
+|  |--63:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=68 row-size=44B cardinality=14.80K
+|  |  |  in pipelines: 62(GETNEXT)
 |  |  |
-|  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
-|  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
+|  |  28:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=68 row-size=44B cardinality=14.80K
+|  |  |  in pipelines: 62(GETNEXT)
+|  |  |
+|  |  62:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=67 row-size=169B cardinality=14.80K
+|  |  |  in pipelines: 62(GETNEXT), 30(OPEN)
+|  |  |
+|  |  61:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  |  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=67 row-size=169B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT)
+|  |  |
+|  |  F15:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |  Per-Host Resources: mem-estimate=46.49MB mem-reservation=34.00MB thread-reservation=1
+|  |  34:AGGREGATE [STREAMING]
+|  |  |  output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=67 row-size=169B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT)
+|  |  |
+|  |  33:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
+|  |  |  runtime filters: RF019[min_max] <- c_customer_sk
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=65,66,64 row-size=185B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT), 29(OPEN)
+|  |  |
+|  |  |--60:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=64 row-size=153B cardinality=100.00K
+|  |  |  |  in pipelines: 29(GETNEXT)
+|  |  |  |
+|  |  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB
+|  |  |  29:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|  |  |     tuple-ids=64 row-size=153B cardinality=100.00K
+|  |  |     in pipelines: 29(GETNEXT)
+|  |  |
+|  |  59:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  |  mem-estimate=2.34MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT)
+|  |  |
+|  |  F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Host Resources: mem-estimate=194.95MB mem-reservation=18.94MB thread-reservation=2 runtime-filters-memory=1.00MB
+|  |  32:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  runtime filters: RF020[bloom] <- d_date_sk, RF021[min_max] <- d_date_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT), 31(OPEN)
+|  |  |
+|  |  |--58:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=66 row-size=8B cardinality=373
+|  |  |  |  in pipelines: 31(GETNEXT)
+|  |  |  |
+|  |  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
+|  |  |  31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |     tuple-ids=66 row-size=8B cardinality=373
+|  |  |     in pipelines: 31(GETNEXT)
+|  |  |
+|  |  30:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF021[min_max] -> ws_sold_date_sk, RF019[min_max] -> ws_bill_customer_sk, RF020[bloom] -> ws_sold_date_sk
 |  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
+|  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=34 row-size=8B cardinality=373
-|  |     in pipelines: 17(GETNEXT)
-|  |
-|  16:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
-|     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF021[min_max] -> cs_sold_date_sk, RF019[min_max] -> cs_bill_customer_sk, RF020[bloom] -> cs_sold_date_sk
-|     stored statistics:
-|       table: rows=1.44M size=96.62MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=650.14K
-|     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=33 row-size=24B cardinality=1.44M
-|     in pipelines: 16(GETNEXT)
-|
-42:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF009[min_max] <- customer_id
-|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=26,12 row-size=137B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 57(OPEN)
-|
-|--58:EXCHANGE [BROADCAST]
-|  |  mem-estimate=2.61MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=12 row-size=44B cardinality=58.90K
-|  |  in pipelines: 57(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
+|  |     tuple-ids=65 row-size=24B cardinality=719.38K
+|  |     in pipelines: 30(GETNEXT)
 |  |
-|  F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=45.17MB mem-reservation=34.00MB thread-reservation=1
-|  00:UNION
+|  07:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=12 row-size=44B cardinality=58.90K
+|  |  tuple-ids=26 row-size=93B cardinality=589.03K
 |  |  in pipelines: 57(GETNEXT)
 |  |
 |  57:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2)
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2) > CAST(0 AS DECIMAL(3,0))
 |  |  mem-estimate=34.66MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=58.90K
-|  |  in pipelines: 57(GETNEXT), 02(OPEN)
+|  |  tuple-ids=17 row-size=169B cardinality=589.03K
+|  |  in pipelines: 57(GETNEXT), 09(OPEN)
 |  |
 |  56:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  tuple-ids=17 row-size=169B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT)
 |  |
 |  F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=58.76MB mem-reservation=42.50MB thread-reservation=1
-|  06:AGGREGATE [STREAMING]
+|  13:AGGREGATE [STREAMING]
 |  |  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  tuple-ids=17 row-size=169B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT)
 |  |
-|  05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  12:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash predicates: ss_customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  |  runtime filters: RF015[min_max] <- c_customer_sk
 |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=1,2,0 row-size=185B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |  tuple-ids=15,16,14 row-size=185B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT), 08(OPEN)
 |  |
 |  |--55:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 01(GETNEXT)
+|  |  |  tuple-ids=14 row-size=153B cardinality=100.00K
+|  |  |  in pipelines: 08(GETNEXT)
 |  |  |
 |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=132.00MB mem-reservation=12.00MB thread-reservation=2 runtime-filters-memory=4.00MB
-|  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  |  08:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF007[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF006[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     runtime filters: RF013[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF012[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
 |  |       table: rows=100.00K size=5.49MB
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=153B cardinality=100.00K
-|  |     in pipelines: 01(GETNEXT)
+|  |     tuple-ids=14 row-size=153B cardinality=100.00K
+|  |     in pipelines: 08(GETNEXT)
 |  |
 |  54:EXCHANGE [HASH(ss_customer_sk)]
 |  |  mem-estimate=6.11MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  tuple-ids=15,16 row-size=32B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT)
 |  |
 |  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=82.95MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB
-|  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  11:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  runtime filters: RF016[bloom] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |  tuple-ids=15,16 row-size=32B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT), 10(OPEN)
 |  |
 |  |--53:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=8B cardinality=373
-|  |  |  in pipelines: 03(GETNEXT)
+|  |  |  tuple-ids=16 row-size=8B cardinality=373
+|  |  |  in pipelines: 10(GETNEXT)
 |  |  |
 |  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
-|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |  |     stored statistics:
 |  |       table: rows=73.05K size=2.15MB
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=8B cardinality=373
-|  |     in pipelines: 03(GETNEXT)
+|  |     tuple-ids=16 row-size=8B cardinality=373
+|  |     in pipelines: 10(GETNEXT)
 |  |
-|  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
 |     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
+|       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=1 row-size=24B cardinality=2.88M
-|     in pipelines: 02(GETNEXT)
+|     tuple-ids=15 row-size=24B cardinality=2.88M
+|     in pipelines: 09(GETNEXT)
+|
+77:EXCHANGE [HASH(customer_id)]
+|  mem-estimate=4.26MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=54 row-size=44B cardinality=294.63K
+|  in pipelines: 52(GETNEXT)
 |
-07:UNION
+F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1
+21:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=26 row-size=93B cardinality=589.03K
+|  tuple-ids=54 row-size=44B cardinality=294.63K
 |  in pipelines: 52(GETNEXT)
 |
 52:AGGREGATE [FINALIZE]
-|  output: sum:merge(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2)
+|  output: sum:merge((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2))
 |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  mem-estimate=34.66MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=17 row-size=169B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 09(OPEN)
+|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=49 row-size=169B cardinality=294.63K
+|  in pipelines: 52(GETNEXT), 23(OPEN)
 |
 51:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=17 row-size=169B cardinality=589.03K
-|  in pipelines: 09(GETNEXT)
+|  tuple-ids=49 row-size=169B cardinality=294.63K
+|  in pipelines: 23(GETNEXT)
 |
-F03:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=58.76MB mem-reservation=42.50MB thread-reservation=1
-13:AGGREGATE [STREAMING]
-|  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
+F03:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=39.77MB mem-reservation=26.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+27:AGGREGATE [STREAMING]
+|  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
 |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=17 row-size=169B cardinality=589.03K
-|  in pipelines: 09(GETNEXT)
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=49 row-size=169B cardinality=294.63K
+|  in pipelines: 23(GETNEXT)
 |
-12:HASH JOIN [INNER JOIN, PARTITIONED]
-|  hash predicates: ss_customer_sk = c_customer_sk
-|  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  runtime filters: RF011[min_max] <- c_customer_sk
+26:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash predicates: cs_bill_customer_sk = c_customer_sk
+|  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
+|  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=15,16,14 row-size=185B cardinality=589.03K
-|  in pipelines: 09(GETNEXT), 08(OPEN)
+|  tuple-ids=47,48,46 row-size=185B cardinality=294.63K
+|  in pipelines: 23(GETNEXT), 22(OPEN)
 |
 |--50:EXCHANGE [HASH(c_customer_sk)]
 |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=14 row-size=153B cardinality=100.00K
-|  |  in pipelines: 08(GETNEXT)
+|  |  tuple-ids=46 row-size=153B cardinality=100.00K
+|  |  in pipelines: 22(GETNEXT)
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=132.00MB mem-reservation=12.00MB thread-reservation=2 runtime-filters-memory=4.00MB
-|  08:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  22:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF007[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF006[bloom] -> tpcds_parqu [...]
+|     runtime filters: RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=14 row-size=153B cardinality=100.00K
-|     in pipelines: 08(GETNEXT)
+|     tuple-ids=46 row-size=153B cardinality=100.00K
+|     in pipelines: 22(GETNEXT)
 |
-49:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=6.11MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=15,16 row-size=32B cardinality=589.03K
-|  in pipelines: 09(GETNEXT)
+49:EXCHANGE [HASH(cs_bill_customer_sk)]
+|  mem-estimate=3.11MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=47,48 row-size=32B cardinality=294.63K
+|  in pipelines: 23(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=82.95MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB
-11:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash predicates: ss_sold_date_sk = d_date_sk
-|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF012[bloom] <- d_date_sk
+Per-Host Resources: mem-estimate=291.95MB mem-reservation=19.94MB thread-reservation=2 runtime-filters-memory=2.00MB
+25:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: cs_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=15,16 row-size=32B cardinality=589.03K
-|  in pipelines: 09(GETNEXT), 10(OPEN)
+|  tuple-ids=47,48 row-size=32B cardinality=294.63K
+|  in pipelines: 23(GETNEXT), 24(OPEN)
 |
 |--48:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16 row-size=8B cardinality=373
-|  |  in pipelines: 10(GETNEXT)
+|  |  tuple-ids=48 row-size=8B cardinality=373
+|  |  in pipelines: 24(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
-|  10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |     HDFS partitions=1/1 files=1 size=2.15MB
 |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |     stored statistics:
@@ -1235,86 +1243,85 @@ Per-Host Resources: mem-estimate=82.95MB mem-reservation=6.94MB thread-reservati
 |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=16 row-size=8B cardinality=373
-|     in pipelines: 10(GETNEXT)
+|     tuple-ids=48 row-size=8B cardinality=373
+|     in pipelines: 24(GETNEXT)
 |
-09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
-   runtime filters: RF011[min_max] -> ss_customer_sk, RF012[bloom] -> ss_sold_date_sk
+23:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+   HDFS partitions=1/1 files=3 size=96.62MB
+   runtime filters: RF007[min_max] -> cs_sold_date_sk, RF005[min_max] -> cs_bill_customer_sk, RF006[bloom] -> cs_sold_date_sk, RF004[bloom] -> cs_bill_customer_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
-     partitions: 1824/1824 rows=2.88M
+     table: rows=1.44M size=96.62MB
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
-   mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=15 row-size=24B cardinality=2.88M
-   in pipelines: 09(GETNEXT)
+   extrapolated-rows=disabled max-scan-range-rows=650.14K
+   mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
+   tuple-ids=47 row-size=24B cardinality=1.44M
+   in pipelines: 23(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=766.50MB Threads=56
-Per-Host Resource Estimates: Memory=1.25GB
-F36:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
+Max Per-Host Resource Reservation: Memory=646.31MB Threads=57
+Per-Host Resource Estimates: Memory=1.12GB
+F37:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-83:MERGING-EXCHANGE [UNPARTITIONED]
+84:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  limit: 100
-|  mem-estimate=49.95KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=26.23KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=84 row-size=77B cardinality=100
 |  in pipelines: 47(GETNEXT)
 |
-F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-reservation=1
+F24:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservation=1
 47:TOP-N [LIMIT=100]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  mem-estimate=7.52KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=84 row-size=77B cardinality=100
 |  in pipelines: 47(GETNEXT), 52(OPEN)
 |
-46:HASH JOIN [INNER JOIN, BROADCAST]
+46:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
-|  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=26,12,40,54,68,82 row-size=313B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 81(OPEN)
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=54,26,68,12,40,82 row-size=313B cardinality=29.06K
+|  in pipelines: 52(GETNEXT), 82(OPEN)
 |
-|--F37:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=41.30MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F38:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.95MB mem-reservation=5.75MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: customer_id
 |  |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
 |  |
-|  82:EXCHANGE [BROADCAST]
-|  |  mem-estimate=6.30MB mem-reservation=0B thread-reservation=0
+|  83:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=3.20MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=82 row-size=44B cardinality=148.00K
-|  |  in pipelines: 81(GETNEXT)
+|  |  in pipelines: 82(GETNEXT)
 |  |
-|  F35:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F36:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
 |  35:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=82 row-size=44B cardinality=148.00K
-|  |  in pipelines: 81(GETNEXT)
+|  |  in pipelines: 82(GETNEXT)
 |  |
-|  81:AGGREGATE [FINALIZE]
+|  82:AGGREGATE [FINALIZE]
 |  |  output: sum:merge((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2))
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=81 row-size=169B cardinality=148.00K
-|  |  in pipelines: 81(GETNEXT), 37(OPEN)
+|  |  in pipelines: 82(GETNEXT), 37(OPEN)
 |  |
-|  80:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  81:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=81 row-size=169B cardinality=148.00K
 |  |  in pipelines: 37(GETNEXT)
 |  |
-|  F33:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  F34:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=19.34MB mem-reservation=17.00MB thread-reservation=1
 |  41:AGGREGATE [STREAMING]
 |  |  output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
@@ -1331,7 +1338,7 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  tuple-ids=79,80,78 row-size=185B cardinality=148.00K
 |  |  in pipelines: 37(GETNEXT), 36(OPEN)
 |  |
-|  |--F38:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |--F39:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=27.15MB mem-reservation=17.00MB thread-reservation=1
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
@@ -1339,12 +1346,12 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  |  runtime filters: RF031[min_max] <- c_customer_sk
 |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  |
-|  |  79:EXCHANGE [HASH(c_customer_sk)]
+|  |  80:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=78 row-size=153B cardinality=100.00K
 |  |  |  in pipelines: 36(GETNEXT)
 |  |  |
-|  |  F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
 |  |  36:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
@@ -1356,12 +1363,12 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |     tuple-ids=78 row-size=153B cardinality=100.00K
 |  |     in pipelines: 36(GETNEXT)
 |  |
-|  78:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  79:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=2.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=79,80 row-size=32B cardinality=148.00K
 |  |  in pipelines: 37(GETNEXT)
 |  |
-|  F30:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F31:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1
 |  39:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1372,7 +1379,7 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  tuple-ids=79,80 row-size=32B cardinality=148.00K
 |  |  in pipelines: 37(GETNEXT), 38(OPEN)
 |  |
-|  |--F39:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |--F40:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
@@ -1380,12 +1387,12 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  |  runtime filters: RF032[bloom] <- d_date_sk, RF033[min_max] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
-|  |  77:EXCHANGE [BROADCAST]
+|  |  78:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=80 row-size=8B cardinality=373
 |  |  |  in pipelines: 38(GETNEXT)
 |  |  |
-|  |  F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
 |  |  38:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -1411,660 +1418,668 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |     tuple-ids=79 row-size=24B cardinality=719.38K
 |     in pipelines: 37(GETNEXT)
 |
-45:HASH JOIN [INNER JOIN, BROADCAST]
+45:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=03
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=26,12,40,54,68 row-size=269B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 75(OPEN)
+|  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=54,26,68,12,40 row-size=269B cardinality=29.06K
+|  in pipelines: 52(GETNEXT), 57(OPEN)
 |
-|--F40:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=7.46MB mem-reservation=6.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F41:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=7.86MB mem-reservation=5.75MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: customer_id
 |  |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
-|  |  mem-estimate=5.75MB mem-reservation=5.75MB spill-buffer=128.00KB thread-reservation=0
-|  |
-|  76:EXCHANGE [BROADCAST]
-|  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=68 row-size=44B cardinality=14.80K
-|  |  in pipelines: 75(GETNEXT)
-|  |
-|  F29:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
-|  28:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=68 row-size=44B cardinality=14.80K
-|  |  in pipelines: 75(GETNEXT)
-|  |
-|  75:AGGREGATE [FINALIZE]
-|  |  output: sum:merge((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=67 row-size=169B cardinality=14.80K
-|  |  in pipelines: 75(GETNEXT), 30(OPEN)
-|  |
-|  74:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=67 row-size=169B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT)
-|  |
-|  F27:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
-|  Per-Instance Resources: mem-estimate=19.34MB mem-reservation=17.00MB thread-reservation=1
-|  34:AGGREGATE [STREAMING]
-|  |  output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=67 row-size=169B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT)
+|  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
 |  |
-|  33:HASH JOIN [INNER JOIN, PARTITIONED]
+|  44:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=04
-|  |  hash predicates: ws_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=65,66,64 row-size=185B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT), 29(OPEN)
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=26,68,12,40 row-size=225B cardinality=29.06K
+|  |  in pipelines: 57(GETNEXT), 75(OPEN)
 |  |
-|  |--F41:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
-|  |  |  Per-Instance Resources: mem-estimate=27.15MB mem-reservation=17.00MB thread-reservation=1
+|  |--F42:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=2.49MB mem-reservation=1.94MB thread-reservation=1
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=03
-|  |  |  build expressions: c_customer_sk
-|  |  |  runtime filters: RF027[min_max] <- c_customer_sk
+|  |  |  build expressions: customer_id
+|  |  |  runtime filters: RF009[min_max] <- customer_id
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |
+|  |  76:EXCHANGE [HASH(customer_id)]
+|  |  |  mem-estimate=566.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=40 row-size=44B cardinality=29.46K
+|  |  |  in pipelines: 75(GETNEXT)
+|  |  |
+|  |  F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1
+|  |  14:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=40 row-size=44B cardinality=29.46K
+|  |  |  in pipelines: 75(GETNEXT)
+|  |  |
+|  |  75:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=35 row-size=169B cardinality=29.46K
+|  |  |  in pipelines: 75(GETNEXT), 16(OPEN)
+|  |  |
+|  |  74:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  |  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=35 row-size=169B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT)
+|  |  |
+|  |  F28:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=20.11MB mem-reservation=17.00MB thread-reservation=1
+|  |  20:AGGREGATE [STREAMING]
+|  |  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=35 row-size=169B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT)
 |  |  |
-|  |  73:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=64 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 29(GETNEXT)
+|  |  19:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=05
+|  |  |  hash predicates: cs_bill_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
+|  |  |  tuple-ids=33,34,32 row-size=185B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT), 15(OPEN)
+|  |  |
+|  |  |--F43:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=18.65MB mem-reservation=8.50MB thread-reservation=1
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=05 plan-id=06 cohort-id=04
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF027[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  |
+|  |  |  73:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=32 row-size=153B cardinality=100.00K
+|  |  |  |  in pipelines: 15(GETNEXT)
+|  |  |  |
+|  |  |  F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
+|  |  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
+|  |  |     tuple-ids=32 row-size=153B cardinality=100.00K
+|  |  |     in pipelines: 15(GETNEXT)
+|  |  |
+|  |  72:EXCHANGE [HASH(cs_bill_customer_sk)]
+|  |  |  mem-estimate=3.11MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT)
 |  |  |
-|  |  F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  29:SCAN HDFS [tpcds_parquet.customer, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
-|  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=64 row-size=153B cardinality=100.00K
-|  |     in pipelines: 29(GETNEXT)
-|  |
-|  72:EXCHANGE [HASH(ws_bill_customer_sk)]
-|  |  mem-estimate=2.34MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT)
-|  |
-|  F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-|  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1
-|  32:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash-table-id=05
-|  |  hash predicates: ws_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
-|  |  in pipelines: 30(GETNEXT), 31(OPEN)
-|  |
-|  |--F42:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  JOIN BUILD
-|  |  |  join-table-id=05 plan-id=06 cohort-id=03
-|  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF028[bloom] <- d_date_sk, RF029[min_max] <- d_date_sk
-|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
+|  |  Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1
+|  |  18:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=06
+|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
+|  |  |  in pipelines: 16(GETNEXT), 17(OPEN)
 |  |  |
-|  |  71:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=66 row-size=8B cardinality=373
-|  |  |  in pipelines: 31(GETNEXT)
+|  |  |--F44:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=06 plan-id=07 cohort-id=04
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF028[bloom] <- d_date_sk, RF029[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |
+|  |  |  71:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=34 row-size=8B cardinality=373
+|  |  |  |  in pipelines: 17(GETNEXT)
+|  |  |  |
+|  |  |  F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=34 row-size=8B cardinality=373
+|  |  |     in pipelines: 17(GETNEXT)
 |  |  |
-|  |  F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  16:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF029[min_max] -> cs_sold_date_sk, RF027[min_max] -> cs_bill_customer_sk, RF028[bloom] -> cs_sold_date_sk
 |  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
+|  |       table: rows=1.44M size=96.62MB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=66 row-size=8B cardinality=373
-|  |     in pipelines: 31(GETNEXT)
-|  |
-|  30:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
-|     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF029[min_max] -> ws_sold_date_sk, RF027[min_max] -> ws_bill_customer_sk, RF028[bloom] -> ws_sold_date_sk
-|     stored statistics:
-|       table: rows=719.38K size=45.09MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=644.77K
-|     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-|     tuple-ids=65 row-size=24B cardinality=719.38K
-|     in pipelines: 30(GETNEXT)
-|
-44:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash-table-id=06
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
-|  mem-estimate=0B mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=26,12,40,54 row-size=225B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 69(OPEN)
-|
-|--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=79.14MB mem-reservation=69.00MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  JOIN BUILD
-|  |  join-table-id=06 plan-id=07 cohort-id=01
-|  |  build expressions: customer_id
-|  |  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  |  mem-estimate=68.00MB mem-reservation=68.00MB spill-buffer=2.00MB thread-reservation=0
-|  |
-|  70:EXCHANGE [BROADCAST]
-|  |  mem-estimate=10.14MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=54 row-size=44B cardinality=294.63K
-|  |  in pipelines: 69(GETNEXT)
-|  |
-|  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Instance Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1
-|  21:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=54 row-size=44B cardinality=294.63K
-|  |  in pipelines: 69(GETNEXT)
-|  |
-|  69:AGGREGATE [FINALIZE]
-|  |  output: sum:merge((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=49 row-size=169B cardinality=294.63K
-|  |  in pipelines: 69(GETNEXT), 23(OPEN)
-|  |
-|  68:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=49 row-size=169B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT)
-|  |
-|  F21:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
-|  Per-Instance Resources: mem-estimate=20.11MB mem-reservation=17.00MB thread-reservation=1
-|  27:AGGREGATE [STREAMING]
-|  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=49 row-size=169B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
+|  |     tuple-ids=33 row-size=24B cardinality=1.44M
+|  |     in pipelines: 16(GETNEXT)
 |  |
-|  26:HASH JOIN [INNER JOIN, PARTITIONED]
+|  43:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=07
-|  |  hash predicates: cs_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=47,48,46 row-size=185B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT), 22(OPEN)
-|  |
-|  |--F44:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=18.65MB mem-reservation=8.50MB thread-reservation=1
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=26,68,12 row-size=181B cardinality=29.06K
+|  |  in pipelines: 57(GETNEXT), 68(OPEN)
+|  |
+|  |--F45:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=3.98MB mem-reservation=2.88MB thread-reservation=1
 |  |  JOIN BUILD
-|  |  |  join-table-id=07 plan-id=08 cohort-id=04
-|  |  |  build expressions: c_customer_sk
-|  |  |  runtime filters: RF023[min_max] <- c_customer_sk
-|  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  join-table-id=07 plan-id=08 cohort-id=03
+|  |  |  build expressions: customer_id
+|  |  |  runtime filters: RF011[min_max] <- customer_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  |  |
-|  |  67:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=46 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 22(GETNEXT)
+|  |  70:EXCHANGE [HASH(customer_id)]
+|  |  |  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=12 row-size=44B cardinality=58.90K
+|  |  |  in pipelines: 68(GETNEXT)
 |  |  |
-|  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  22:SCAN HDFS [tpcds_parquet.customer, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
-|  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=46 row-size=153B cardinality=100.00K
-|  |     in pipelines: 22(GETNEXT)
-|  |
-|  66:EXCHANGE [HASH(cs_bill_customer_sk)]
-|  |  mem-estimate=3.11MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=47,48 row-size=32B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT)
-|  |
-|  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-|  Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1
-|  25:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash-table-id=08
-|  |  hash predicates: cs_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=47,48 row-size=32B cardinality=294.63K
-|  |  in pipelines: 23(GETNEXT), 24(OPEN)
-|  |
-|  |--F45:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  JOIN BUILD
-|  |  |  join-table-id=08 plan-id=09 cohort-id=04
-|  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF024[bloom] <- d_date_sk, RF025[min_max] <- d_date_sk
-|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
+|  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1
+|  |  00:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=12 row-size=44B cardinality=58.90K
+|  |  |  in pipelines: 68(GETNEXT)
 |  |  |
-|  |  65:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=48 row-size=8B cardinality=373
-|  |  |  in pipelines: 24(GETNEXT)
+|  |  68:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2)
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=169B cardinality=58.90K
+|  |  |  in pipelines: 68(GETNEXT), 02(OPEN)
 |  |  |
-|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
-|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=48 row-size=8B cardinality=373
-|  |     in pipelines: 24(GETNEXT)
-|  |
-|  23:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
-|     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF025[min_max] -> cs_sold_date_sk, RF023[min_max] -> cs_bill_customer_sk, RF024[bloom] -> cs_sold_date_sk
-|     stored statistics:
-|       table: rows=1.44M size=96.62MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=650.14K
-|     mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
-|     tuple-ids=47 row-size=24B cardinality=1.44M
-|     in pipelines: 23(GETNEXT)
-|
-43:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash-table-id=09
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=26,12,40 row-size=181B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 63(OPEN)
-|
-|--F46:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=11.88MB mem-reservation=10.50MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  JOIN BUILD
-|  |  join-table-id=09 plan-id=10 cohort-id=01
-|  |  build expressions: customer_id
-|  |  runtime filters: RF006[bloom] <- customer_id, RF007[min_max] <- customer_id
-|  |  mem-estimate=9.50MB mem-reservation=9.50MB spill-buffer=256.00KB thread-reservation=0
-|  |
-|  64:EXCHANGE [BROADCAST]
-|  |  mem-estimate=1.38MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=40 row-size=44B cardinality=29.46K
-|  |  in pipelines: 63(GETNEXT)
-|  |
-|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Instance Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1
-|  14:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=40 row-size=44B cardinality=29.46K
-|  |  in pipelines: 63(GETNEXT)
-|  |
-|  63:AGGREGATE [FINALIZE]
-|  |  output: sum:merge((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=35 row-size=169B cardinality=29.46K
-|  |  in pipelines: 63(GETNEXT), 16(OPEN)
-|  |
-|  62:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=35 row-size=169B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT)
-|  |
-|  F15:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
-|  Per-Instance Resources: mem-estimate=20.11MB mem-reservation=17.00MB thread-reservation=1
-|  20:AGGREGATE [STREAMING]
-|  |  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=35 row-size=169B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT)
-|  |
-|  19:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash-table-id=10
-|  |  hash predicates: cs_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=33,34,32 row-size=185B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT), 15(OPEN)
-|  |
-|  |--F47:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=18.65MB mem-reservation=8.50MB thread-reservation=1
-|  |  JOIN BUILD
-|  |  |  join-table-id=10 plan-id=11 cohort-id=05
-|  |  |  build expressions: c_customer_sk
-|  |  |  runtime filters: RF019[min_max] <- c_customer_sk
-|  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  67:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  |  |  mem-estimate=11.01MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=169B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT)
 |  |  |
-|  |  61:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=32 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 15(GETNEXT)
+|  |  F21:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=23.23MB mem-reservation=17.00MB thread-reservation=1
+|  |  06:AGGREGATE [STREAMING]
+|  |  |  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=169B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT)
 |  |  |
-|  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=08
+|  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=1,2,0 row-size=185B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |  |
+|  |  |--F46:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|  |  |  |  Per-Instance Resources: mem-estimate=14.90MB mem-reservation=4.75MB thread-reservation=1
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=08 plan-id=09 cohort-id=05
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF023[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |
+|  |  |  66:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=0 row-size=153B cardinality=100.00K
+|  |  |  |  in pipelines: 01(GETNEXT)
+|  |  |  |
+|  |  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
+|  |  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
+|  |  |     tuple-ids=0 row-size=153B cardinality=100.00K
+|  |  |     in pipelines: 01(GETNEXT)
+|  |  |
+|  |  65:EXCHANGE [HASH(ss_customer_sk)]
+|  |  |  mem-estimate=6.23MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT)
+|  |  |
+|  |  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1
+|  |  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=09
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |  |
+|  |  |--F47:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=09 plan-id=10 cohort-id=05
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF024[bloom] <- d_date_sk
+|  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |
+|  |  |  64:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=2 row-size=8B cardinality=373
+|  |  |  |  in pipelines: 03(GETNEXT)
+|  |  |  |
+|  |  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=2 row-size=8B cardinality=373
+|  |  |     in pipelines: 03(GETNEXT)
+|  |  |
+|  |  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     runtime filters: RF023[min_max] -> ss_customer_sk, RF024[bloom] -> ss_sold_date_sk
 |  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
+|  |       table: rows=2.88M size=200.95MB
+|  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=32 row-size=153B cardinality=100.00K
-|  |     in pipelines: 15(GETNEXT)
-|  |
-|  60:EXCHANGE [HASH(cs_bill_customer_sk)]
-|  |  mem-estimate=3.11MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=1 row-size=24B cardinality=2.88M
+|  |     in pipelines: 02(GETNEXT)
+|  |
+|  69:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=2.11MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=26,68 row-size=137B cardinality=29.06K
+|  |  in pipelines: 57(GETNEXT)
 |  |
-|  F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-|  Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1
-|  18:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash-table-id=11
-|  |  hash predicates: cs_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=33,34 row-size=32B cardinality=294.63K
-|  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1
+|  42:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=10
+|  |  hash predicates: customer_id = customer_id
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=26,68 row-size=137B cardinality=29.06K
+|  |  in pipelines: 57(GETNEXT), 62(OPEN)
 |  |
 |  |--F48:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=7.46MB mem-reservation=6.75MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
-|  |  |  join-table-id=11 plan-id=12 cohort-id=05
-|  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF020[bloom] <- d_date_sk, RF021[min_max] <- d_date_sk
-|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  join-table-id=10 plan-id=11 cohort-id=03
+|  |  |  build expressions: customer_id
+|  |  |  runtime filters: RF012[bloom] <- customer_id, RF013[min_max] <- customer_id
+|  |  |  mem-estimate=5.75MB mem-reservation=5.75MB spill-buffer=128.00KB thread-reservation=0
 |  |  |
-|  |  59:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=34 row-size=8B cardinality=373
-|  |  |  in pipelines: 17(GETNEXT)
+|  |  63:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=68 row-size=44B cardinality=14.80K
+|  |  |  in pipelines: 62(GETNEXT)
 |  |  |
-|  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
+|  |  28:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=68 row-size=44B cardinality=14.80K
+|  |  |  in pipelines: 62(GETNEXT)
+|  |  |
+|  |  62:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  having: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / 2)) > CAST(0 AS DECIMAL(3,0))
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=67 row-size=169B cardinality=14.80K
+|  |  |  in pipelines: 62(GETNEXT), 30(OPEN)
+|  |  |
+|  |  61:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  |  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=67 row-size=169B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT)
+|  |  |
+|  |  F15:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=19.34MB mem-reservation=17.00MB thread-reservation=1
+|  |  34:AGGREGATE [STREAMING]
+|  |  |  output: sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
+|  |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=67 row-size=169B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT)
+|  |  |
+|  |  33:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=11
+|  |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=65,66,64 row-size=185B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT), 29(OPEN)
+|  |  |
+|  |  |--F49:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=27.15MB mem-reservation=17.00MB thread-reservation=1
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=11 plan-id=12 cohort-id=06
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF019[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  |
+|  |  |  60:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=64 row-size=153B cardinality=100.00K
+|  |  |  |  in pipelines: 29(GETNEXT)
+|  |  |  |
+|  |  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
+|  |  |  29:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
+|  |  |     tuple-ids=64 row-size=153B cardinality=100.00K
+|  |  |     in pipelines: 29(GETNEXT)
+|  |  |
+|  |  59:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  |  mem-estimate=2.34MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT)
+|  |  |
+|  |  F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=1
+|  |  32:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=12
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=65,66 row-size=32B cardinality=148.00K
+|  |  |  in pipelines: 30(GETNEXT), 31(OPEN)
+|  |  |
+|  |  |--F50:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=12 plan-id=13 cohort-id=06
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF020[bloom] <- d_date_sk, RF021[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |
+|  |  |  58:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=66 row-size=8B cardinality=373
+|  |  |  |  in pipelines: 31(GETNEXT)
+|  |  |  |
+|  |  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=66 row-size=8B cardinality=373
+|  |  |     in pipelines: 31(GETNEXT)
+|  |  |
+|  |  30:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF021[min_max] -> ws_sold_date_sk, RF019[min_max] -> ws_bill_customer_sk, RF020[bloom] -> ws_sold_date_sk
 |  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
+|  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=34 row-size=8B cardinality=373
-|  |     in pipelines: 17(GETNEXT)
-|  |
-|  16:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
-|     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF021[min_max] -> cs_sold_date_sk, RF019[min_max] -> cs_bill_customer_sk, RF020[bloom] -> cs_sold_date_sk
-|     stored statistics:
-|       table: rows=1.44M size=96.62MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=650.14K
-|     mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
-|     tuple-ids=33 row-size=24B cardinality=1.44M
-|     in pipelines: 16(GETNEXT)
-|
-42:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash-table-id=12
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=26,12 row-size=137B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 57(OPEN)
-|
-|--F49:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=19.75MB mem-reservation=17.00MB thread-reservation=1
-|  JOIN BUILD
-|  |  join-table-id=12 plan-id=13 cohort-id=01
-|  |  build expressions: customer_id
-|  |  runtime filters: RF009[min_max] <- customer_id
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=512.00KB thread-reservation=0
-|  |
-|  58:EXCHANGE [BROADCAST]
-|  |  mem-estimate=2.75MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=12 row-size=44B cardinality=58.90K
-|  |  in pipelines: 57(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
+|  |     tuple-ids=65 row-size=24B cardinality=719.38K
+|  |     in pipelines: 30(GETNEXT)
 |  |
-|  F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1
-|  00:UNION
+|  07:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=12 row-size=44B cardinality=58.90K
+|  |  tuple-ids=26 row-size=93B cardinality=589.03K
 |  |  in pipelines: 57(GETNEXT)
 |  |
 |  57:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2)
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2) > CAST(0 AS DECIMAL(3,0))
 |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=58.90K
-|  |  in pipelines: 57(GETNEXT), 02(OPEN)
+|  |  tuple-ids=17 row-size=169B cardinality=589.03K
+|  |  in pipelines: 57(GETNEXT), 09(OPEN)
 |  |
 |  56:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  |  mem-estimate=11.01MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  tuple-ids=17 row-size=169B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT)
 |  |
 |  F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
 |  Per-Instance Resources: mem-estimate=23.23MB mem-reservation=17.00MB thread-reservation=1
-|  06:AGGREGATE [STREAMING]
+|  13:AGGREGATE [STREAMING]
 |  |  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  tuple-ids=17 row-size=169B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT)
 |  |
-|  05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  12:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=13
 |  |  hash predicates: ss_customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  |  tuple-ids=1,2,0 row-size=185B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |  tuple-ids=15,16,14 row-size=185B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT), 08(OPEN)
 |  |
-|  |--F50:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|  |--F51:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
 |  |  |  Per-Instance Resources: mem-estimate=14.90MB mem-reservation=4.75MB thread-reservation=1
 |  |  JOIN BUILD
-|  |  |  join-table-id=13 plan-id=14 cohort-id=06
+|  |  |  join-table-id=13 plan-id=14 cohort-id=03
 |  |  |  build expressions: c_customer_sk
 |  |  |  runtime filters: RF015[min_max] <- c_customer_sk
 |  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
 |  |  |
 |  |  55:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 01(GETNEXT)
+|  |  |  tuple-ids=14 row-size=153B cardinality=100.00K
+|  |  |  in pipelines: 08(GETNEXT)
 |  |  |
 |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB
+|  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  08:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF007[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF006[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     runtime filters: RF013[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> tpcds_parquet.customer.c_customer_id, RF012[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
 |  |       table: rows=100.00K size=5.49MB
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=0 row-size=153B cardinality=100.00K
-|  |     in pipelines: 01(GETNEXT)
+|  |     tuple-ids=14 row-size=153B cardinality=100.00K
+|  |     in pipelines: 08(GETNEXT)
 |  |
 |  54:EXCHANGE [HASH(ss_customer_sk)]
 |  |  mem-estimate=6.23MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  tuple-ids=15,16 row-size=32B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT)
 |  |
 |  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1
-|  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  11:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=14
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=1,2 row-size=32B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |  tuple-ids=15,16 row-size=32B cardinality=589.03K
+|  |  in pipelines: 09(GETNEXT), 10(OPEN)
 |  |
-|  |--F51:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |--F52:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
-|  |  |  join-table-id=14 plan-id=15 cohort-id=06
+|  |  |  join-table-id=14 plan-id=15 cohort-id=03
 |  |  |  build expressions: d_date_sk
 |  |  |  runtime filters: RF016[bloom] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  53:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=8B cardinality=373
-|  |  |  in pipelines: 03(GETNEXT)
+|  |  |  tuple-ids=16 row-size=8B cardinality=373
+|  |  |  in pipelines: 10(GETNEXT)
 |  |  |
 |  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |  |     stored statistics:
 |  |       table: rows=73.05K size=2.15MB
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=2 row-size=8B cardinality=373
-|  |     in pipelines: 03(GETNEXT)
+|  |     tuple-ids=16 row-size=8B cardinality=373
+|  |     in pipelines: 10(GETNEXT)
 |  |
-|  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
 |     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
+|       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=1 row-size=24B cardinality=2.88M
-|     in pipelines: 02(GETNEXT)
+|     tuple-ids=15 row-size=24B cardinality=2.88M
+|     in pipelines: 09(GETNEXT)
 |
-07:UNION
+77:EXCHANGE [HASH(customer_id)]
+|  mem-estimate=4.26MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=54 row-size=44B cardinality=294.63K
+|  in pipelines: 52(GETNEXT)
+|
+F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=44.51MB mem-reservation=34.00MB thread-reservation=1
+21:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=26 row-size=93B cardinality=589.03K
+|  tuple-ids=54 row-size=44B cardinality=294.63K
 |  in pipelines: 52(GETNEXT)
 |
 52:AGGREGATE [FINALIZE]
-|  output: sum:merge(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / 2)
+|  output: sum:merge((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / 2))
 |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=17 row-size=169B cardinality=589.03K
-|  in pipelines: 52(GETNEXT), 09(OPEN)
+|  tuple-ids=49 row-size=169B cardinality=294.63K
+|  in pipelines: 52(GETNEXT), 23(OPEN)
 |
 51:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  mem-estimate=11.01MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=17 row-size=169B cardinality=589.03K
-|  in pipelines: 09(GETNEXT)
+|  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=49 row-size=169B cardinality=294.63K
+|  in pipelines: 23(GETNEXT)
 |
-F03:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=23.23MB mem-reservation=17.00MB thread-reservation=1
-13:AGGREGATE [STREAMING]
-|  output: sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + ss_ext_sales_price) / CAST(2 AS DECIMAL(3,0)))
+F03:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=20.11MB mem-reservation=17.00MB thread-reservation=1
+27:AGGREGATE [STREAMING]
+|  output: sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + cs_ext_sales_price) / CAST(2 AS DECIMAL(3,0))))
 |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=17 row-size=169B cardinality=589.03K
-|  in pipelines: 09(GETNEXT)
+|  tuple-ids=49 row-size=169B cardinality=294.63K
+|  in pipelines: 23(GETNEXT)
 |
-12:HASH JOIN [INNER JOIN, PARTITIONED]
+26:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=15
-|  hash predicates: ss_customer_sk = c_customer_sk
-|  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=15,16,14 row-size=185B cardinality=589.03K
-|  in pipelines: 09(GETNEXT), 08(OPEN)
+|  hash predicates: cs_bill_customer_sk = c_customer_sk
+|  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=47,48,46 row-size=185B cardinality=294.63K
+|  in pipelines: 23(GETNEXT), 22(OPEN)
 |
-|--F52:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-|  |  Per-Instance Resources: mem-estimate=14.90MB mem-reservation=4.75MB thread-reservation=1
+|--F53:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=19.65MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=15 plan-id=16 cohort-id=01
 |  |  build expressions: c_customer_sk
-|  |  runtime filters: RF011[min_max] <- c_customer_sk
-|  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
+|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |
 |  50:EXCHANGE [HASH(c_customer_sk)]
 |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=14 row-size=153B cardinality=100.00K
-|  |  in pipelines: 08(GETNEXT)
+|  |  tuple-ids=46 row-size=153B cardinality=100.00K
+|  |  in pipelines: 22(GETNEXT)
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
-|  08:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  22:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF007[min_max] -> tpcds_parquet.customer.c_customer_id, RF009[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF006[bloom] -> tpcds_parqu [...]
+|     runtime filters: RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=14 row-size=153B cardinality=100.00K
-|     in pipelines: 08(GETNEXT)
+|     tuple-ids=46 row-size=153B cardinality=100.00K
+|     in pipelines: 22(GETNEXT)
 |
-49:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=6.23MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=15,16 row-size=32B cardinality=589.03K
-|  in pipelines: 09(GETNEXT)
+49:EXCHANGE [HASH(cs_bill_customer_sk)]
+|  mem-estimate=3.11MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=47,48 row-size=32B cardinality=294.63K
+|  in pipelines: 23(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1
-11:HASH JOIN [INNER JOIN, BROADCAST]
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=1
+25:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=16
-|  hash predicates: ss_sold_date_sk = d_date_sk
-|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  hash predicates: cs_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=15,16 row-size=32B cardinality=589.03K
-|  in pipelines: 09(GETNEXT), 10(OPEN)
+|  tuple-ids=47,48 row-size=32B cardinality=294.63K
+|  in pipelines: 23(GETNEXT), 24(OPEN)
 |
-|--F53:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|--F54:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=16 plan-id=17 cohort-id=01
 |  |  build expressions: d_date_sk
-|  |  runtime filters: RF012[bloom] <- d_date_sk
+|  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  48:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16 row-size=8B cardinality=373
-|  |  in pipelines: 10(GETNEXT)
+|  |  tuple-ids=48 row-size=8B cardinality=373
+|  |  in pipelines: 24(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  10:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |     HDFS partitions=1/1 files=1 size=2.15MB
 |     predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |     stored statistics:
@@ -2074,18 +2089,17 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2002 AS INT)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|     tuple-ids=16 row-size=8B cardinality=373
-|     in pipelines: 10(GETNEXT)
+|     tuple-ids=48 row-size=8B cardinality=373
+|     in pipelines: 24(GETNEXT)
 |
-09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
-   runtime filters: RF011[min_max] -> ss_customer_sk, RF012[bloom] -> ss_sold_date_sk
+23:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+   HDFS partitions=1/1 files=3 size=96.62MB
+   runtime filters: RF007[min_max] -> cs_sold_date_sk, RF005[min_max] -> cs_bill_customer_sk, RF006[bloom] -> cs_sold_date_sk, RF004[bloom] -> cs_bill_customer_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
-     partitions: 1824/1824 rows=2.88M
+     table: rows=1.44M size=96.62MB
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
-   mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=15 row-size=24B cardinality=2.88M
-   in pipelines: 09(GETNEXT)
+   extrapolated-rows=disabled max-scan-range-rows=650.14K
+   mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
+   tuple-ids=47 row-size=24B cardinality=1.44M
+   in pipelines: 23(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
index 3160b91..82440bf 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
@@ -173,29 +173,11 @@ PLAN-ROOT SINK
 |  |  in pipelines: 26(GETNEXT), 18(OPEN), 20(OPEN)
 |  |
 |  25:HASH JOIN [INNER JOIN]
-|  |  hash predicates: wsr_web_site_sk = web_site_sk
-|  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF008[bloom] <- web_site_sk, RF009[min_max] <- web_site_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,21,22 row-size=82B cardinality=791.15K
-|  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 23(OPEN)
-|  |
-|  |--23:SCAN HDFS [tpcds_parquet.web_site]
-|  |     HDFS partitions=1/1 files=1 size=11.91KB
-|  |     stored statistics:
-|  |       table: rows=30 size=11.91KB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=30
-|  |     mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
-|  |     tuple-ids=22 row-size=32B cardinality=30
-|  |     in pipelines: 23(GETNEXT)
-|  |
-|  24:HASH JOIN [INNER JOIN]
 |  |  hash predicates: date_sk = d_date_sk
 |  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  runtime filters: RF008[bloom] <- d_date_sk, RF009[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,21 row-size=50B cardinality=791.15K
+|  |  tuple-ids=19,22,21 row-size=82B cardinality=395.57K
 |  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 22(OPEN)
 |  |
 |  |--22:SCAN HDFS [tpcds_parquet.date_dim]
@@ -211,6 +193,24 @@ PLAN-ROOT SINK
 |  |     tuple-ids=21 row-size=26B cardinality=7.30K
 |  |     in pipelines: 22(GETNEXT)
 |  |
+|  24:HASH JOIN [INNER JOIN]
+|  |  hash predicates: wsr_web_site_sk = web_site_sk
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF010[bloom] <- web_site_sk, RF011[min_max] <- web_site_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=19,22 row-size=56B cardinality=395.57K
+|  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 23(OPEN)
+|  |
+|  |--23:SCAN HDFS [tpcds_parquet.web_site]
+|  |     HDFS partitions=1/1 files=1 size=11.91KB
+|  |     stored statistics:
+|  |       table: rows=30 size=11.91KB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=30
+|  |     mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
+|  |     tuple-ids=22 row-size=32B cardinality=30
+|  |     in pipelines: 23(GETNEXT)
+|  |
 |  17:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=19 row-size=24B cardinality=791.15K
@@ -226,7 +226,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  |--19:SCAN HDFS [tpcds_parquet.web_returns]
 |  |  |     HDFS partitions=1/1 files=1 size=5.66MB
-|  |  |     runtime filters: RF011[min_max] -> tpcds_parquet.web_returns.wr_returned_date_sk, RF010[bloom] -> tpcds_parquet.web_returns.wr_returned_date_sk
+|  |  |     runtime filters: RF009[min_max] -> tpcds_parquet.web_returns.wr_returned_date_sk, RF008[bloom] -> tpcds_parquet.web_returns.wr_returned_date_sk
 |  |  |     stored statistics:
 |  |  |       table: rows=71.76K size=5.66MB
 |  |  |       columns: all
@@ -237,7 +237,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  20:SCAN HDFS [tpcds_parquet.web_sales]
 |  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF014[min_max] -> ws_item_sk, RF015[min_max] -> ws_order_number, RF009[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF012[bloom] -> ws_item_sk, RF013[bloom] -> ws_order_number, RF008[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk
+|  |     runtime filters: RF014[min_max] -> ws_item_sk, RF015[min_max] -> ws_order_number, RF011[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF012[bloom] -> ws_item_sk, RF013[bloom] -> ws_order_number, RF010[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk
 |  |     stored statistics:
 |  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
@@ -248,7 +248,7 @@ PLAN-ROOT SINK
 |  |
 |  18:SCAN HDFS [tpcds_parquet.web_sales]
 |     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF009[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF011[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF008[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk
+|     runtime filters: RF011[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF009[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk, RF008[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk
 |     stored statistics:
 |       table: rows=719.38K size=45.09MB
 |       columns: all
@@ -265,29 +265,11 @@ PLAN-ROOT SINK
 |  |  in pipelines: 16(GETNEXT), 10(OPEN), 11(OPEN)
 |  |
 |  15:HASH JOIN [INNER JOIN]
-|  |  hash predicates: page_sk = cp_catalog_page_sk
-|  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF004[bloom] <- cp_catalog_page_sk, RF005[min_max] <- cp_catalog_page_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=10,12,13 row-size=82B cardinality=1.59M
-|  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 13(OPEN)
-|  |
-|  |--13:SCAN HDFS [tpcds_parquet.catalog_page]
-|  |     HDFS partitions=1/1 files=1 size=739.17KB
-|  |     stored statistics:
-|  |       table: rows=11.72K size=739.17KB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=11.72K
-|  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=13 row-size=32B cardinality=11.72K
-|  |     in pipelines: 13(GETNEXT)
-|  |
-|  14:HASH JOIN [INNER JOIN]
 |  |  hash predicates: date_sk = d_date_sk
 |  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
+|  |  runtime filters: RF004[bloom] <- d_date_sk, RF005[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=10,12 row-size=50B cardinality=1.59M
+|  |  tuple-ids=10,13,12 row-size=82B cardinality=1.41M
 |  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 12(OPEN)
 |  |
 |  |--12:SCAN HDFS [tpcds_parquet.date_dim]
@@ -303,6 +285,24 @@ PLAN-ROOT SINK
 |  |     tuple-ids=12 row-size=26B cardinality=7.30K
 |  |     in pipelines: 12(GETNEXT)
 |  |
+|  14:HASH JOIN [INNER JOIN]
+|  |  hash predicates: page_sk = cp_catalog_page_sk
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF006[bloom] <- cp_catalog_page_sk, RF007[min_max] <- cp_catalog_page_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=10,13 row-size=56B cardinality=1.41M
+|  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 13(OPEN)
+|  |
+|  |--13:SCAN HDFS [tpcds_parquet.catalog_page]
+|  |     HDFS partitions=1/1 files=1 size=739.17KB
+|  |     stored statistics:
+|  |       table: rows=11.72K size=739.17KB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=11.72K
+|  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
+|  |     tuple-ids=13 row-size=32B cardinality=11.72K
+|  |     in pipelines: 13(GETNEXT)
+|  |
 |  09:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=10 row-size=24B cardinality=1.59M
@@ -310,7 +310,7 @@ PLAN-ROOT SINK
 |  |
 |  |--11:SCAN HDFS [tpcds_parquet.catalog_returns]
 |  |     HDFS partitions=1/1 files=1 size=10.62MB
-|  |     runtime filters: RF005[min_max] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF007[min_max] -> tpcds_parquet.catalog_returns.cr_returned_date_sk, RF004[bloom] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF006[bloom] -> tpcds_parquet.catalog_returns.cr_returned_date_sk
+|  |     runtime filters: RF007[min_max] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF005[min_max] -> tpcds_parquet.catalog_returns.cr_returned_date_sk, RF006[bloom] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF004[bloom] -> tpcds_parquet.catalog_returns.cr_returned_date_sk
 |  |     stored statistics:
 |  |       table: rows=144.07K size=10.62MB
 |  |       columns: all
@@ -321,7 +321,7 @@ PLAN-ROOT SINK
 |  |
 |  10:SCAN HDFS [tpcds_parquet.catalog_sales]
 |     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF005[min_max] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF007[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF004[bloom] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF006[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk
+|     runtime filters: RF007[min_max] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF005[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF006[bloom] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF004[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk
 |     stored statistics:
 |       table: rows=1.44M size=96.62MB
 |       columns: all
@@ -393,10 +393,10 @@ PLAN-ROOT SINK
 |     in pipelines: 03(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.store_sales]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
    runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_store_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_store_sk, RF002[bloom] -> tpcds_parquet.store_sales.ss_sold_date_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -496,44 +496,19 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |  in pipelines: 18(GETNEXT), 20(GETNEXT)
 |  |
 |  25:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: wsr_web_site_sk = web_site_sk
-|  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF008[bloom] <- web_site_sk, RF009[min_max] <- web_site_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,21,22 row-size=82B cardinality=791.15K
-|  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 23(OPEN)
-|  |
-|  |--41:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=22 row-size=32B cardinality=30
-|  |  |  in pipelines: 23(GETNEXT)
-|  |  |
-|  |  F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2
-|  |  23:SCAN HDFS [tpcds_parquet.web_site, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=11.91KB
-|  |     stored statistics:
-|  |       table: rows=30 size=11.91KB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=30
-|  |     mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
-|  |     tuple-ids=22 row-size=32B cardinality=30
-|  |     in pipelines: 23(GETNEXT)
-|  |
-|  24:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: date_sk = d_date_sk
 |  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  runtime filters: RF008[bloom] <- d_date_sk, RF009[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,21 row-size=50B cardinality=791.15K
+|  |  tuple-ids=19,22,21 row-size=82B cardinality=395.57K
 |  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 22(OPEN)
 |  |
-|  |--40:EXCHANGE [BROADCAST]
+|  |--41:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=215.48KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=21 row-size=26B cardinality=7.30K
 |  |  |  in pipelines: 22(GETNEXT)
 |  |  |
-|  |  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2
 |  |  22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -548,6 +523,31 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |     tuple-ids=21 row-size=26B cardinality=7.30K
 |  |     in pipelines: 22(GETNEXT)
 |  |
+|  24:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: wsr_web_site_sk = web_site_sk
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF010[bloom] <- web_site_sk, RF011[min_max] <- web_site_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=19,22 row-size=56B cardinality=395.57K
+|  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 23(OPEN)
+|  |
+|  |--40:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=22 row-size=32B cardinality=30
+|  |  |  in pipelines: 23(GETNEXT)
+|  |  |
+|  |  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2
+|  |  23:SCAN HDFS [tpcds_parquet.web_site, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=11.91KB
+|  |     stored statistics:
+|  |       table: rows=30 size=11.91KB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=30
+|  |     mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
+|  |     tuple-ids=22 row-size=32B cardinality=30
+|  |     in pipelines: 23(GETNEXT)
+|  |
 |  17:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=19 row-size=24B cardinality=791.15K
@@ -570,7 +570,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |  |  Per-Host Resources: mem-estimate=81.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 |  |  |  19:SCAN HDFS [tpcds_parquet.web_returns, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=5.66MB
-|  |  |     runtime filters: RF011[min_max] -> tpcds_parquet.web_returns.wr_returned_date_sk, RF010[bloom] -> tpcds_parquet.web_returns.wr_returned_date_sk
+|  |  |     runtime filters: RF009[min_max] -> tpcds_parquet.web_returns.wr_returned_date_sk, RF008[bloom] -> tpcds_parquet.web_returns.wr_returned_date_sk
 |  |  |     stored statistics:
 |  |  |       table: rows=71.76K size=5.66MB
 |  |  |       columns: all
@@ -588,7 +588,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |  Per-Host Resources: mem-estimate=99.00MB mem-reservation=7.00MB thread-reservation=2 runtime-filters-memory=3.00MB
 |  |  20:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
 |  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF014[min_max] -> ws_item_sk, RF015[min_max] -> ws_order_number, RF009[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF012[bloom] -> ws_item_sk, RF013[bloom] -> ws_order_number, RF008[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk
+|  |     runtime filters: RF014[min_max] -> ws_item_sk, RF015[min_max] -> ws_order_number, RF011[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF012[bloom] -> ws_item_sk, RF013[bloom] -> ws_order_number, RF010[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk
 |  |     stored statistics:
 |  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
@@ -599,7 +599,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |
 |  18:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
 |     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF009[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF011[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF008[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk
+|     runtime filters: RF011[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF009[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk, RF008[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk
 |     stored statistics:
 |       table: rows=719.38K size=45.09MB
 |       columns: all
@@ -630,44 +630,19 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |  in pipelines: 10(GETNEXT), 11(GETNEXT)
 |  |
 |  15:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: page_sk = cp_catalog_page_sk
-|  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF004[bloom] <- cp_catalog_page_sk, RF005[min_max] <- cp_catalog_page_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=10,12,13 row-size=82B cardinality=1.59M
-|  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 13(OPEN)
-|  |
-|  |--35:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=402.19KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=13 row-size=32B cardinality=11.72K
-|  |  |  in pipelines: 13(GETNEXT)
-|  |  |
-|  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2
-|  |  13:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=739.17KB
-|  |     stored statistics:
-|  |       table: rows=11.72K size=739.17KB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=11.72K
-|  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=13 row-size=32B cardinality=11.72K
-|  |     in pipelines: 13(GETNEXT)
-|  |
-|  14:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: date_sk = d_date_sk
 |  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
+|  |  runtime filters: RF004[bloom] <- d_date_sk, RF005[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=10,12 row-size=50B cardinality=1.59M
+|  |  tuple-ids=10,13,12 row-size=82B cardinality=1.41M
 |  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 12(OPEN)
 |  |
-|  |--34:EXCHANGE [BROADCAST]
+|  |--35:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=215.48KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=12 row-size=26B cardinality=7.30K
 |  |  |  in pipelines: 12(GETNEXT)
 |  |  |
-|  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=2
 |  |  12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -682,6 +657,31 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |     tuple-ids=12 row-size=26B cardinality=7.30K
 |  |     in pipelines: 12(GETNEXT)
 |  |
+|  14:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: page_sk = cp_catalog_page_sk
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF006[bloom] <- cp_catalog_page_sk, RF007[min_max] <- cp_catalog_page_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=10,13 row-size=56B cardinality=1.41M
+|  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 13(OPEN)
+|  |
+|  |--34:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=402.19KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=13 row-size=32B cardinality=11.72K
+|  |  |  in pipelines: 13(GETNEXT)
+|  |  |
+|  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=2
+|  |  13:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=739.17KB
+|  |     stored statistics:
+|  |       table: rows=11.72K size=739.17KB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=11.72K
+|  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
+|  |     tuple-ids=13 row-size=32B cardinality=11.72K
+|  |     in pipelines: 13(GETNEXT)
+|  |
 |  09:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=10 row-size=24B cardinality=1.59M
@@ -689,7 +689,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |
 |  |--11:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=10.62MB
-|  |     runtime filters: RF005[min_max] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF007[min_max] -> tpcds_parquet.catalog_returns.cr_returned_date_sk, RF004[bloom] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF006[bloom] -> tpcds_parquet.catalog_returns.cr_returned_date_sk
+|  |     runtime filters: RF007[min_max] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF005[min_max] -> tpcds_parquet.catalog_returns.cr_returned_date_sk, RF006[bloom] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF004[bloom] -> tpcds_parquet.catalog_returns.cr_returned_date_sk
 |  |     stored statistics:
 |  |       table: rows=144.07K size=10.62MB
 |  |       columns: all
@@ -700,7 +700,7 @@ Per-Host Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reservati
 |  |
 |  10:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
 |     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF005[min_max] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF007[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF004[bloom] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF006[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk
+|     runtime filters: RF007[min_max] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF005[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF006[bloom] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF004[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk
 |     stored statistics:
 |       table: rows=1.44M size=96.62MB
 |       columns: all
@@ -800,10 +800,10 @@ Per-Host Resources: mem-estimate=112.10MB mem-reservation=11.88MB thread-reserva
 |     in pipelines: 03(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
    runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_store_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_store_sk, RF002[bloom] -> tpcds_parquet.store_sales.ss_sold_date_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -905,72 +905,72 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser
 |  |
 |  25:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=04
-|  |  hash predicates: wsr_web_site_sk = web_site_sk
+|  |  hash predicates: date_sk = d_date_sk
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,21,22 row-size=82B cardinality=791.15K
-|  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 23(OPEN)
+|  |  tuple-ids=19,22,21 row-size=82B cardinality=395.57K
+|  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 22(OPEN)
 |  |
 |  |--F27:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=5.09MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=03
-|  |  |  build expressions: web_site_sk
-|  |  |  runtime filters: RF008[bloom] <- web_site_sk, RF009[min_max] <- web_site_sk
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF008[bloom] <- d_date_sk, RF009[min_max] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  41:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=22 row-size=32B cardinality=30
-|  |  |  in pipelines: 23(GETNEXT)
+|  |  |  mem-estimate=215.48KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=21 row-size=26B cardinality=7.30K
+|  |  |  in pipelines: 22(GETNEXT)
 |  |  |
 |  |  F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1
-|  |  23:SCAN HDFS [tpcds_parquet.web_site, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=11.91KB
+|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1
+|  |  22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
 |  |     stored statistics:
-|  |       table: rows=30 size=11.91KB
+|  |       table: rows=73.05K size=2.15MB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=30
-|  |     mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=0
-|  |     tuple-ids=22 row-size=32B cardinality=30
-|  |     in pipelines: 23(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
+|  |     parquet dictionary predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=21 row-size=26B cardinality=7.30K
+|  |     in pipelines: 22(GETNEXT)
 |  |
 |  24:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=05
-|  |  hash predicates: date_sk = d_date_sk
+|  |  hash predicates: wsr_web_site_sk = web_site_sk
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,21 row-size=50B cardinality=791.15K
-|  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 22(OPEN)
+|  |  tuple-ids=19,22 row-size=56B cardinality=395.57K
+|  |  in pipelines: 18(GETNEXT), 20(GETNEXT), 23(OPEN)
 |  |
 |  |--F28:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  |  |  Per-Instance Resources: mem-estimate=5.09MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=03
-|  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  |  build expressions: web_site_sk
+|  |  |  runtime filters: RF010[bloom] <- web_site_sk, RF011[min_max] <- web_site_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  40:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=215.48KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=21 row-size=26B cardinality=7.30K
-|  |  |  in pipelines: 22(GETNEXT)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=22 row-size=32B cardinality=30
+|  |  |  in pipelines: 23(GETNEXT)
 |  |  |
 |  |  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1
-|  |  22:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
+|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1
+|  |  23:SCAN HDFS [tpcds_parquet.web_site, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=11.91KB
 |  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
+|  |       table: rows=30 size=11.91KB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
-|  |     parquet dictionary predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
-|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=21 row-size=26B cardinality=7.30K
-|  |     in pipelines: 22(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=30
+|  |     mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=0
+|  |     tuple-ids=22 row-size=32B cardinality=30
+|  |     in pipelines: 23(GETNEXT)
 |  |
 |  17:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -1003,7 +1003,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser
 |  |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1
 |  |  |  19:SCAN HDFS [tpcds_parquet.web_returns, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=5.66MB
-|  |  |     runtime filters: RF011[min_max] -> tpcds_parquet.web_returns.wr_returned_date_sk, RF010[bloom] -> tpcds_parquet.web_returns.wr_returned_date_sk
+|  |  |     runtime filters: RF009[min_max] -> tpcds_parquet.web_returns.wr_returned_date_sk, RF008[bloom] -> tpcds_parquet.web_returns.wr_returned_date_sk
 |  |  |     stored statistics:
 |  |  |       table: rows=71.76K size=5.66MB
 |  |  |       columns: all
@@ -1022,7 +1022,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser
 |  |  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=1
 |  |  20:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
 |  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF014[min_max] -> ws_item_sk, RF015[min_max] -> ws_order_number, RF009[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF012[bloom] -> ws_item_sk, RF013[bloom] -> ws_order_number, RF008[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk
+|  |     runtime filters: RF014[min_max] -> ws_item_sk, RF015[min_max] -> ws_order_number, RF011[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF012[bloom] -> ws_item_sk, RF013[bloom] -> ws_order_number, RF010[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk
 |  |     stored statistics:
 |  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
@@ -1033,7 +1033,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser
 |  |
 |  18:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
 |     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF009[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF011[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF008[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk
+|     runtime filters: RF011[min_max] -> tpcds_parquet.web_sales.ws_web_site_sk, RF009[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_web_site_sk, RF008[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk
 |     stored statistics:
 |       table: rows=719.38K size=45.09MB
 |       columns: all
@@ -1066,72 +1066,72 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser
 |  |
 |  15:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=02
-|  |  hash predicates: page_sk = cp_catalog_page_sk
+|  |  hash predicates: date_sk = d_date_sk
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=10,12,13 row-size=82B cardinality=1.59M
-|  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 13(OPEN)
+|  |  tuple-ids=10,13,12 row-size=82B cardinality=1.41M
+|  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 12(OPEN)
 |  |
 |  |--F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=5.27MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=5.09MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
-|  |  |  build expressions: cp_catalog_page_sk
-|  |  |  runtime filters: RF004[bloom] <- cp_catalog_page_sk, RF005[min_max] <- cp_catalog_page_sk
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF004[bloom] <- d_date_sk, RF005[min_max] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  35:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=402.19KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=13 row-size=32B cardinality=11.72K
-|  |  |  in pipelines: 13(GETNEXT)
+|  |  |  mem-estimate=215.48KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=12 row-size=26B cardinality=7.30K
+|  |  |  in pipelines: 12(GETNEXT)
 |  |  |
 |  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  13:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=739.17KB
+|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1
+|  |  12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
 |  |     stored statistics:
-|  |       table: rows=11.72K size=739.17KB
+|  |       table: rows=73.05K size=2.15MB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=11.72K
-|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=13 row-size=32B cardinality=11.72K
-|  |     in pipelines: 13(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
+|  |     parquet dictionary predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=12 row-size=26B cardinality=7.30K
+|  |     in pipelines: 12(GETNEXT)
 |  |
 |  14:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=03
-|  |  hash predicates: date_sk = d_date_sk
+|  |  hash predicates: page_sk = cp_catalog_page_sk
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=10,12 row-size=50B cardinality=1.59M
-|  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 12(OPEN)
+|  |  tuple-ids=10,13 row-size=56B cardinality=1.41M
+|  |  in pipelines: 10(GETNEXT), 11(GETNEXT), 13(OPEN)
 |  |
 |  |--F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=5.09MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=5.27MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=02
-|  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
+|  |  |  build expressions: cp_catalog_page_sk
+|  |  |  runtime filters: RF006[bloom] <- cp_catalog_page_sk, RF007[min_max] <- cp_catalog_page_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  34:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=215.48KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=12 row-size=26B cardinality=7.30K
-|  |  |  in pipelines: 12(GETNEXT)
+|  |  |  mem-estimate=402.19KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=13 row-size=32B cardinality=11.72K
+|  |  |  in pipelines: 13(GETNEXT)
 |  |  |
 |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1
-|  |  12:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
+|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=1
+|  |  13:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=739.17KB
 |  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
+|  |       table: rows=11.72K size=739.17KB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
-|  |     parquet dictionary predicates: CAST(d_date AS DATE) <= DATE '1998-08-18', CAST(d_date AS DATE) >= DATE '1998-08-04'
-|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=12 row-size=26B cardinality=7.30K
-|  |     in pipelines: 12(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=11.72K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=13 row-size=32B cardinality=11.72K
+|  |     in pipelines: 13(GETNEXT)
 |  |
 |  09:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -1140,7 +1140,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser
 |  |
 |  |--11:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=10.62MB
-|  |     runtime filters: RF005[min_max] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF007[min_max] -> tpcds_parquet.catalog_returns.cr_returned_date_sk, RF004[bloom] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF006[bloom] -> tpcds_parquet.catalog_returns.cr_returned_date_sk
+|  |     runtime filters: RF007[min_max] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF005[min_max] -> tpcds_parquet.catalog_returns.cr_returned_date_sk, RF006[bloom] -> tpcds_parquet.catalog_returns.cr_catalog_page_sk, RF004[bloom] -> tpcds_parquet.catalog_returns.cr_returned_date_sk
 |  |     stored statistics:
 |  |       table: rows=144.07K size=10.62MB
 |  |       columns: all
@@ -1151,7 +1151,7 @@ Per-Instance Resources: mem-estimate=40.62MB mem-reservation=7.94MB thread-reser
 |  |
 |  10:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
 |     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF005[min_max] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF007[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF004[bloom] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF006[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk
+|     runtime filters: RF007[min_max] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF005[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF006[bloom] -> tpcds_parquet.catalog_sales.cs_catalog_page_sk, RF004[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk
 |     stored statistics:
 |       table: rows=1.44M size=96.62MB
 |       columns: all
@@ -1268,10 +1268,10 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=6.00MB thread-reser
 |     in pipelines: 03(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
    runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_store_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_store_sk, RF002[bloom] -> tpcds_parquet.store_sales.ss_sold_date_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
index 0bf5521..574f6f5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
@@ -100,7 +100,7 @@ PLAN-ROOT SINK
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN (year_total * CAST(1.0000 AS DECIMAL(5,4))) / year_total ELSE CAST(0 AS DECIMAL(38,6)) END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN (year_total * CAST(1.0000 AS DECIMAL(5,4))) / year_total ELSE CAST(0 AS DECIMAL(38,6)) END
 |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
 |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=18,8,28,38 row-size=225B cardinality=589.03K
+|  tuple-ids=18,28,8,38 row-size=225B cardinality=43.59K
 |  in pipelines: 13(GETNEXT), 27(OPEN)
 |
 |--21:UNION
@@ -170,8 +170,81 @@ PLAN-ROOT SINK
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
+|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=18,28,8 row-size=181B cardinality=43.59K
+|  in pipelines: 13(GETNEXT), 06(OPEN)
+|
+|--00:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=44B cardinality=58.90K
+|  |  in pipelines: 06(GETNEXT)
+|  |
+|  06:AGGREGATE [FINALIZE]
+|  |  output: sum(ss_ext_list_price - ss_ext_discount_amt)
+|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  having: sum(ss_ext_list_price - ss_ext_discount_amt) > CAST(0 AS DECIMAL(3,0))
+|  |  mem-estimate=103.99MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=3 row-size=169B cardinality=58.90K
+|  |  in pipelines: 06(GETNEXT), 01(OPEN)
+|  |
+|  05:HASH JOIN [INNER JOIN]
+|  |  hash predicates: c_customer_sk = ss_customer_sk
+|  |  fk/pk conjuncts: c_customer_sk = ss_customer_sk
+|  |  runtime filters: RF014[bloom] <- ss_customer_sk, RF015[min_max] <- ss_customer_sk
+|  |  mem-estimate=40.89MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=0,1,2 row-size=177B cardinality=589.03K
+|  |  in pipelines: 01(GETNEXT), 02(OPEN)
+|  |
+|  |--04:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  runtime filters: RF016[bloom] <- d_date_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=1,2 row-size=24B cardinality=589.03K
+|  |  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |  |
+|  |  |--03:SCAN HDFS [tpcds_parquet.date_dim]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |     tuple-ids=2 row-size=8B cardinality=373
+|  |  |     in pipelines: 03(GETNEXT)
+|  |  |
+|  |  02:SCAN HDFS [tpcds_parquet.store_sales]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     runtime filters: RF016[bloom] -> ss_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.95MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
+|  |     tuple-ids=1 row-size=16B cardinality=2.88M
+|  |     in pipelines: 02(GETNEXT)
+|  |
+|  01:SCAN HDFS [tpcds_parquet.customer]
+|     HDFS partitions=1/1 files=1 size=5.49MB
+|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF015[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF014[bloom] -> c_customer_sk
+|     stored statistics:
+|       table: rows=100.00K size=5.49MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|     tuple-ids=0 row-size=153B cardinality=100.00K
+|     in pipelines: 01(GETNEXT)
+|
+28:HASH JOIN [INNER JOIN]
+|  hash predicates: customer_id = customer_id
+|  fk/pk conjuncts: assumed fk/pk
+|  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=18,8,28 row-size=181B cardinality=589.03K
+|  tuple-ids=18,28 row-size=137B cardinality=43.59K
 |  in pipelines: 13(GETNEXT), 20(OPEN)
 |
 |--14:UNION
@@ -190,7 +263,7 @@ PLAN-ROOT SINK
 |  19:HASH JOIN [INNER JOIN]
 |  |  hash predicates: ws_sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
-|  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
+|  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=24,25,26 row-size=177B cardinality=148.00K
 |  |  in pipelines: 15(GETNEXT), 17(OPEN)
@@ -211,14 +284,14 @@ PLAN-ROOT SINK
 |  18:HASH JOIN [INNER JOIN]
 |  |  hash predicates: c_customer_sk = ws_bill_customer_sk
 |  |  fk/pk conjuncts: none
-|  |  runtime filters: RF016[bloom] <- ws_bill_customer_sk, RF017[min_max] <- ws_bill_customer_sk
+|  |  runtime filters: RF012[bloom] <- ws_bill_customer_sk, RF013[min_max] <- ws_bill_customer_sk
 |  |  mem-estimate=58.42MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  |  tuple-ids=24,25 row-size=169B cardinality=719.38K
 |  |  in pipelines: 15(GETNEXT), 16(OPEN)
 |  |
 |  |--16:SCAN HDFS [tpcds_parquet.web_sales]
 |  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF015[min_max] -> ws_sold_date_sk, RF014[bloom] -> ws_sold_date_sk
+|  |     runtime filters: RF011[min_max] -> ws_sold_date_sk, RF010[bloom] -> ws_sold_date_sk
 |  |     stored statistics:
 |  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
@@ -229,7 +302,7 @@ PLAN-ROOT SINK
 |  |
 |  15:SCAN HDFS [tpcds_parquet.customer]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF017[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF016[bloom] -> c_customer_sk
+|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF013[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF012[bloom] -> c_customer_sk
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
@@ -238,79 +311,6 @@ PLAN-ROOT SINK
 |     tuple-ids=24 row-size=153B cardinality=100.00K
 |     in pipelines: 15(GETNEXT)
 |
-28:HASH JOIN [INNER JOIN]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=18,8 row-size=137B cardinality=589.03K
-|  in pipelines: 13(GETNEXT), 06(OPEN)
-|
-|--00:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=44B cardinality=58.90K
-|  |  in pipelines: 06(GETNEXT)
-|  |
-|  06:AGGREGATE [FINALIZE]
-|  |  output: sum(ss_ext_list_price - ss_ext_discount_amt)
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum(ss_ext_list_price - ss_ext_discount_amt) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=103.99MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=58.90K
-|  |  in pipelines: 06(GETNEXT), 01(OPEN)
-|  |
-|  05:HASH JOIN [INNER JOIN]
-|  |  hash predicates: c_customer_sk = ss_customer_sk
-|  |  fk/pk conjuncts: c_customer_sk = ss_customer_sk
-|  |  runtime filters: RF010[bloom] <- ss_customer_sk, RF011[min_max] <- ss_customer_sk
-|  |  mem-estimate=40.89MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=0,1,2 row-size=177B cardinality=589.03K
-|  |  in pipelines: 01(GETNEXT), 02(OPEN)
-|  |
-|  |--04:HASH JOIN [INNER JOIN]
-|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  |  |  runtime filters: RF012[bloom] <- d_date_sk
-|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=1,2 row-size=24B cardinality=589.03K
-|  |  |  in pipelines: 02(GETNEXT), 03(OPEN)
-|  |  |
-|  |  |--03:SCAN HDFS [tpcds_parquet.date_dim]
-|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |  |     stored statistics:
-|  |  |       table: rows=73.05K size=2.15MB
-|  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=2 row-size=8B cardinality=373
-|  |  |     in pipelines: 03(GETNEXT)
-|  |  |
-|  |  02:SCAN HDFS [tpcds_parquet.store_sales]
-|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|  |     runtime filters: RF012[bloom] -> ss_sold_date_sk
-|  |     stored statistics:
-|  |       table: rows=2.88M size=200.96MB
-|  |       partitions: 1824/1824 rows=2.88M
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
-|  |     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=16B cardinality=2.88M
-|  |     in pipelines: 02(GETNEXT)
-|  |
-|  01:SCAN HDFS [tpcds_parquet.customer]
-|     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF010[bloom] -> c_customer_sk
-|     stored statistics:
-|       table: rows=100.00K size=5.49MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=0 row-size=153B cardinality=100.00K
-|     in pipelines: 01(GETNEXT)
-|
 07:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  tuple-ids=18 row-size=93B cardinality=589.03K
@@ -353,10 +353,10 @@ PLAN-ROOT SINK
 |  |     in pipelines: 10(GETNEXT)
 |  |
 |  09:SCAN HDFS [tpcds_parquet.store_sales]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
 |     runtime filters: RF008[bloom] -> ss_sold_date_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
+|       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -366,7 +366,7 @@ PLAN-ROOT SINK
 |
 08:SCAN HDFS [tpcds_parquet.customer]
    HDFS partitions=1/1 files=1 size=5.49MB
-   runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF007[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF006[bloom] -> c_customer_sk
+   runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF007[min_max] -> c_customer_sk, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF006[bloom] -> c_customer_sk
    stored statistics:
      table: rows=100.00K size=5.49MB
      columns: all
@@ -375,63 +375,63 @@ PLAN-ROOT SINK
    tuple-ids=10 row-size=153B cardinality=100.00K
    in pipelines: 08(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=370.12MB Threads=33
-Per-Host Resource Estimates: Memory=1.39GB
-F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+Max Per-Host Resource Reservation: Memory=352.25MB Threads=34
+Per-Host Resource Estimates: Memory=1.37GB
+F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-55:MERGING-EXCHANGE [UNPARTITIONED]
+56:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  limit: 100
 |  mem-estimate=26.23KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=40 row-size=77B cardinality=100
 |  in pipelines: 31(GETNEXT)
 |
-F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservation=1 runtime-filters-memory=3.00MB
+F18:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=16.11MB mem-reservation=9.62MB thread-reservation=1 runtime-filters-memory=2.00MB
 31:TOP-N [LIMIT=100]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  mem-estimate=7.52KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=40 row-size=77B cardinality=100
 |  in pipelines: 31(GETNEXT), 36(OPEN)
 |
-30:HASH JOIN [INNER JOIN, BROADCAST]
+30:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN (year_total * CAST(1.0000 AS DECIMAL(5,4))) / year_total ELSE CAST(0 AS DECIMAL(38,6)) END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN (year_total * CAST(1.0000 AS DECIMAL(5,4))) / year_total ELSE CAST(0 AS DECIMAL(38,6)) END
 |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
-|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=18,8,28,38 row-size=225B cardinality=589.03K
-|  in pipelines: 36(GETNEXT), 53(OPEN)
+|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=18,28,8,38 row-size=225B cardinality=43.59K
+|  in pipelines: 36(GETNEXT), 54(OPEN)
 |
-|--54:EXCHANGE [BROADCAST]
-|  |  mem-estimate=6.30MB mem-reservation=0B thread-reservation=0
+|--55:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=3.20MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=38 row-size=44B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT)
+|  |  in pipelines: 54(GETNEXT)
 |  |
-|  F23:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
 |  21:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=38 row-size=44B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT)
+|  |  in pipelines: 54(GETNEXT)
 |  |
-|  53:AGGREGATE [FINALIZE]
+|  54:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_ext_list_price - ws_ext_discount_amt)
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=37 row-size=169B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT), 23(OPEN)
+|  |  in pipelines: 54(GETNEXT), 23(OPEN)
 |  |
-|  52:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  53:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=37 row-size=169B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F20:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  F21:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=52.63MB mem-reservation=36.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  27:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_list_price - ws_ext_discount_amt)
@@ -448,12 +448,12 @@ Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservat
 |  |  tuple-ids=35,34,36 row-size=177B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT), 24(OPEN)
 |  |
-|  |--51:EXCHANGE [BROADCAST]
+|  |--52:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=36 row-size=8B cardinality=373
 |  |  |  in pipelines: 24(GETNEXT)
 |  |  |
-|  |  F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
 |  |  24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -476,12 +476,12 @@ Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservat
 |  |  tuple-ids=35,34 row-size=169B cardinality=719.38K
 |  |  in pipelines: 23(GETNEXT), 22(OPEN)
 |  |
-|  |--50:EXCHANGE [HASH(c_customer_sk)]
+|  |--51:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=34 row-size=153B cardinality=100.00K
 |  |  |  in pipelines: 22(GETNEXT)
 |  |  |
-|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=2
 |  |  22:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
@@ -493,12 +493,12 @@ Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservat
 |  |     tuple-ids=34 row-size=153B cardinality=100.00K
 |  |     in pipelines: 22(GETNEXT)
 |  |
-|  49:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  50:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=35 row-size=16B cardinality=719.38K
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F18:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F19:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 |  23:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
 |     HDFS partitions=1/1 files=2 size=45.09MB
@@ -511,154 +511,40 @@ Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservat
 |     tuple-ids=35 row-size=16B cardinality=719.38K
 |     in pipelines: 23(GETNEXT)
 |
-29:HASH JOIN [INNER JOIN, BROADCAST]
+29:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=18,8,28 row-size=181B cardinality=589.03K
+|  tuple-ids=18,28,8 row-size=181B cardinality=43.59K
 |  in pipelines: 36(GETNEXT), 47(OPEN)
 |
-|--48:EXCHANGE [BROADCAST]
-|  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=44B cardinality=14.80K
-|  |  in pipelines: 47(GETNEXT)
-|  |
-|  F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
-|  14:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=44B cardinality=14.80K
-|  |  in pipelines: 47(GETNEXT)
-|  |
-|  47:AGGREGATE [FINALIZE]
-|  |  output: sum:merge(ws_ext_list_price - ws_ext_discount_amt)
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum(ws_ext_list_price - ws_ext_discount_amt) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=27 row-size=169B cardinality=14.80K
-|  |  in pipelines: 47(GETNEXT), 16(OPEN)
-|  |
-|  46:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=27 row-size=169B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT)
-|  |
-|  F14:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=53.63MB mem-reservation=37.94MB thread-reservation=1 runtime-filters-memory=2.00MB
-|  20:AGGREGATE [STREAMING]
-|  |  output: sum(ws_ext_list_price - ws_ext_discount_amt)
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=27 row-size=169B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT)
-|  |
-|  19:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: ws_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
-|  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=25,24,26 row-size=177B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT), 17(OPEN)
-|  |
-|  |--45:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=26 row-size=8B cardinality=373
-|  |  |  in pipelines: 17(GETNEXT)
-|  |  |
-|  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
-|  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=26 row-size=8B cardinality=373
-|  |     in pipelines: 17(GETNEXT)
-|  |
-|  18:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash predicates: ws_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: none
-|  |  runtime filters: RF016[bloom] <- c_customer_sk, RF017[min_max] <- c_customer_sk
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=25,24 row-size=169B cardinality=719.38K
-|  |  in pipelines: 16(GETNEXT), 15(OPEN)
-|  |
-|  |--44:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=24 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 15(GETNEXT)
-|  |  |
-|  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB
-|  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
-|  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=24 row-size=153B cardinality=100.00K
-|  |     in pipelines: 15(GETNEXT)
-|  |
-|  43:EXCHANGE [HASH(ws_bill_customer_sk)]
-|  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=25 row-size=16B cardinality=719.38K
-|  |  in pipelines: 16(GETNEXT)
-|  |
-|  F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB
-|  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
-|     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF015[min_max] -> ws_sold_date_sk, RF017[min_max] -> ws_bill_customer_sk, RF014[bloom] -> ws_sold_date_sk, RF016[bloom] -> ws_bill_customer_sk
-|     stored statistics:
-|       table: rows=719.38K size=45.09MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=644.77K
-|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=25 row-size=16B cardinality=719.38K
-|     in pipelines: 16(GETNEXT)
-|
-28:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=18,8 row-size=137B cardinality=589.03K
-|  in pipelines: 36(GETNEXT), 41(OPEN)
-|
-|--42:EXCHANGE [BROADCAST]
-|  |  mem-estimate=2.61MB mem-reservation=0B thread-reservation=0
+|--49:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=987.66KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=8 row-size=44B cardinality=58.90K
-|  |  in pipelines: 41(GETNEXT)
+|  |  in pipelines: 47(GETNEXT)
 |  |
-|  F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=45.17MB mem-reservation=34.00MB thread-reservation=1
 |  00:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=8 row-size=44B cardinality=58.90K
-|  |  in pipelines: 41(GETNEXT)
+|  |  in pipelines: 47(GETNEXT)
 |  |
-|  41:AGGREGATE [FINALIZE]
+|  47:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ss_ext_list_price - ss_ext_discount_amt)
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  having: sum(ss_ext_list_price - ss_ext_discount_amt) > CAST(0 AS DECIMAL(3,0))
 |  |  mem-estimate=34.66MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  |  tuple-ids=3 row-size=169B cardinality=58.90K
-|  |  in pipelines: 41(GETNEXT), 02(OPEN)
+|  |  in pipelines: 47(GETNEXT), 02(OPEN)
 |  |
-|  40:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  46:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=3 row-size=169B cardinality=589.03K
 |  |  in pipelines: 02(GETNEXT)
 |  |
-|  F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
+|  F15:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=58.24MB mem-reservation=43.50MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  06:AGGREGATE [STREAMING]
 |  |  output: sum(ss_ext_list_price - ss_ext_discount_amt)
@@ -670,21 +556,21 @@ Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservat
 |  05:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash predicates: ss_customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  |  runtime filters: RF010[bloom] <- c_customer_sk, RF011[min_max] <- c_customer_sk
+|  |  runtime filters: RF014[bloom] <- c_customer_sk, RF015[min_max] <- c_customer_sk
 |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |  tuple-ids=1,2,0 row-size=177B cardinality=589.03K
 |  |  in pipelines: 02(GETNEXT), 01(OPEN)
 |  |
-|  |--39:EXCHANGE [HASH(c_customer_sk)]
+|  |--45:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=0 row-size=153B cardinality=100.00K
 |  |  |  in pipelines: 01(GETNEXT)
 |  |  |
-|  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Resources: mem-estimate=129.00MB mem-reservation=9.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 |  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
 |  |       table: rows=100.00K size=5.49MB
 |  |       columns: all
@@ -693,27 +579,27 @@ Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservat
 |  |     tuple-ids=0 row-size=153B cardinality=100.00K
 |  |     in pipelines: 01(GETNEXT)
 |  |
-|  38:EXCHANGE [HASH(ss_customer_sk)]
+|  44:EXCHANGE [HASH(ss_customer_sk)]
 |  |  mem-estimate=4.59MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1,2 row-size=24B cardinality=589.03K
 |  |  in pipelines: 02(GETNEXT)
 |  |
-|  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=51.95MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB
 |  04:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF012[bloom] <- d_date_sk
+|  |  runtime filters: RF016[bloom] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=1,2 row-size=24B cardinality=589.03K
 |  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |
-|  |--37:EXCHANGE [BROADCAST]
+|  |--43:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=2 row-size=8B cardinality=373
 |  |  |  in pipelines: 03(GETNEXT)
 |  |  |
-|  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
 |  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -729,10 +615,10 @@ Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservat
 |  |     in pipelines: 03(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|     runtime filters: RF011[min_max] -> ss_customer_sk, RF012[bloom] -> ss_sold_date_sk, RF010[bloom] -> ss_customer_sk
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk, RF014[bloom] -> ss_customer_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
+|       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -740,14 +626,135 @@ Per-Host Resources: mem-estimate=86.18MB mem-reservation=65.38MB thread-reservat
 |     tuple-ids=1 row-size=16B cardinality=2.88M
 |     in pipelines: 02(GETNEXT)
 |
-07:UNION
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=18 row-size=93B cardinality=589.03K
+48:EXCHANGE [HASH(customer_id)]
+|  mem-estimate=2.32MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=18,28 row-size=137B cardinality=43.59K
 |  in pipelines: 36(GETNEXT)
 |
-36:AGGREGATE [FINALIZE]
-|  output: sum:merge(ss_ext_list_price - ss_ext_discount_amt)
-|  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=49.76MB mem-reservation=37.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+28:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: customer_id = customer_id
+|  fk/pk conjuncts: assumed fk/pk
+|  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
+|  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=18,28 row-size=137B cardinality=43.59K
+|  in pipelines: 36(GETNEXT), 41(OPEN)
+|
+|--42:EXCHANGE [BROADCAST]
+|  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=28 row-size=44B cardinality=14.80K
+|  |  in pipelines: 41(GETNEXT)
+|  |
+|  F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Host Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
+|  14:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=28 row-size=44B cardinality=14.80K
+|  |  in pipelines: 41(GETNEXT)
+|  |
+|  41:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ws_ext_list_price - ws_ext_discount_amt)
+|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  having: sum(ws_ext_list_price - ws_ext_discount_amt) > CAST(0 AS DECIMAL(3,0))
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=27 row-size=169B cardinality=14.80K
+|  |  in pipelines: 41(GETNEXT), 16(OPEN)
+|  |
+|  40:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=27 row-size=169B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT)
+|  |
+|  F08:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  Per-Host Resources: mem-estimate=53.63MB mem-reservation=37.94MB thread-reservation=1 runtime-filters-memory=2.00MB
+|  20:AGGREGATE [STREAMING]
+|  |  output: sum(ws_ext_list_price - ws_ext_discount_amt)
+|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=27 row-size=169B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT)
+|  |
+|  19:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=25,24,26 row-size=177B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |
+|  |--39:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=26 row-size=8B cardinality=373
+|  |  |  in pipelines: 17(GETNEXT)
+|  |  |
+|  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
+|  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |     tuple-ids=26 row-size=8B cardinality=373
+|  |     in pipelines: 17(GETNEXT)
+|  |
+|  18:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash predicates: ws_bill_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: none
+|  |  runtime filters: RF012[bloom] <- c_customer_sk, RF013[min_max] <- c_customer_sk
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=25,24 row-size=169B cardinality=719.38K
+|  |  in pipelines: 16(GETNEXT), 15(OPEN)
+|  |
+|  |--38:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=24 row-size=153B cardinality=100.00K
+|  |  |  in pipelines: 15(GETNEXT)
+|  |  |
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|  |     tuple-ids=24 row-size=153B cardinality=100.00K
+|  |     in pipelines: 15(GETNEXT)
+|  |
+|  37:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=25 row-size=16B cardinality=719.38K
+|  |  in pipelines: 16(GETNEXT)
+|  |
+|  F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Host Resources: mem-estimate=130.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|     HDFS partitions=1/1 files=2 size=45.09MB
+|     runtime filters: RF011[min_max] -> ws_sold_date_sk, RF013[min_max] -> ws_bill_customer_sk, RF010[bloom] -> ws_sold_date_sk, RF012[bloom] -> ws_bill_customer_sk
+|     stored statistics:
+|       table: rows=719.38K size=45.09MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|     tuple-ids=25 row-size=16B cardinality=719.38K
+|     in pipelines: 16(GETNEXT)
+|
+07:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=18 row-size=93B cardinality=589.03K
+|  in pipelines: 36(GETNEXT)
+|
+36:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_ext_list_price - ss_ext_discount_amt)
+|  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  mem-estimate=34.66MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=13 row-size=169B cardinality=589.03K
 |  in pipelines: 36(GETNEXT), 09(OPEN)
@@ -783,7 +790,7 @@ Per-Host Resources: mem-estimate=58.24MB mem-reservation=43.50MB thread-reservat
 |  Per-Host Resources: mem-estimate=131.00MB mem-reservation=11.00MB thread-reservation=2 runtime-filters-memory=3.00MB
 |  08:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id
+|     runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
@@ -828,10 +835,10 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=5.94MB thread-reservati
 |     in pipelines: 10(GETNEXT)
 |
 09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
    runtime filters: RF007[min_max] -> ss_customer_sk, RF008[bloom] -> ss_sold_date_sk, RF006[bloom] -> ss_customer_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -839,71 +846,71 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=5.94MB thread-reservati
    tuple-ids=11 row-size=16B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=484.25MB Threads=40
-Per-Host Resource Estimates: Memory=848MB
-F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+Max Per-Host Resource Reservation: Memory=444.88MB Threads=44
+Per-Host Resource Estimates: Memory=813MB
+F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-55:MERGING-EXCHANGE [UNPARTITIONED]
+56:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  limit: 100
 |  mem-estimate=49.95KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=40 row-size=77B cardinality=100
 |  in pipelines: 31(GETNEXT)
 |
-F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-reservation=1
+F18:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=2.75MB mem-reservation=0B thread-reservation=1
 31:TOP-N [LIMIT=100]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
 |  mem-estimate=7.52KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=40 row-size=77B cardinality=100
 |  in pipelines: 31(GETNEXT), 36(OPEN)
 |
-30:HASH JOIN [INNER JOIN, BROADCAST]
+30:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN (year_total * CAST(1.0000 AS DECIMAL(5,4))) / year_total ELSE CAST(0 AS DECIMAL(38,6)) END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN (year_total * CAST(1.0000 AS DECIMAL(5,4))) / year_total ELSE CAST(0 AS DECIMAL(38,6)) END
-|  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=18,8,28,38 row-size=225B cardinality=589.03K
-|  in pipelines: 36(GETNEXT), 53(OPEN)
+|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=18,28,8,38 row-size=225B cardinality=43.59K
+|  in pipelines: 36(GETNEXT), 54(OPEN)
 |
-|--F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=41.30MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F26:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=7.07MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: customer_id
 |  |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  |
-|  54:EXCHANGE [BROADCAST]
-|  |  mem-estimate=6.30MB mem-reservation=0B thread-reservation=0
+|  55:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=3.20MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=38 row-size=44B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT)
+|  |  in pipelines: 54(GETNEXT)
 |  |
-|  F23:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
 |  21:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=38 row-size=44B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT)
+|  |  in pipelines: 54(GETNEXT)
 |  |
-|  53:AGGREGATE [FINALIZE]
+|  54:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_ext_list_price - ws_ext_discount_amt)
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=37 row-size=169B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT), 23(OPEN)
+|  |  in pipelines: 54(GETNEXT), 23(OPEN)
 |  |
-|  52:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  53:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=37 row-size=169B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F20:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  F21:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=22.53MB mem-reservation=17.00MB thread-reservation=1
 |  27:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_list_price - ws_ext_discount_amt)
@@ -920,7 +927,7 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  tuple-ids=35,34,36 row-size=177B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT), 24(OPEN)
 |  |
-|  |--F26:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |--F27:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
@@ -928,12 +935,12 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  |  runtime filters: RF018[bloom] <- d_date_sk, RF019[min_max] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
-|  |  51:EXCHANGE [BROADCAST]
+|  |  52:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=36 row-size=8B cardinality=373
 |  |  |  in pipelines: 24(GETNEXT)
 |  |  |
-|  |  F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
 |  |  24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -956,7 +963,7 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  tuple-ids=35,34 row-size=169B cardinality=719.38K
 |  |  in pipelines: 23(GETNEXT), 22(OPEN)
 |  |
-|  |--F27:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |--F28:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=27.15MB mem-reservation=17.00MB thread-reservation=1
 |  |  JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
@@ -964,12 +971,12 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  |  runtime filters: RF021[min_max] <- c_customer_sk
 |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  |
-|  |  50:EXCHANGE [HASH(c_customer_sk)]
+|  |  51:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=34 row-size=153B cardinality=100.00K
 |  |  |  in pipelines: 22(GETNEXT)
 |  |  |
-|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
 |  |  22:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
@@ -981,12 +988,12 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |     tuple-ids=34 row-size=153B cardinality=100.00K
 |  |     in pipelines: 22(GETNEXT)
 |  |
-|  49:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  50:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=35 row-size=16B cardinality=719.38K
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F18:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F19:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
 |  23:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
@@ -1000,48 +1007,196 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |     tuple-ids=35 row-size=16B cardinality=719.38K
 |     in pipelines: 23(GETNEXT)
 |
-29:HASH JOIN [INNER JOIN, BROADCAST]
+29:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=03
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=18,8,28 row-size=181B cardinality=589.03K
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=18,28,8 row-size=181B cardinality=43.59K
 |  in pipelines: 36(GETNEXT), 47(OPEN)
 |
-|--F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=7.46MB mem-reservation=6.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F29:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: customer_id
 |  |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |
+|  49:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=44B cardinality=58.90K
+|  |  in pipelines: 47(GETNEXT)
+|  |
+|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1
+|  00:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=44B cardinality=58.90K
+|  |  in pipelines: 47(GETNEXT)
+|  |
+|  47:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ss_ext_list_price - ss_ext_discount_amt)
+|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  having: sum(ss_ext_list_price - ss_ext_discount_amt) > CAST(0 AS DECIMAL(3,0))
+|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=3 row-size=169B cardinality=58.90K
+|  |  in pipelines: 47(GETNEXT), 02(OPEN)
+|  |
+|  46:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  |  mem-estimate=11.01MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=169B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F15:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|  Per-Instance Resources: mem-estimate=21.68MB mem-reservation=17.00MB thread-reservation=1
+|  06:AGGREGATE [STREAMING]
+|  |  output: sum(ss_ext_list_price - ss_ext_discount_amt)
+|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=3 row-size=169B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=04
+|  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=1,2,0 row-size=177B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |
+|  |--F30:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|  |  |  Per-Instance Resources: mem-estimate=15.90MB mem-reservation=5.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=03
+|  |  |  build expressions: c_customer_sk
+|  |  |  runtime filters: RF014[bloom] <- c_customer_sk, RF015[min_max] <- c_customer_sk
+|  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  |  |
+|  |  45:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=0 row-size=153B cardinality=100.00K
+|  |  |  in pipelines: 01(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
+|  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
+|  |     tuple-ids=0 row-size=153B cardinality=100.00K
+|  |     in pipelines: 01(GETNEXT)
+|  |
+|  44:EXCHANGE [HASH(ss_customer_sk)]
+|  |  mem-estimate=4.68MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1,2 row-size=24B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1
+|  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=05
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=1,2 row-size=24B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |
+|  |--F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=03
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF016[bloom] <- d_date_sk
+|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
+|  |  |
+|  |  43:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=2 row-size=8B cardinality=373
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=2 row-size=8B cardinality=373
+|  |     in pipelines: 03(GETNEXT)
+|  |
+|  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk, RF014[bloom] -> ss_customer_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.95MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|     tuple-ids=1 row-size=16B cardinality=2.88M
+|     in pipelines: 02(GETNEXT)
+|
+48:EXCHANGE [HASH(customer_id)]
+|  mem-estimate=2.75MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=18,28 row-size=137B cardinality=43.59K
+|  in pipelines: 36(GETNEXT)
+|
+F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1
+28:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=06
+|  hash predicates: customer_id = customer_id
+|  fk/pk conjuncts: assumed fk/pk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=18,28 row-size=137B cardinality=43.59K
+|  in pipelines: 36(GETNEXT), 41(OPEN)
+|
+|--F32:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=7.46MB mem-reservation=6.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  JOIN BUILD
+|  |  join-table-id=06 plan-id=07 cohort-id=01
+|  |  build expressions: customer_id
+|  |  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
 |  |  mem-estimate=5.75MB mem-reservation=5.75MB spill-buffer=128.00KB thread-reservation=0
 |  |
-|  48:EXCHANGE [BROADCAST]
+|  42:EXCHANGE [BROADCAST]
 |  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=28 row-size=44B cardinality=14.80K
-|  |  in pipelines: 47(GETNEXT)
+|  |  in pipelines: 41(GETNEXT)
 |  |
-|  F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=27.34MB mem-reservation=17.00MB thread-reservation=1
 |  14:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=28 row-size=44B cardinality=14.80K
-|  |  in pipelines: 47(GETNEXT)
+|  |  in pipelines: 41(GETNEXT)
 |  |
-|  47:AGGREGATE [FINALIZE]
+|  41:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_ext_list_price - ws_ext_discount_amt)
 |  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
 |  |  having: sum(ws_ext_list_price - ws_ext_discount_amt) > CAST(0 AS DECIMAL(3,0))
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=27 row-size=169B cardinality=14.80K
-|  |  in pipelines: 47(GETNEXT), 16(OPEN)
+|  |  in pipelines: 41(GETNEXT), 16(OPEN)
 |  |
-|  46:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
+|  40:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
 |  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=27 row-size=169B cardinality=148.00K
 |  |  in pipelines: 16(GETNEXT)
 |  |
-|  F14:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  F08:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=22.53MB mem-reservation=17.00MB thread-reservation=1
 |  20:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_list_price - ws_ext_discount_amt)
@@ -1051,27 +1206,27 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |  in pipelines: 16(GETNEXT)
 |  |
 |  19:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash-table-id=04
+|  |  hash-table-id=07
 |  |  hash predicates: ws_sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=25,24,26 row-size=177B cardinality=148.00K
 |  |  in pipelines: 16(GETNEXT), 17(OPEN)
 |  |
-|  |--F29:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |--F33:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
-|  |  |  join-table-id=04 plan-id=05 cohort-id=03
+|  |  |  join-table-id=07 plan-id=08 cohort-id=04
 |  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
+|  |  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
-|  |  45:EXCHANGE [BROADCAST]
+|  |  39:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=26 row-size=8B cardinality=373
 |  |  |  in pipelines: 17(GETNEXT)
 |  |  |
-|  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
 |  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -1087,32 +1242,32 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |     in pipelines: 17(GETNEXT)
 |  |
 |  18:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash-table-id=05
+|  |  hash-table-id=08
 |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: none
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=25,24 row-size=169B cardinality=719.38K
 |  |  in pipelines: 16(GETNEXT), 15(OPEN)
 |  |
-|  |--F30:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |--F34:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=28.15MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
-|  |  |  join-table-id=05 plan-id=06 cohort-id=03
+|  |  |  join-table-id=08 plan-id=09 cohort-id=04
 |  |  |  build expressions: c_customer_sk
-|  |  |  runtime filters: RF016[bloom] <- c_customer_sk, RF017[min_max] <- c_customer_sk
+|  |  |  runtime filters: RF012[bloom] <- c_customer_sk, RF013[min_max] <- c_customer_sk
 |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  |
-|  |  44:EXCHANGE [HASH(c_customer_sk)]
+|  |  38:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=24 row-size=153B cardinality=100.00K
 |  |  |  in pipelines: 15(GETNEXT)
 |  |  |
-|  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
 |  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
 |  |       table: rows=100.00K size=5.49MB
 |  |       columns: all
@@ -1121,17 +1276,17 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |  |     tuple-ids=24 row-size=153B cardinality=100.00K
 |  |     in pipelines: 15(GETNEXT)
 |  |
-|  43:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  37:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=25 row-size=16B cardinality=719.38K
 |  |  in pipelines: 16(GETNEXT)
 |  |
-|  F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
 |  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
 |     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF015[min_max] -> ws_sold_date_sk, RF017[min_max] -> ws_bill_customer_sk, RF014[bloom] -> ws_sold_date_sk, RF016[bloom] -> ws_bill_customer_sk
+|     runtime filters: RF011[min_max] -> ws_sold_date_sk, RF013[min_max] -> ws_bill_customer_sk, RF010[bloom] -> ws_sold_date_sk, RF012[bloom] -> ws_bill_customer_sk
 |     stored statistics:
 |       table: rows=719.38K size=45.09MB
 |       columns: all
@@ -1140,147 +1295,6 @@ Per-Instance Resources: mem-estimate=45.02MB mem-reservation=34.00MB thread-rese
 |     tuple-ids=25 row-size=16B cardinality=719.38K
 |     in pipelines: 16(GETNEXT)
 |
-28:HASH JOIN [INNER JOIN, BROADCAST]
-|  hash-table-id=06
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=18,8 row-size=137B cardinality=589.03K
-|  in pipelines: 36(GETNEXT), 41(OPEN)
-|
-|--F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=20.75MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  JOIN BUILD
-|  |  join-table-id=06 plan-id=07 cohort-id=01
-|  |  build expressions: customer_id
-|  |  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=512.00KB thread-reservation=0
-|  |
-|  42:EXCHANGE [BROADCAST]
-|  |  mem-estimate=2.75MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=44B cardinality=58.90K
-|  |  in pipelines: 41(GETNEXT)
-|  |
-|  F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=45.01MB mem-reservation=34.00MB thread-reservation=1
-|  00:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=44B cardinality=58.90K
-|  |  in pipelines: 41(GETNEXT)
-|  |
-|  41:AGGREGATE [FINALIZE]
-|  |  output: sum:merge(ss_ext_list_price - ss_ext_discount_amt)
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  having: sum(ss_ext_list_price - ss_ext_discount_amt) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=58.90K
-|  |  in pipelines: 41(GETNEXT), 02(OPEN)
-|  |
-|  40:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year)]
-|  |  mem-estimate=11.01MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
-|  |
-|  F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=21.68MB mem-reservation=17.00MB thread-reservation=1
-|  06:AGGREGATE [STREAMING]
-|  |  output: sum(ss_ext_list_price - ss_ext_discount_amt)
-|  |  group by: c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=169B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
-|  |
-|  05:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash-table-id=07
-|  |  hash predicates: ss_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  |  tuple-ids=1,2,0 row-size=177B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 01(OPEN)
-|  |
-|  |--F32:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-|  |  |  Per-Instance Resources: mem-estimate=15.90MB mem-reservation=5.75MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  JOIN BUILD
-|  |  |  join-table-id=07 plan-id=08 cohort-id=04
-|  |  |  build expressions: c_customer_sk
-|  |  |  runtime filters: RF010[bloom] <- c_customer_sk, RF011[min_max] <- c_customer_sk
-|  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  |  |
-|  |  39:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=10.15MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0 row-size=153B cardinality=100.00K
-|  |  |  in pipelines: 01(GETNEXT)
-|  |  |
-|  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
-|  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=0 row-size=153B cardinality=100.00K
-|  |     in pipelines: 01(GETNEXT)
-|  |
-|  38:EXCHANGE [HASH(ss_customer_sk)]
-|  |  mem-estimate=4.68MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=1,2 row-size=24B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
-|  |
-|  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
-|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=1
-|  04:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash-table-id=08
-|  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=1,2 row-size=24B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 03(OPEN)
-|  |
-|  |--F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  JOIN BUILD
-|  |  |  join-table-id=08 plan-id=09 cohort-id=04
-|  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF012[bloom] <- d_date_sk
-|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
-|  |  |
-|  |  37:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=8B cardinality=373
-|  |  |  in pipelines: 03(GETNEXT)
-|  |  |
-|  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=2 row-size=8B cardinality=373
-|  |     in pipelines: 03(GETNEXT)
-|  |
-|  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|     runtime filters: RF011[min_max] -> ss_customer_sk, RF012[bloom] -> ss_sold_date_sk, RF010[bloom] -> ss_customer_sk
-|     stored statistics:
-|       table: rows=2.88M size=200.96MB
-|       partitions: 1824/1824 rows=2.88M
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
-|     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-|     tuple-ids=1 row-size=16B cardinality=2.88M
-|     in pipelines: 02(GETNEXT)
-|
 07:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  tuple-ids=18 row-size=93B cardinality=589.03K
@@ -1315,7 +1329,7 @@ Per-Instance Resources: mem-estimate=21.68MB mem-reservation=17.00MB thread-rese
 |  tuple-ids=11,12,10 row-size=177B cardinality=589.03K
 |  in pipelines: 09(GETNEXT), 08(OPEN)
 |
-|--F34:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|--F35:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
 |  |  Per-Instance Resources: mem-estimate=15.90MB mem-reservation=5.75MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=09 plan-id=10 cohort-id=01
@@ -1333,7 +1347,7 @@ Per-Instance Resources: mem-estimate=21.68MB mem-reservation=17.00MB thread-rese
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
 |  08:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id
+|     runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
@@ -1358,7 +1372,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  tuple-ids=11,12 row-size=24B cardinality=589.03K
 |  in pipelines: 09(GETNEXT), 10(OPEN)
 |
-|--F35:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|--F36:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=10 plan-id=11 cohort-id=01
@@ -1387,10 +1401,10 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |     in pipelines: 10(GETNEXT)
 |
 09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
    runtime filters: RF007[min_max] -> ss_customer_sk, RF008[bloom] -> ss_sold_date_sk, RF006[bloom] -> ss_customer_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
index 7b53d67..a5fe877 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
@@ -57,10 +57,10 @@ ORDER BY SEGMENT,
 LIMIT 100;
 
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=97.58MB Threads=11
-Per-Host Resource Estimates: Memory=568MB
+Max Per-Host Resource Reservation: Memory=27.14MB Threads=11
+Per-Host Resource Estimates: Memory=546MB
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=567.56MB mem-reservation=97.58MB thread-reservation=11 runtime-filters-memory=8.00MB
+|  Per-Host Resources: mem-estimate=545.81MB mem-reservation=27.14MB thread-reservation=11 runtime-filters-memory=8.00MB
 PLAN-ROOT SINK
 |  output exprs: SEGMENT, count(*), CAST(SEGMENT AS BIGINT) * CAST(50 AS BIGINT)
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
@@ -75,21 +75,21 @@ PLAN-ROOT SINK
 |  output: count(*)
 |  group by: CAST((sum(ss_ext_sales_price) / CAST(50 AS DECIMAL(3,0))) AS INT)
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=22 row-size=12B cardinality=18.94K
+|  tuple-ids=22 row-size=12B cardinality=329
 |  in pipelines: 27(GETNEXT), 26(OPEN)
 |
 26:AGGREGATE [FINALIZE]
 |  output: sum(ss_ext_sales_price)
 |  group by: c_customer_sk
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=19 row-size=20B cardinality=18.94K
-|  in pipelines: 26(GETNEXT), 10(OPEN)
+|  tuple-ids=19 row-size=20B cardinality=329
+|  in pipelines: 26(GETNEXT), 13(OPEN)
 |
 25:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: CAST(d_month_seq AS BIGINT) <= d_month_seq + 3
 |  mem-estimate=8B mem-reservation=0B thread-reservation=0
-|  tuple-ids=9,7,10,11,12,14,17 row-size=131B cardinality=18.94K
-|  in pipelines: 10(GETNEXT), 19(OPEN)
+|  tuple-ids=12,10,9,7,11,14,17 row-size=131B cardinality=329
+|  in pipelines: 13(GETNEXT), 19(OPEN)
 |
 |--19:CARDINALITY CHECK
 |  |  limit: 1
@@ -120,8 +120,8 @@ PLAN-ROOT SINK
 24:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: CAST(d_month_seq AS BIGINT) >= d_month_seq + 1
 |  mem-estimate=8B mem-reservation=0B thread-reservation=0
-|  tuple-ids=9,7,10,11,12,14 row-size=123B cardinality=18.94K
-|  in pipelines: 10(GETNEXT), 16(OPEN)
+|  tuple-ids=12,10,9,7,11,14 row-size=123B cardinality=329
+|  in pipelines: 13(GETNEXT), 16(OPEN)
 |
 |--16:CARDINALITY CHECK
 |  |  limit: 1
@@ -150,280 +150,281 @@ PLAN-ROOT SINK
 |     in pipelines: 14(GETNEXT)
 |
 23:HASH JOIN [INNER JOIN]
-|  hash predicates: ss_sold_date_sk = d_date_sk
-|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF000[bloom] <- d_date_sk
-|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=9,7,10,11,12 row-size=115B cardinality=18.94K
-|  in pipelines: 10(GETNEXT), 13(OPEN)
-|
-|--13:SCAN HDFS [tpcds_parquet.date_dim]
-|     HDFS partitions=1/1 files=1 size=2.15MB
-|     stored statistics:
-|       table: rows=73.05K size=2.15MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=12 row-size=8B cardinality=73.05K
-|     in pipelines: 13(GETNEXT)
-|
-22:HASH JOIN [INNER JOIN]
-|  hash predicates: ca_state = s_state, ca_county = s_county
-|  fk/pk conjuncts: none
-|  runtime filters: RF002[bloom] <- s_state, RF003[bloom] <- s_county, RF004[min_max] <- s_state, RF005[min_max] <- s_county
+|  hash predicates: d_date_sk = ss_sold_date_sk
+|  fk/pk conjuncts: d_date_sk = ss_sold_date_sk
+|  runtime filters: RF000[bloom] <- ss_sold_date_sk, RF001[min_max] <- ss_sold_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=9,7,10,11 row-size=107B cardinality=18.94K
-|  in pipelines: 10(GETNEXT), 12(OPEN)
-|
-|--12:SCAN HDFS [tpcds_parquet.store]
-|     HDFS partitions=1/1 files=1 size=9.93KB
-|     stored statistics:
-|       table: rows=12 size=9.93KB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=12
-|     mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
-|     tuple-ids=11 row-size=43B cardinality=12
-|     in pipelines: 12(GETNEXT)
-|
-21:HASH JOIN [INNER JOIN]
-|  hash predicates: c_current_addr_sk = ca_address_sk
-|  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
-|  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
-|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=9,7,10 row-size=64B cardinality=2.88M
-|  in pipelines: 10(GETNEXT), 11(OPEN)
-|
-|--11:SCAN HDFS [tpcds_parquet.customer_address]
-|     HDFS partitions=1/1 files=1 size=1.16MB
-|     runtime filters: RF004[min_max] -> ca_state, RF005[min_max] -> ca_county, RF002[bloom] -> ca_state, RF003[bloom] -> ca_county
-|     stored statistics:
-|       table: rows=50.00K size=1.16MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=50.00K
-|     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=10 row-size=44B cardinality=50.00K
-|     in pipelines: 11(GETNEXT)
-|
-20:HASH JOIN [INNER JOIN]
-|  hash predicates: ss_customer_sk = c_customer_sk
-|  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  runtime filters: RF008[bloom] <- c_customer_sk, RF009[min_max] <- c_customer_sk
-|  mem-estimate=127.66MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=9,7 row-size=20B cardinality=2.88M
-|  in pipelines: 10(GETNEXT), 09(OPEN)
+|  tuple-ids=12,10,9,7,11 row-size=115B cardinality=329
+|  in pipelines: 13(GETNEXT), 11(OPEN)
 |
-|--09:AGGREGATE [FINALIZE]
-|  |  group by: c_customer_sk, c_current_addr_sk
-|  |  mem-estimate=49.46MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
-|  |  in pipelines: 09(GETNEXT), 01(OPEN), 02(OPEN)
-|  |
-|  08:HASH JOIN [INNER JOIN]
-|  |  hash predicates: customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF010[bloom] <- c_customer_sk, RF011[min_max] <- c_customer_sk
-|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=2,4,5,6 row-size=82B cardinality=2.16M
-|  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 05(OPEN)
-|  |
-|  |--05:SCAN HDFS [tpcds_parquet.customer]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF007[min_max] -> tpcds_parquet.customer.c_current_addr_sk, RF006[bloom] -> tpcds_parquet.customer.c_current_addr_sk
-|  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=6 row-size=8B cardinality=100.00K
-|  |     in pipelines: 05(GETNEXT)
-|  |
-|  07:HASH JOIN [INNER JOIN]
-|  |  hash predicates: sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF012[bloom] <- d_date_sk, RF013[min_max] <- d_date_sk
+|--22:HASH JOIN [INNER JOIN]
+|  |  hash predicates: ca_state = s_state, ca_county = s_county
+|  |  fk/pk conjuncts: none
+|  |  runtime filters: RF002[bloom] <- s_state, RF003[bloom] <- s_county, RF004[min_max] <- s_state, RF005[min_max] <- s_county
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,4,5 row-size=74B cardinality=2.16M
-|  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 04(OPEN)
+|  |  tuple-ids=10,9,7,11 row-size=107B cardinality=329
+|  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |
-|  |--04:SCAN HDFS [tpcds_parquet.date_dim]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: d_year = CAST(1998 AS INT), d_moy = CAST(12 AS INT)
+|  |--12:SCAN HDFS [tpcds_parquet.store]
+|  |     HDFS partitions=1/1 files=1 size=9.93KB
 |  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
+|  |       table: rows=12 size=9.93KB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: d_year = CAST(1998 AS INT), d_moy = CAST(12 AS INT)
-|  |     parquet dictionary predicates: d_year = CAST(1998 AS INT), d_moy = CAST(12 AS INT)
-|  |     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=5 row-size=12B cardinality=108
-|  |     in pipelines: 04(GETNEXT)
-|  |
-|  06:HASH JOIN [INNER JOIN]
-|  |  hash predicates: item_sk = i_item_sk
-|  |  fk/pk conjuncts: assumed fk/pk
-|  |  runtime filters: RF014[bloom] <- i_item_sk, RF015[min_max] <- i_item_sk
+|  |     extrapolated-rows=disabled max-scan-range-rows=12
+|  |     mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
+|  |     tuple-ids=11 row-size=43B cardinality=12
+|  |     in pipelines: 12(GETNEXT)
+|  |
+|  21:HASH JOIN [INNER JOIN]
+|  |  hash predicates: ca_address_sk = c_current_addr_sk
+|  |  fk/pk conjuncts: ca_address_sk = c_current_addr_sk
+|  |  runtime filters: RF006[bloom] <- c_current_addr_sk, RF007[min_max] <- c_current_addr_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,4 row-size=62B cardinality=2.16M
-|  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 03(OPEN)
-|  |
-|  |--03:SCAN HDFS [tpcds_parquet.item]
-|  |     HDFS partitions=1/1 files=1 size=1.73MB
-|  |     predicates: i_category = 'Women', i_class = 'maternity'
-|  |     stored statistics:
-|  |       table: rows=18.00K size=1.73MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
-|  |     parquet statistics predicates: i_category = 'Women', i_class = 'maternity'
-|  |     parquet dictionary predicates: i_category = 'Women', i_class = 'maternity'
-|  |     mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=4 row-size=46B cardinality=57
-|  |     in pipelines: 03(GETNEXT)
-|  |
-|  00:UNION
-|  |  pass-through-operands: all
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=16B cardinality=2.16M
-|  |  in pipelines: 01(GETNEXT), 02(GETNEXT)
-|  |
-|  |--02:SCAN HDFS [tpcds_parquet.web_sales]
-|  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF011[min_max] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF013[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF015[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF012[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF014[bloom] -> tpcds_parquet.web_sales.ws_item_sk
+|  |  tuple-ids=10,9,7 row-size=64B cardinality=3.43K
+|  |  in pipelines: 11(GETNEXT), 10(OPEN)
+|  |
+|  |--20:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  runtime filters: RF008[bloom] <- c_customer_sk, RF009[min_max] <- c_customer_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,7 row-size=20B cardinality=3.43K
+|  |  |  in pipelines: 10(GETNEXT), 09(OPEN)
+|  |  |
+|  |  |--09:AGGREGATE [FINALIZE]
+|  |  |  |  group by: c_customer_sk, c_current_addr_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=7 row-size=8B cardinality=108
+|  |  |  |  in pipelines: 09(GETNEXT), 05(OPEN)
+|  |  |  |
+|  |  |  08:HASH JOIN [INNER JOIN]
+|  |  |  |  hash predicates: c_customer_sk = customer_sk
+|  |  |  |  fk/pk conjuncts: assumed fk/pk
+|  |  |  |  runtime filters: RF010[bloom] <- customer_sk, RF011[min_max] <- customer_sk
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=6,2,4,5 row-size=82B cardinality=108
+|  |  |  |  in pipelines: 05(GETNEXT), 01(OPEN), 02(OPEN)
+|  |  |  |
+|  |  |  |--07:HASH JOIN [INNER JOIN]
+|  |  |  |  |  hash predicates: sold_date_sk = d_date_sk
+|  |  |  |  |  fk/pk conjuncts: assumed fk/pk
+|  |  |  |  |  runtime filters: RF012[bloom] <- d_date_sk, RF013[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=2,4,5 row-size=74B cardinality=108
+|  |  |  |  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 04(OPEN)
+|  |  |  |  |
+|  |  |  |  |--04:SCAN HDFS [tpcds_parquet.date_dim]
+|  |  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |  |     predicates: d_year = CAST(1998 AS INT), d_moy = CAST(12 AS INT)
+|  |  |  |  |     stored statistics:
+|  |  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |  |       columns: all
+|  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |  |     parquet statistics predicates: d_year = CAST(1998 AS INT), d_moy = CAST(12 AS INT)
+|  |  |  |  |     parquet dictionary predicates: d_year = CAST(1998 AS INT), d_moy = CAST(12 AS INT)
+|  |  |  |  |     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  |     tuple-ids=5 row-size=12B cardinality=108
+|  |  |  |  |     in pipelines: 04(GETNEXT)
+|  |  |  |  |
+|  |  |  |  06:HASH JOIN [INNER JOIN]
+|  |  |  |  |  hash predicates: item_sk = i_item_sk
+|  |  |  |  |  fk/pk conjuncts: assumed fk/pk
+|  |  |  |  |  runtime filters: RF014[bloom] <- i_item_sk, RF015[min_max] <- i_item_sk
+|  |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=2,4 row-size=62B cardinality=3.43K
+|  |  |  |  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 03(OPEN)
+|  |  |  |  |
+|  |  |  |  |--03:SCAN HDFS [tpcds_parquet.item]
+|  |  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |  |     predicates: i_category = 'Women', i_class = 'maternity'
+|  |  |  |  |     stored statistics:
+|  |  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |  |       columns: all
+|  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |  |     parquet statistics predicates: i_category = 'Women', i_class = 'maternity'
+|  |  |  |  |     parquet dictionary predicates: i_category = 'Women', i_class = 'maternity'
+|  |  |  |  |     mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  |     tuple-ids=4 row-size=46B cardinality=57
+|  |  |  |  |     in pipelines: 03(GETNEXT)
+|  |  |  |  |
+|  |  |  |  00:UNION
+|  |  |  |  |  pass-through-operands: all
+|  |  |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=2 row-size=16B cardinality=2.16M
+|  |  |  |  |  in pipelines: 01(GETNEXT), 02(GETNEXT)
+|  |  |  |  |
+|  |  |  |  |--02:SCAN HDFS [tpcds_parquet.web_sales]
+|  |  |  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |  |  |     runtime filters: RF015[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF013[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF014[bloom] -> tpcds_parquet.web_sales.ws_item_sk, RF012[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk
+|  |  |  |  |     stored statistics:
+|  |  |  |  |       table: rows=719.38K size=45.09MB
+|  |  |  |  |       columns: all
+|  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |  |  |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
+|  |  |  |  |     tuple-ids=1 row-size=16B cardinality=719.38K
+|  |  |  |  |     in pipelines: 02(GETNEXT)
+|  |  |  |  |
+|  |  |  |  01:SCAN HDFS [tpcds_parquet.catalog_sales]
+|  |  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |  |     runtime filters: RF015[min_max] -> tpcds_parquet.catalog_sales.cs_item_sk, RF013[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF014[bloom] -> tpcds_parquet.catalog_sales.cs_item_sk, RF012[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=1.44M size=96.62MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |  |     mem-estimate=144.00MB mem-reservation=4.00MB thread-reservation=1
+|  |  |  |     tuple-ids=0 row-size=16B cardinality=1.44M
+|  |  |  |     in pipelines: 01(GETNEXT)
+|  |  |  |
+|  |  |  05:SCAN HDFS [tpcds_parquet.customer]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF011[min_max] -> c_customer_sk, RF010[bloom] -> c_customer_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
+|  |  |     tuple-ids=6 row-size=8B cardinality=100.00K
+|  |  |     in pipelines: 05(GETNEXT)
+|  |  |
+|  |  10:SCAN HDFS [tpcds_parquet.store_sales]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     runtime filters: RF009[min_max] -> ss_customer_sk, RF008[bloom] -> ss_customer_sk
 |  |     stored statistics:
-|  |       table: rows=719.38K size=45.09MB
+|  |       table: rows=2.88M size=200.95MB
+|  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
-|  |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=16B cardinality=719.38K
-|  |     in pipelines: 02(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
+|  |     tuple-ids=9 row-size=12B cardinality=2.88M
+|  |     in pipelines: 10(GETNEXT)
 |  |
-|  01:SCAN HDFS [tpcds_parquet.catalog_sales]
-|     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF011[min_max] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk, RF013[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF015[min_max] -> tpcds_parquet.catalog_sales.cs_item_sk, RF010[bloom] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk, RF012[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF014[bloom] -> tpcds_parquet.catalog_sales.cs_item_sk
+|  11:SCAN HDFS [tpcds_parquet.customer_address]
+|     HDFS partitions=1/1 files=1 size=1.16MB
+|     runtime filters: RF007[min_max] -> ca_address_sk, RF004[min_max] -> ca_state, RF005[min_max] -> ca_county, RF006[bloom] -> ca_address_sk, RF002[bloom] -> ca_state, RF003[bloom] -> ca_county
 |     stored statistics:
-|       table: rows=1.44M size=96.62MB
+|       table: rows=50.00K size=1.16MB
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=650.14K
-|     mem-estimate=144.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=0 row-size=16B cardinality=1.44M
-|     in pipelines: 01(GETNEXT)
+|     extrapolated-rows=disabled max-scan-range-rows=50.00K
+|     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=10 row-size=44B cardinality=50.00K
+|     in pipelines: 11(GETNEXT)
 |
-10:SCAN HDFS [tpcds_parquet.store_sales]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
-   runtime filters: RF009[min_max] -> ss_customer_sk, RF000[bloom] -> ss_sold_date_sk, RF008[bloom] -> ss_customer_sk
+13:SCAN HDFS [tpcds_parquet.date_dim]
+   HDFS partitions=1/1 files=1 size=2.15MB
+   runtime filters: RF001[min_max] -> d_date_sk, RF000[bloom] -> d_date_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
-     partitions: 1824/1824 rows=2.88M
+     table: rows=73.05K size=2.15MB
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
-   mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=9 row-size=12B cardinality=2.88M
-   in pipelines: 10(GETNEXT)
+   extrapolated-rows=disabled max-scan-range-rows=73.05K
+   mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+   tuple-ids=12 row-size=8B cardinality=73.05K
+   in pipelines: 13(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=150.77MB Threads=30
-Per-Host Resource Estimates: Memory=753MB
-F21:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+Max Per-Host Resource Reservation: Memory=60.39MB Threads=31
+Per-Host Resource Estimates: Memory=648MB
+F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: SEGMENT, count(*), CAST(SEGMENT AS BIGINT) * CAST(50 AS BIGINT)
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-52:MERGING-EXCHANGE [UNPARTITIONED]
+53:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: SEGMENT ASC, count(*) ASC
 |  limit: 100
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=23 row-size=12B cardinality=100
 |  in pipelines: 28(GETNEXT)
 |
-F20:PLAN FRAGMENT [HASH(SEGMENT)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=10.12MB mem-reservation=1.94MB thread-reservation=1
+F21:PLAN FRAGMENT [HASH(SEGMENT)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
 28:TOP-N [LIMIT=100]
 |  order by: SEGMENT ASC, count(*) ASC
 |  mem-estimate=1.17KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=23 row-size=12B cardinality=100
-|  in pipelines: 28(GETNEXT), 51(OPEN)
+|  in pipelines: 28(GETNEXT), 52(OPEN)
 |
-51:AGGREGATE [FINALIZE]
+52:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |  group by: SEGMENT
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=22 row-size=12B cardinality=18.94K
-|  in pipelines: 51(GETNEXT), 49(OPEN)
+|  tuple-ids=22 row-size=12B cardinality=329
+|  in pipelines: 52(GETNEXT), 50(OPEN)
 |
-50:EXCHANGE [HASH(SEGMENT)]
-|  mem-estimate=121.98KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=22 row-size=12B cardinality=18.94K
-|  in pipelines: 49(GETNEXT)
+51:EXCHANGE [HASH(SEGMENT)]
+|  mem-estimate=16.71KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=22 row-size=12B cardinality=329
+|  in pipelines: 50(GETNEXT)
 |
-F19:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=20.19MB mem-reservation=3.94MB thread-reservation=1
+F20:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=20.02MB mem-reservation=3.94MB thread-reservation=1
 27:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: CAST((sum(ss_ext_sales_price) / CAST(50 AS DECIMAL(3,0))) AS INT)
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=22 row-size=12B cardinality=18.94K
-|  in pipelines: 49(GETNEXT)
+|  tuple-ids=22 row-size=12B cardinality=329
+|  in pipelines: 50(GETNEXT)
 |
-49:AGGREGATE [FINALIZE]
+50:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: c_customer_sk
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=19 row-size=20B cardinality=18.94K
-|  in pipelines: 49(GETNEXT), 10(OPEN)
+|  tuple-ids=19 row-size=20B cardinality=329
+|  in pipelines: 50(GETNEXT), 10(OPEN)
 |
-48:EXCHANGE [HASH(c_customer_sk)]
-|  mem-estimate=195.31KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=19 row-size=20B cardinality=18.94K
+49:EXCHANGE [HASH(c_customer_sk)]
+|  mem-estimate=25.27KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=19 row-size=20B cardinality=329
 |  in pipelines: 10(GETNEXT)
 |
-F12:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=14.54MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+F13:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=13.66MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 26:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price)
 |  group by: c_customer_sk
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=19 row-size=20B cardinality=18.94K
+|  tuple-ids=19 row-size=20B cardinality=329
 |  in pipelines: 10(GETNEXT)
 |
 25:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  predicates: CAST(d_month_seq AS BIGINT) <= d_month_seq + 3
 |  mem-estimate=8B mem-reservation=0B thread-reservation=0
-|  tuple-ids=9,7,10,11,12,14,17 row-size=131B cardinality=18.94K
+|  tuple-ids=9,7,10,11,12,14,17 row-size=131B cardinality=329
 |  in pipelines: 10(GETNEXT), 19(OPEN)
 |
-|--47:EXCHANGE [BROADCAST]
+|--48:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=1
 |  |  in pipelines: 19(GETNEXT)
 |  |
-|  F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 |  19:CARDINALITY CHECK
 |  |  limit: 1
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=1
-|  |  in pipelines: 19(GETNEXT), 45(OPEN)
+|  |  in pipelines: 19(GETNEXT), 46(OPEN)
 |  |
-|  46:EXCHANGE [UNPARTITIONED]
+|  47:EXCHANGE [UNPARTITIONED]
 |  |  limit: 2
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=2
-|  |  in pipelines: 45(GETNEXT)
+|  |  in pipelines: 46(GETNEXT)
 |  |
-|  F17:PLAN FRAGMENT [HASH(d_month_seq + 3)] hosts=1 instances=1
+|  F18:PLAN FRAGMENT [HASH(d_month_seq + 3)] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
-|  45:AGGREGATE [FINALIZE]
+|  46:AGGREGATE [FINALIZE]
 |  |  group by: d_month_seq + 3
 |  |  limit: 2
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=2
-|  |  in pipelines: 45(GETNEXT), 17(OPEN)
+|  |  in pipelines: 46(GETNEXT), 17(OPEN)
 |  |
-|  44:EXCHANGE [HASH(d_month_seq + 3)]
+|  45:EXCHANGE [HASH(d_month_seq + 3)]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=108
 |  |  in pipelines: 17(GETNEXT)
 |  |
-|  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=58.00MB mem-reservation=2.25MB thread-reservation=2
 |  18:AGGREGATE [STREAMING]
 |  |  group by: CAST(d_month_seq AS BIGINT) + CAST(3 AS BIGINT)
@@ -447,43 +448,43 @@ Per-Host Resources: mem-estimate=14.54MB mem-reservation=4.94MB thread-reservati
 24:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  predicates: CAST(d_month_seq AS BIGINT) >= d_month_seq + 1
 |  mem-estimate=8B mem-reservation=0B thread-reservation=0
-|  tuple-ids=9,7,10,11,12,14 row-size=123B cardinality=18.94K
+|  tuple-ids=9,7,10,11,12,14 row-size=123B cardinality=329
 |  in pipelines: 10(GETNEXT), 16(OPEN)
 |
-|--43:EXCHANGE [BROADCAST]
+|--44:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=1
 |  |  in pipelines: 16(GETNEXT)
 |  |
-|  F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 |  16:CARDINALITY CHECK
 |  |  limit: 1
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=1
-|  |  in pipelines: 16(GETNEXT), 41(OPEN)
+|  |  in pipelines: 16(GETNEXT), 42(OPEN)
 |  |
-|  42:EXCHANGE [UNPARTITIONED]
+|  43:EXCHANGE [UNPARTITIONED]
 |  |  limit: 2
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=2
-|  |  in pipelines: 41(GETNEXT)
+|  |  in pipelines: 42(GETNEXT)
 |  |
-|  F14:PLAN FRAGMENT [HASH(d_month_seq + 1)] hosts=1 instances=1
+|  F15:PLAN FRAGMENT [HASH(d_month_seq + 1)] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
-|  41:AGGREGATE [FINALIZE]
+|  42:AGGREGATE [FINALIZE]
 |  |  group by: d_month_seq + 1
 |  |  limit: 2
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=2
-|  |  in pipelines: 41(GETNEXT), 14(OPEN)
+|  |  in pipelines: 42(GETNEXT), 14(OPEN)
 |  |
-|  40:EXCHANGE [HASH(d_month_seq + 1)]
+|  41:EXCHANGE [HASH(d_month_seq + 1)]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=108
 |  |  in pipelines: 14(GETNEXT)
 |  |
-|  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=58.00MB mem-reservation=2.25MB thread-reservation=2
 |  15:AGGREGATE [STREAMING]
 |  |  group by: CAST(d_month_seq AS BIGINT) + CAST(1 AS BIGINT)
@@ -509,15 +510,15 @@ Per-Host Resources: mem-estimate=14.54MB mem-reservation=4.94MB thread-reservati
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF000[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=9,7,10,11,12 row-size=115B cardinality=18.94K
+|  tuple-ids=9,7,10,11,12 row-size=115B cardinality=329
 |  in pipelines: 10(GETNEXT), 13(OPEN)
 |
-|--39:EXCHANGE [HASH(d_date_sk)]
+|--40:EXCHANGE [HASH(d_date_sk)]
 |  |  mem-estimate=582.70KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=12 row-size=8B cardinality=73.05K
 |  |  in pipelines: 13(GETNEXT)
 |  |
-|  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
 |  13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -529,27 +530,27 @@ Per-Host Resources: mem-estimate=14.54MB mem-reservation=4.94MB thread-reservati
 |     tuple-ids=12 row-size=8B cardinality=73.05K
 |     in pipelines: 13(GETNEXT)
 |
-38:EXCHANGE [HASH(ss_sold_date_sk)]
-|  mem-estimate=1.00MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=9,7,10,11 row-size=107B cardinality=18.94K
+39:EXCHANGE [HASH(ss_sold_date_sk)]
+|  mem-estimate=129.97KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=9,7,10,11 row-size=107B cardinality=329
 |  in pipelines: 10(GETNEXT)
 |
-F08:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=74.73MB mem-reservation=48.44MB thread-reservation=1 runtime-filters-memory=4.00MB
+F10:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=10.08MB mem-reservation=7.81MB thread-reservation=1 runtime-filters-memory=3.00MB
 22:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ca_state = s_state, ca_county = s_county
 |  fk/pk conjuncts: none
 |  runtime filters: RF002[bloom] <- s_state, RF003[bloom] <- s_county, RF004[min_max] <- s_state, RF005[min_max] <- s_county
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=9,7,10,11 row-size=107B cardinality=18.94K
+|  tuple-ids=9,7,10,11 row-size=107B cardinality=329
 |  in pipelines: 10(GETNEXT), 12(OPEN)
 |
-|--37:EXCHANGE [BROADCAST]
+|--38:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=11 row-size=43B cardinality=12
 |  |  in pipelines: 12(GETNEXT)
 |  |
-|  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=2
 |  12:SCAN HDFS [tpcds_parquet.store, RANDOM]
 |     HDFS partitions=1/1 files=1 size=9.93KB
@@ -561,15 +562,15 @@ Per-Host Resources: mem-estimate=74.73MB mem-reservation=48.44MB thread-reservat
 |     tuple-ids=11 row-size=43B cardinality=12
 |     in pipelines: 12(GETNEXT)
 |
-21:HASH JOIN [INNER JOIN, BROADCAST]
+21:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: c_current_addr_sk = ca_address_sk
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
-|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=9,7,10 row-size=64B cardinality=2.88M
+|  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=9,7,10 row-size=64B cardinality=3.43K
 |  in pipelines: 10(GETNEXT), 11(OPEN)
 |
-|--36:EXCHANGE [BROADCAST]
+|--37:EXCHANGE [HASH(ca_address_sk)]
 |  |  mem-estimate=2.14MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=10 row-size=44B cardinality=50.00K
 |  |  in pipelines: 11(GETNEXT)
@@ -587,49 +588,56 @@ Per-Host Resources: mem-estimate=74.73MB mem-reservation=48.44MB thread-reservat
 |     tuple-ids=10 row-size=44B cardinality=50.00K
 |     in pipelines: 11(GETNEXT)
 |
-20:HASH JOIN [INNER JOIN, PARTITIONED]
+36:EXCHANGE [HASH(c_current_addr_sk)]
+|  mem-estimate=106.35KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=9,7 row-size=20B cardinality=3.43K
+|  in pipelines: 10(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB
+20:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_customer_sk = c_customer_sk
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF008[bloom] <- c_customer_sk, RF009[min_max] <- c_customer_sk
-|  mem-estimate=42.55MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=9,7 row-size=20B cardinality=2.88M
-|  in pipelines: 10(GETNEXT), 33(OPEN)
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=9,7 row-size=20B cardinality=3.43K
+|  in pipelines: 10(GETNEXT), 34(OPEN)
 |
-|--35:EXCHANGE [HASH(c_customer_sk)]
-|  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
-|  |  in pipelines: 33(GETNEXT)
-|  |
-|  F07:PLAN FRAGMENT [HASH(c_customer_sk,c_current_addr_sk)] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=39.53MB mem-reservation=34.00MB thread-reservation=1
-|  33:AGGREGATE [FINALIZE]
+|--35:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=8B cardinality=108
+|  |  in pipelines: 34(GETNEXT)
+|  |
+|  F08:PLAN FRAGMENT [HASH(c_customer_sk,c_current_addr_sk)] hosts=3 instances=3
+|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
+|  34:AGGREGATE [FINALIZE]
 |  |  group by: c_customer_sk, c_current_addr_sk
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
-|  |  in pipelines: 33(GETNEXT), 01(OPEN), 02(OPEN)
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=8B cardinality=108
+|  |  in pipelines: 34(GETNEXT), 01(OPEN), 02(OPEN)
 |  |
-|  32:EXCHANGE [HASH(c_customer_sk,c_current_addr_sk)]
-|  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
+|  33:EXCHANGE [HASH(c_customer_sk,c_current_addr_sk)]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=8B cardinality=108
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT)
 |  |
-|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=177.18MB mem-reservation=36.38MB thread-reservation=2 runtime-filters-memory=3.00MB
+|  F07:PLAN FRAGMENT [HASH(customer_sk)] hosts=3 instances=3
+|  Per-Host Resources: mem-estimate=14.68MB mem-reservation=5.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  09:AGGREGATE [STREAMING]
 |  |  group by: c_customer_sk, c_current_addr_sk
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=8B cardinality=108
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT)
 |  |
-|  08:HASH JOIN [INNER JOIN, BROADCAST]
+|  08:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash predicates: customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  runtime filters: RF010[bloom] <- c_customer_sk, RF011[min_max] <- c_customer_sk
-|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=2,4,5,6 row-size=82B cardinality=2.16M
+|  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=2,4,5,6 row-size=82B cardinality=108
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 05(OPEN)
 |  |
-|  |--31:EXCHANGE [BROADCAST]
+|  |--32:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=793.25KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=6 row-size=8B cardinality=100.00K
 |  |  |  in pipelines: 05(GETNEXT)
@@ -647,12 +655,19 @@ Per-Host Resources: mem-estimate=74.73MB mem-reservation=48.44MB thread-reservat
 |  |     tuple-ids=6 row-size=8B cardinality=100.00K
 |  |     in pipelines: 05(GETNEXT)
 |  |
+|  31:EXCHANGE [HASH(customer_sk)]
+|  |  mem-estimate=29.70KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2,4,5 row-size=74B cardinality=108
+|  |  in pipelines: 01(GETNEXT), 02(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Resources: mem-estimate=150.91MB mem-reservation=10.88MB thread-reservation=2 runtime-filters-memory=3.00MB
 |  07:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  runtime filters: RF012[bloom] <- d_date_sk, RF013[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,4,5 row-size=74B cardinality=2.16M
+|  |  tuple-ids=2,4,5 row-size=74B cardinality=108
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 04(OPEN)
 |  |
 |  |--30:EXCHANGE [BROADCAST]
@@ -680,7 +695,7 @@ Per-Host Resources: mem-estimate=74.73MB mem-reservation=48.44MB thread-reservat
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  runtime filters: RF014[bloom] <- i_item_sk, RF015[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,4 row-size=62B cardinality=2.16M
+|  |  tuple-ids=2,4 row-size=62B cardinality=3.43K
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 03(OPEN)
 |  |
 |  |--29:EXCHANGE [BROADCAST]
@@ -711,7 +726,7 @@ Per-Host Resources: mem-estimate=74.73MB mem-reservation=48.44MB thread-reservat
 |  |
 |  |--02:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
 |  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF011[min_max] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF013[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF015[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF012[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF014[bloom] -> tpcds_parquet.web_sales.ws_item_sk
+|  |     runtime filters: RF015[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF013[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF011[min_max] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF014[bloom] -> tpcds_parquet.web_sales.ws_item_sk, RF012[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_bill_customer_sk
 |  |     stored statistics:
 |  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
@@ -722,7 +737,7 @@ Per-Host Resources: mem-estimate=74.73MB mem-reservation=48.44MB thread-reservat
 |  |
 |  01:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
 |     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF011[min_max] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk, RF013[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF015[min_max] -> tpcds_parquet.catalog_sales.cs_item_sk, RF010[bloom] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk, RF012[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF014[bloom] -> tpcds_parquet.catalog_sales.cs_item_sk
+|     runtime filters: RF015[min_max] -> tpcds_parquet.catalog_sales.cs_item_sk, RF013[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF011[min_max] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk, RF014[bloom] -> tpcds_parquet.catalog_sales.cs_item_sk, RF012[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF010[bloom] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk
 |     stored statistics:
 |       table: rows=1.44M size=96.62MB
 |       columns: all
@@ -731,18 +746,11 @@ Per-Host Resources: mem-estimate=74.73MB mem-reservation=48.44MB thread-reservat
 |     tuple-ids=0 row-size=16B cardinality=1.44M
 |     in pipelines: 01(GETNEXT)
 |
-34:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=10.05MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=9 row-size=12B cardinality=2.88M
-|  in pipelines: 10(GETNEXT)
-|
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=34.00MB mem-reservation=3.00MB thread-reservation=2 runtime-filters-memory=2.00MB
 10:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
-   runtime filters: RF009[min_max] -> ss_customer_sk, RF000[bloom] -> ss_sold_date_sk, RF008[bloom] -> ss_customer_sk
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
+   runtime filters: RF009[min_max] -> ss_customer_sk, RF008[bloom] -> ss_customer_sk, RF000[bloom] -> ss_sold_date_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -750,118 +758,118 @@ Per-Host Resources: mem-estimate=34.00MB mem-reservation=3.00MB thread-reservati
    tuple-ids=9 row-size=12B cardinality=2.88M
    in pipelines: 10(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=223.39MB Threads=36
-Per-Host Resource Estimates: Memory=563MB
-F21:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+Max Per-Host Resource Reservation: Memory=84.95MB Threads=37
+Per-Host Resource Estimates: Memory=404MB
+F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: SEGMENT, count(*), CAST(SEGMENT AS BIGINT) * CAST(50 AS BIGINT)
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-52:MERGING-EXCHANGE [UNPARTITIONED]
+53:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: SEGMENT ASC, count(*) ASC
 |  limit: 100
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=23 row-size=12B cardinality=100
 |  in pipelines: 28(GETNEXT)
 |
-F20:PLAN FRAGMENT [HASH(SEGMENT)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=10.17MB mem-reservation=1.94MB thread-reservation=1
+F21:PLAN FRAGMENT [HASH(SEGMENT)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=10.03MB mem-reservation=1.94MB thread-reservation=1
 28:TOP-N [LIMIT=100]
 |  order by: SEGMENT ASC, count(*) ASC
 |  mem-estimate=1.17KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=23 row-size=12B cardinality=100
-|  in pipelines: 28(GETNEXT), 51(OPEN)
+|  in pipelines: 28(GETNEXT), 52(OPEN)
 |
-51:AGGREGATE [FINALIZE]
+52:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |  group by: SEGMENT
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=22 row-size=12B cardinality=18.94K
-|  in pipelines: 51(GETNEXT), 49(OPEN)
+|  tuple-ids=22 row-size=12B cardinality=329
+|  in pipelines: 52(GETNEXT), 50(OPEN)
 |
-50:EXCHANGE [HASH(SEGMENT)]
-|  mem-estimate=169.98KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=22 row-size=12B cardinality=18.94K
-|  in pipelines: 49(GETNEXT)
+51:EXCHANGE [HASH(SEGMENT)]
+|  mem-estimate=32.13KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=22 row-size=12B cardinality=329
+|  in pipelines: 50(GETNEXT)
 |
-F19:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=20.26MB mem-reservation=3.94MB thread-reservation=1
+F20:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=20.05MB mem-reservation=3.94MB thread-reservation=1
 27:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: CAST((sum(ss_ext_sales_price) / CAST(50 AS DECIMAL(3,0))) AS INT)
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=22 row-size=12B cardinality=18.94K
-|  in pipelines: 49(GETNEXT)
+|  tuple-ids=22 row-size=12B cardinality=329
+|  in pipelines: 50(GETNEXT)
 |
-49:AGGREGATE [FINALIZE]
+50:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: c_customer_sk
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=19 row-size=20B cardinality=18.94K
-|  in pipelines: 49(GETNEXT), 10(OPEN)
+|  tuple-ids=19 row-size=20B cardinality=329
+|  in pipelines: 50(GETNEXT), 10(OPEN)
 |
-48:EXCHANGE [HASH(c_customer_sk)]
-|  mem-estimate=267.31KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=19 row-size=20B cardinality=18.94K
+49:EXCHANGE [HASH(c_customer_sk)]
+|  mem-estimate=48.41KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=19 row-size=20B cardinality=329
 |  in pipelines: 10(GETNEXT)
 |
-F12:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=11.36MB mem-reservation=2.00MB thread-reservation=1
+F13:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=10.24MB mem-reservation=2.00MB thread-reservation=1
 26:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price)
 |  group by: c_customer_sk
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=19 row-size=20B cardinality=18.94K
+|  tuple-ids=19 row-size=20B cardinality=329
 |  in pipelines: 10(GETNEXT)
 |
 25:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  join table id: 00
 |  predicates: CAST(d_month_seq AS BIGINT) <= d_month_seq + 3
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=9,7,10,11,12,14,17 row-size=131B cardinality=18.94K
+|  tuple-ids=9,7,10,11,12,14,17 row-size=131B cardinality=329
 |  in pipelines: 10(GETNEXT), 19(OPEN)
 |
-|--F22:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=3
+|--F23:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=16.01KB mem-reservation=0B thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  mem-estimate=8B mem-reservation=0B thread-reservation=0
 |  |
-|  47:EXCHANGE [BROADCAST]
+|  48:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=1
 |  |  in pipelines: 19(GETNEXT)
 |  |
-|  F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 |  19:CARDINALITY CHECK
 |  |  limit: 1
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=1
-|  |  in pipelines: 19(GETNEXT), 45(OPEN)
+|  |  in pipelines: 19(GETNEXT), 46(OPEN)
 |  |
-|  46:EXCHANGE [UNPARTITIONED]
+|  47:EXCHANGE [UNPARTITIONED]
 |  |  limit: 2
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=2
-|  |  in pipelines: 45(GETNEXT)
+|  |  in pipelines: 46(GETNEXT)
 |  |
-|  F17:PLAN FRAGMENT [HASH(d_month_seq + 3)] hosts=1 instances=1
+|  F18:PLAN FRAGMENT [HASH(d_month_seq + 3)] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
-|  45:AGGREGATE [FINALIZE]
+|  46:AGGREGATE [FINALIZE]
 |  |  group by: d_month_seq + 3
 |  |  limit: 2
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=2
-|  |  in pipelines: 45(GETNEXT), 17(OPEN)
+|  |  in pipelines: 46(GETNEXT), 17(OPEN)
 |  |
-|  44:EXCHANGE [HASH(d_month_seq + 3)]
+|  45:EXCHANGE [HASH(d_month_seq + 3)]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=17 row-size=8B cardinality=108
 |  |  in pipelines: 17(GETNEXT)
 |  |
-|  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1
 |  18:AGGREGATE [STREAMING]
 |  |  group by: CAST(d_month_seq AS BIGINT) + CAST(3 AS BIGINT)
@@ -886,49 +894,49 @@ Per-Instance Resources: mem-estimate=11.36MB mem-reservation=2.00MB thread-reser
 |  join table id: 01
 |  predicates: CAST(d_month_seq AS BIGINT) >= d_month_seq + 1
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=9,7,10,11,12,14 row-size=123B cardinality=18.94K
+|  tuple-ids=9,7,10,11,12,14 row-size=123B cardinality=329
 |  in pipelines: 10(GETNEXT), 16(OPEN)
 |
-|--F23:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=3
+|--F24:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=16.01KB mem-reservation=0B thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  mem-estimate=8B mem-reservation=0B thread-reservation=0
 |  |
-|  43:EXCHANGE [BROADCAST]
+|  44:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=1
 |  |  in pipelines: 16(GETNEXT)
 |  |
-|  F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 |  16:CARDINALITY CHECK
 |  |  limit: 1
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=1
-|  |  in pipelines: 16(GETNEXT), 41(OPEN)
+|  |  in pipelines: 16(GETNEXT), 42(OPEN)
 |  |
-|  42:EXCHANGE [UNPARTITIONED]
+|  43:EXCHANGE [UNPARTITIONED]
 |  |  limit: 2
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=2
-|  |  in pipelines: 41(GETNEXT)
+|  |  in pipelines: 42(GETNEXT)
 |  |
-|  F14:PLAN FRAGMENT [HASH(d_month_seq + 1)] hosts=1 instances=1
+|  F15:PLAN FRAGMENT [HASH(d_month_seq + 1)] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
-|  41:AGGREGATE [FINALIZE]
+|  42:AGGREGATE [FINALIZE]
 |  |  group by: d_month_seq + 1
 |  |  limit: 2
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=2
-|  |  in pipelines: 41(GETNEXT), 14(OPEN)
+|  |  in pipelines: 42(GETNEXT), 14(OPEN)
 |  |
-|  40:EXCHANGE [HASH(d_month_seq + 1)]
+|  41:EXCHANGE [HASH(d_month_seq + 1)]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=8B cardinality=108
 |  |  in pipelines: 14(GETNEXT)
 |  |
-|  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=26.00MB mem-reservation=2.25MB thread-reservation=1
 |  15:AGGREGATE [STREAMING]
 |  |  group by: CAST(d_month_seq AS BIGINT) + CAST(1 AS BIGINT)
@@ -954,10 +962,10 @@ Per-Instance Resources: mem-estimate=11.36MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_sold_date_sk = d_date_sk
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=9,7,10,11,12 row-size=115B cardinality=18.94K
+|  tuple-ids=9,7,10,11,12 row-size=115B cardinality=329
 |  in pipelines: 10(GETNEXT), 13(OPEN)
 |
-|--F24:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=6
+|--F25:PLAN FRAGMENT [HASH(ss_sold_date_sk)] hosts=3 instances=6
 |  |  Per-Instance Resources: mem-estimate=3.51MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
@@ -965,12 +973,12 @@ Per-Instance Resources: mem-estimate=11.36MB mem-reservation=2.00MB thread-reser
 |  |  runtime filters: RF000[bloom] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  39:EXCHANGE [HASH(d_date_sk)]
+|  40:EXCHANGE [HASH(d_date_sk)]
 |  |  mem-estimate=582.70KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=12 row-size=8B cardinality=73.05K
 |  |  in pipelines: 13(GETNEXT)
 |  |
-|  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
 |  13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -982,22 +990,22 @@ Per-Instance Resources: mem-estimate=11.36MB mem-reservation=2.00MB thread-reser
 |     tuple-ids=12 row-size=8B cardinality=73.05K
 |     in pipelines: 13(GETNEXT)
 |
-38:EXCHANGE [HASH(ss_sold_date_sk)]
-|  mem-estimate=1.36MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=9,7,10,11 row-size=107B cardinality=18.94K
+39:EXCHANGE [HASH(ss_sold_date_sk)]
+|  mem-estimate=248.49KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=9,7,10,11 row-size=107B cardinality=329
 |  in pipelines: 10(GETNEXT)
 |
-F08:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservation=1
+F10:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=190.35KB mem-reservation=0B thread-reservation=1
 22:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=03
 |  hash predicates: ca_state = s_state, ca_county = s_county
 |  fk/pk conjuncts: none
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=9,7,10,11 row-size=107B cardinality=18.94K
+|  tuple-ids=9,7,10,11 row-size=107B cardinality=329
 |  in pipelines: 10(GETNEXT), 12(OPEN)
 |
-|--F25:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
+|--F26:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=5.89MB mem-reservation=5.88MB thread-reservation=1 runtime-filters-memory=2.00MB
 |  JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
@@ -1005,12 +1013,12 @@ Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservati
 |  |  runtime filters: RF002[bloom] <- s_state, RF003[bloom] <- s_county, RF004[min_max] <- s_state, RF005[min_max] <- s_county
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  37:EXCHANGE [BROADCAST]
+|  38:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=11 row-size=43B cardinality=12
 |  |  in pipelines: 12(GETNEXT)
 |  |
-|  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1
 |  12:SCAN HDFS [tpcds_parquet.store, RANDOM]
 |     HDFS partitions=1/1 files=1 size=9.93KB
@@ -1022,23 +1030,23 @@ Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservati
 |     tuple-ids=11 row-size=43B cardinality=12
 |     in pipelines: 12(GETNEXT)
 |
-21:HASH JOIN [INNER JOIN, BROADCAST]
+21:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=04
 |  hash predicates: c_current_addr_sk = ca_address_sk
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=9,7,10 row-size=64B cardinality=2.88M
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=9,7,10 row-size=64B cardinality=3.43K
 |  in pipelines: 10(GETNEXT), 11(OPEN)
 |
-|--F26:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=20.14MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F27:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=5.08MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: ca_address_sk
 |  |  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=512.00KB thread-reservation=0
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  36:EXCHANGE [BROADCAST]
+|  37:EXCHANGE [HASH(ca_address_sk)]
 |  |  mem-estimate=2.14MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=10 row-size=44B cardinality=50.00K
 |  |  in pipelines: 11(GETNEXT)
@@ -1057,66 +1065,73 @@ Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservati
 |     tuple-ids=10 row-size=44B cardinality=50.00K
 |     in pipelines: 11(GETNEXT)
 |
-20:HASH JOIN [INNER JOIN, PARTITIONED]
+36:EXCHANGE [HASH(c_current_addr_sk)]
+|  mem-estimate=190.35KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=9,7 row-size=20B cardinality=3.43K
+|  in pipelines: 10(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1
+20:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=05
 |  hash predicates: ss_customer_sk = c_customer_sk
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=9,7 row-size=20B cardinality=2.88M
-|  in pipelines: 10(GETNEXT), 33(OPEN)
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=9,7 row-size=20B cardinality=3.43K
+|  in pipelines: 10(GETNEXT), 34(OPEN)
 |
-|--F27:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-|  |  Per-Instance Resources: mem-estimate=40.53MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  build expressions: c_customer_sk
 |  |  runtime filters: RF008[bloom] <- c_customer_sk, RF009[min_max] <- c_customer_sk
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  35:EXCHANGE [HASH(c_customer_sk)]
-|  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
-|  |  in pipelines: 33(GETNEXT)
+|  35:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=8B cardinality=108
+|  |  in pipelines: 34(GETNEXT)
 |  |
-|  F07:PLAN FRAGMENT [HASH(c_customer_sk,c_current_addr_sk)] hosts=3 instances=3
-|  Per-Instance Resources: mem-estimate=39.53MB mem-reservation=34.00MB thread-reservation=1
-|  33:AGGREGATE [FINALIZE]
+|  F08:PLAN FRAGMENT [HASH(c_customer_sk,c_current_addr_sk)] hosts=3 instances=3
+|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
+|  34:AGGREGATE [FINALIZE]
 |  |  group by: c_customer_sk, c_current_addr_sk
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
-|  |  in pipelines: 33(GETNEXT), 01(OPEN), 02(OPEN)
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=8B cardinality=108
+|  |  in pipelines: 34(GETNEXT), 01(OPEN), 02(OPEN)
 |  |
-|  32:EXCHANGE [HASH(c_customer_sk,c_current_addr_sk)]
-|  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
+|  33:EXCHANGE [HASH(c_customer_sk,c_current_addr_sk)]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=8B cardinality=108
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT)
 |  |
-|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
-|  Per-Instance Resources: mem-estimate=65.00MB mem-reservation=21.00MB thread-reservation=1
+|  F07:PLAN FRAGMENT [HASH(customer_sk)] hosts=3 instances=3
+|  Per-Instance Resources: mem-estimate=10.03MB mem-reservation=2.00MB thread-reservation=1
 |  09:AGGREGATE [STREAMING]
 |  |  group by: c_customer_sk, c_current_addr_sk
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=7 row-size=8B cardinality=2.16M
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=8B cardinality=108
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT)
 |  |
-|  08:HASH JOIN [INNER JOIN, BROADCAST]
+|  08:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=06
 |  |  hash predicates: customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: assumed fk/pk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=2,4,5,6 row-size=82B cardinality=2.16M
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=2,4,5,6 row-size=82B cardinality=108
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 05(OPEN)
 |  |
-|  |--F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=18.77MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |--F29:PLAN FRAGMENT [HASH(customer_sk)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=4.65MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=06 plan-id=07 cohort-id=02
 |  |  |  build expressions: c_customer_sk
 |  |  |  runtime filters: RF010[bloom] <- c_customer_sk, RF011[min_max] <- c_customer_sk
-|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  |  |
-|  |  31:EXCHANGE [BROADCAST]
+|  |  32:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=793.25KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=6 row-size=8B cardinality=100.00K
 |  |  |  in pipelines: 05(GETNEXT)
@@ -1135,15 +1150,23 @@ Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservati
 |  |     tuple-ids=6 row-size=8B cardinality=100.00K
 |  |     in pipelines: 05(GETNEXT)
 |  |
+|  31:EXCHANGE [HASH(customer_sk)]
+|  |  mem-estimate=29.70KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2,4,5 row-size=74B cardinality=108
+|  |  in pipelines: 01(GETNEXT), 02(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+|  Per-Instance Resources: mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
 |  07:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=07
 |  |  hash predicates: sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,4,5 row-size=74B cardinality=2.16M
+|  |  tuple-ids=2,4,5 row-size=74B cardinality=108
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 04(OPEN)
 |  |
-|  |--F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=02
@@ -1176,10 +1199,10 @@ Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservati
 |  |  hash predicates: item_sk = i_item_sk
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,4 row-size=62B cardinality=2.16M
+|  |  tuple-ids=2,4 row-size=62B cardinality=3.43K
 |  |  in pipelines: 01(GETNEXT), 02(GETNEXT), 03(OPEN)
 |  |
-|  |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |--F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=08 plan-id=09 cohort-id=02
@@ -1215,7 +1238,7 @@ Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservati
 |  |
 |  |--02:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
 |  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF011[min_max] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF013[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF015[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF012[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF014[bloom] -> tpcds_parquet.web_sales.ws_item_sk
+|  |     runtime filters: RF015[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF013[min_max] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF011[min_max] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF014[bloom] -> tpcds_parquet.web_sales.ws_item_sk, RF012[bloom] -> tpcds_parquet.web_sales.ws_sold_date_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_bill_customer_sk
 |  |     stored statistics:
 |  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
@@ -1226,7 +1249,7 @@ Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservati
 |  |
 |  01:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
 |     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF011[min_max] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk, RF013[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF015[min_max] -> tpcds_parquet.catalog_sales.cs_item_sk, RF010[bloom] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk, RF012[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF014[bloom] -> tpcds_parquet.catalog_sales.cs_item_sk
+|     runtime filters: RF015[min_max] -> tpcds_parquet.catalog_sales.cs_item_sk, RF013[min_max] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF011[min_max] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk, RF014[bloom] -> tpcds_parquet.catalog_sales.cs_item_sk, RF012[bloom] -> tpcds_parquet.catalog_sales.cs_sold_date_sk, RF010[bloom] -> tpcds_parquet.catalog_sales.cs_bill_customer_sk
 |     stored statistics:
 |       table: rows=1.44M size=96.62MB
 |       columns: all
@@ -1235,19 +1258,11 @@ Per-Instance Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservati
 |     tuple-ids=0 row-size=16B cardinality=1.44M
 |     in pipelines: 01(GETNEXT)
 |
-34:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=10.09MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=9 row-size=12B cardinality=2.88M
-|  in pipelines: 10(GETNEXT)
-|
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
-Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1
 10:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
-   runtime filters: RF009[min_max] -> ss_customer_sk, RF000[bloom] -> ss_sold_date_sk, RF008[bloom] -> ss_customer_sk
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
+   runtime filters: RF009[min_max] -> ss_customer_sk, RF008[bloom] -> ss_customer_sk, RF000[bloom] -> ss_sold_date_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
index ef8299e..f7f7371 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
@@ -46,116 +46,135 @@ ORDER BY ext_price DESC,
          i_brand_id ;
 
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=56.12MB Threads=5
-Per-Host Resource Estimates: Memory=382MB
+Max Per-Host Resource Reservation: Memory=21.00MB Threads=5
+Per-Host Resource Estimates: Memory=377MB
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=381.62MB mem-reservation=56.12MB thread-reservation=5 runtime-filters-memory=5.00MB
+|  Per-Host Resources: mem-estimate=376.88MB mem-reservation=21.00MB thread-reservation=5 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, t_hour, t_minute, sum(ext_price)
-|  mem-estimate=16.01MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:SORT
 |  order by: sum(ext_price) DESC, i_brand_id ASC
-|  mem-estimate=16.01MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=11 row-size=56B cardinality=298.71K
+|  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=11 row-size=56B cardinality=1.01K
 |  in pipelines: 15(GETNEXT), 14(OPEN)
 |
 14:AGGREGATE [FINALIZE]
 |  output: sum(ext_price)
 |  group by: i_brand, i_brand_id, t_hour, t_minute
-|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=298.71K
-|  in pipelines: 14(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.01K
+|  in pipelines: 14(GETNEXT), 11(OPEN)
 |
 13:HASH JOIN [INNER JOIN]
-|  hash predicates: time_sk = t_time_sk
-|  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF000[bloom] <- t_time_sk, RF001[min_max] <- t_time_sk
-|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=7,0,9 row-size=91B cardinality=298.71K
-|  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 11(OPEN)
-|
-|--11:SCAN HDFS [tpcds_parquet.time_dim]
-|     HDFS partitions=1/1 files=1 size=1.31MB
-|     predicates: t_meal_time IN ('breakfast', 'dinner')
-|     stored statistics:
-|       table: rows=86.40K size=1.31MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=86.40K
-|     parquet statistics predicates: t_meal_time IN ('breakfast', 'dinner')
-|     parquet dictionary predicates: t_meal_time IN ('breakfast', 'dinner')
-|     mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=9 row-size=31B cardinality=57.60K
-|     in pipelines: 11(GETNEXT)
-|
-12:HASH JOIN [INNER JOIN]
-|  hash predicates: sold_item_sk = i_item_sk
+|  hash predicates: t_time_sk = time_sk
 |  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF002[bloom] <- i_item_sk, RF003[min_max] <- i_item_sk
+|  runtime filters: RF000[bloom] <- time_sk, RF001[min_max] <- time_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7,0 row-size=60B cardinality=298.71K
-|  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 00(OPEN)
+|  tuple-ids=9,7,0 row-size=91B cardinality=1.01K
+|  in pipelines: 11(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
 |
-|--00:SCAN HDFS [tpcds_parquet.item]
-|     HDFS partitions=1/1 files=1 size=1.73MB
-|     predicates: i_manager_id = CAST(1 AS INT)
-|     stored statistics:
-|       table: rows=18.00K size=1.73MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=18.00K
-|     parquet statistics predicates: i_manager_id = CAST(1 AS INT)
-|     parquet dictionary predicates: i_manager_id = CAST(1 AS INT)
-|     mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=0 row-size=44B cardinality=182
-|     in pipelines: 00(GETNEXT)
-|
-01:UNION
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=7 row-size=16B cardinality=298.71K
-|  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
-|
-|--10:HASH JOIN [INNER JOIN]
-|  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF008[bloom] <- d_date_sk
+|--12:HASH JOIN [INNER JOIN]
+|  |  hash predicates: sold_item_sk = i_item_sk
+|  |  fk/pk conjuncts: assumed fk/pk
+|  |  runtime filters: RF002[bloom] <- i_item_sk, RF003[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=5,6 row-size=32B cardinality=170.55K
-|  |  in pipelines: 08(GETNEXT), 09(OPEN)
+|  |  tuple-ids=7,0 row-size=60B cardinality=1.01K
+|  |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 00(OPEN)
 |  |
-|  |--09:SCAN HDFS [tpcds_parquet.date_dim]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
+|  |--00:SCAN HDFS [tpcds_parquet.item]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     predicates: i_manager_id = CAST(1 AS INT)
 |  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
+|  |       table: rows=18.00K size=1.73MB
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
-|  |     parquet dictionary predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
-|  |     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=6 row-size=12B cardinality=108
-|  |     in pipelines: 09(GETNEXT)
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     parquet statistics predicates: i_manager_id = CAST(1 AS INT)
+|  |     parquet dictionary predicates: i_manager_id = CAST(1 AS INT)
+|  |     mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=1
+|  |     tuple-ids=0 row-size=44B cardinality=182
+|  |     in pipelines: 00(GETNEXT)
 |  |
-|  08:SCAN HDFS [tpcds_parquet.store_sales]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_sold_time_sk, RF003[min_max] -> tpcds_parquet.store_sales.ss_item_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_sold_time_sk, RF002[bloom] -> tpcds_parquet.store_sales.ss_item_sk, RF008[bloom] -> ss_sold_date_sk
-|     stored statistics:
-|       table: rows=2.88M size=200.96MB
-|       partitions: 1824/1824 rows=2.88M
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
-|     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=5 row-size=20B cardinality=2.88M
-|     in pipelines: 08(GETNEXT)
-|
-|--07:HASH JOIN [INNER JOIN]
-|  |  hash predicates: cs_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
-|  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
+|  01:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=16B cardinality=298.71K
+|  |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
+|  |
+|  |--10:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  runtime filters: RF008[bloom] <- d_date_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=5,6 row-size=32B cardinality=170.55K
+|  |  |  in pipelines: 08(GETNEXT), 09(OPEN)
+|  |  |
+|  |  |--09:SCAN HDFS [tpcds_parquet.date_dim]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
+|  |  |     parquet dictionary predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
+|  |  |     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |     tuple-ids=6 row-size=12B cardinality=108
+|  |  |     in pipelines: 09(GETNEXT)
+|  |  |
+|  |  08:SCAN HDFS [tpcds_parquet.store_sales]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     runtime filters: RF003[min_max] -> tpcds_parquet.store_sales.ss_item_sk, RF002[bloom] -> tpcds_parquet.store_sales.ss_item_sk, RF008[bloom] -> ss_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.95MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
+|  |     tuple-ids=5 row-size=20B cardinality=2.88M
+|  |     in pipelines: 08(GETNEXT)
+|  |
+|  |--07:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=3,4 row-size=32B cardinality=85.31K
+|  |  |  in pipelines: 05(GETNEXT), 06(OPEN)
+|  |  |
+|  |  |--06:SCAN HDFS [tpcds_parquet.date_dim]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
+|  |  |     parquet dictionary predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
+|  |  |     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
+|  |  |     tuple-ids=4 row-size=12B cardinality=108
+|  |  |     in pipelines: 06(GETNEXT)
+|  |  |
+|  |  05:SCAN HDFS [tpcds_parquet.catalog_sales]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF003[min_max] -> tpcds_parquet.catalog_sales.cs_item_sk, RF007[min_max] -> cs_sold_date_sk, RF002[bloom] -> tpcds_parquet.catalog_sales.cs_item_sk, RF006[bloom] -> cs_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
+|  |     tuple-ids=3 row-size=20B cardinality=1.44M
+|  |     in pipelines: 05(GETNEXT)
+|  |
+|  04:HASH JOIN [INNER JOIN]
+|  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  runtime filters: RF004[bloom] <- d_date_sk, RF005[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=3,4 row-size=32B cardinality=85.31K
-|  |  in pipelines: 05(GETNEXT), 06(OPEN)
+|  |  tuple-ids=1,2 row-size=32B cardinality=42.85K
+|  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |
-|  |--06:SCAN HDFS [tpcds_parquet.date_dim]
+|  |--03:SCAN HDFS [tpcds_parquet.date_dim]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |     predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
 |  |     stored statistics:
@@ -165,104 +184,86 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
 |  |     parquet dictionary predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
 |  |     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=4 row-size=12B cardinality=108
-|  |     in pipelines: 06(GETNEXT)
+|  |     tuple-ids=2 row-size=12B cardinality=108
+|  |     in pipelines: 03(GETNEXT)
 |  |
-|  05:SCAN HDFS [tpcds_parquet.catalog_sales]
-|     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.catalog_sales.cs_sold_time_sk, RF003[min_max] -> tpcds_parquet.catalog_sales.cs_item_sk, RF007[min_max] -> cs_sold_date_sk, RF000[bloom] -> tpcds_parquet.catalog_sales.cs_sold_time_sk, RF002[bloom] -> tpcds_parquet.catalog_sales.cs_item_sk, RF006[bloom] -> cs_sold_date_sk
-|     stored statistics:
-|       table: rows=1.44M size=96.62MB
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=650.14K
-|     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=3 row-size=20B cardinality=1.44M
-|     in pipelines: 05(GETNEXT)
-|
-04:HASH JOIN [INNER JOIN]
-|  hash predicates: ws_sold_date_sk = d_date_sk
-|  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
-|  runtime filters: RF004[bloom] <- d_date_sk, RF005[min_max] <- d_date_sk
-|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=32B cardinality=42.85K
-|  in pipelines: 02(GETNEXT), 03(OPEN)
-|
-|--03:SCAN HDFS [tpcds_parquet.date_dim]
-|     HDFS partitions=1/1 files=1 size=2.15MB
-|     predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
+|  02:SCAN HDFS [tpcds_parquet.web_sales]
+|     HDFS partitions=1/1 files=2 size=45.09MB
+|     runtime filters: RF003[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF005[min_max] -> ws_sold_date_sk, RF002[bloom] -> tpcds_parquet.web_sales.ws_item_sk, RF004[bloom] -> ws_sold_date_sk
 |     stored statistics:
-|       table: rows=73.05K size=2.15MB
+|       table: rows=719.38K size=45.09MB
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|     parquet statistics predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
-|     parquet dictionary predicates: d_year = CAST(1999 AS INT), d_moy = CAST(11 AS INT)
-|     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=2 row-size=12B cardinality=108
-|     in pipelines: 03(GETNEXT)
-|
-02:SCAN HDFS [tpcds_parquet.web_sales]
-   HDFS partitions=1/1 files=2 size=45.09MB
-   runtime filters: RF001[min_max] -> tpcds_parquet.web_sales.ws_sold_time_sk, RF003[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF005[min_max] -> ws_sold_date_sk, RF000[bloom] -> tpcds_parquet.web_sales.ws_sold_time_sk, RF002[bloom] -> tpcds_parquet.web_sales.ws_item_sk, RF004[bloom] -> ws_sold_date_sk
+|     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
+|     tuple-ids=1 row-size=20B cardinality=719.38K
+|     in pipelines: 02(GETNEXT)
+|
+11:SCAN HDFS [tpcds_parquet.time_dim]
+   HDFS partitions=1/1 files=1 size=1.31MB
+   predicates: t_meal_time IN ('breakfast', 'dinner')
+   runtime filters: RF001[min_max] -> t_time_sk, RF000[bloom] -> t_time_sk
    stored statistics:
-     table: rows=719.38K size=45.09MB
+     table: rows=86.40K size=1.31MB
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=644.77K
-   mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=1 row-size=20B cardinality=719.38K
-   in pipelines: 02(GETNEXT)
+   extrapolated-rows=disabled max-scan-range-rows=86.40K
+   parquet statistics predicates: t_meal_time IN ('breakfast', 'dinner')
+   parquet dictionary predicates: t_meal_time IN ('breakfast', 'dinner')
+   mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=1
+   tuple-ids=9 row-size=31B cardinality=57.60K
+   in pipelines: 11(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=57.38MB Threads=14
-Per-Host Resource Estimates: Memory=533MB
-F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=21.52MB mem-reservation=4.00MB thread-reservation=1
+Max Per-Host Resource Reservation: Memory=42.94MB Threads=15
+Per-Host Resource Estimates: Memory=515MB
+F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=4.19MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, t_hour, t_minute, sum(ext_price)
-|  mem-estimate=16.01MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-23:MERGING-EXCHANGE [UNPARTITIONED]
+24:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ext_price) DESC, i_brand_id ASC
-|  mem-estimate=5.51MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=11 row-size=56B cardinality=298.71K
+|  mem-estimate=196.21KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=11 row-size=56B cardinality=1.01K
 |  in pipelines: 15(GETNEXT)
 |
-F09:PLAN FRAGMENT [HASH(i_brand,i_brand_id,t_hour,t_minute)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=22.00MB mem-reservation=20.50MB thread-reservation=1
+F10:PLAN FRAGMENT [HASH(i_brand,i_brand_id,t_hour,t_minute)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-reservation=1
 15:SORT
 |  order by: sum(ext_price) DESC, i_brand_id ASC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=11 row-size=56B cardinality=298.71K
-|  in pipelines: 15(GETNEXT), 22(OPEN)
+|  tuple-ids=11 row-size=56B cardinality=1.01K
+|  in pipelines: 15(GETNEXT), 23(OPEN)
 |
-22:AGGREGATE [FINALIZE]
+23:AGGREGATE [FINALIZE]
 |  output: sum:merge(ext_price)
 |  group by: i_brand, i_brand_id, t_hour, t_minute
-|  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=298.71K
-|  in pipelines: 22(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.01K
+|  in pipelines: 23(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
 |
-21:EXCHANGE [HASH(i_brand,i_brand_id,t_hour,t_minute)]
-|  mem-estimate=5.51MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=298.71K
+22:EXCHANGE [HASH(i_brand,i_brand_id,t_hour,t_minute)]
+|  mem-estimate=196.21KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.01K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
-F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=217.39MB mem-reservation=30.62MB thread-reservation=2 runtime-filters-memory=5.00MB
+F09:PLAN FRAGMENT [HASH(time_sk)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=15.82MB mem-reservation=5.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 14:AGGREGATE [STREAMING]
 |  output: sum(ext_price)
 |  group by: i_brand, i_brand_id, t_hour, t_minute
-|  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=298.71K
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.01K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
-13:HASH JOIN [INNER JOIN, BROADCAST]
+13:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: time_sk = t_time_sk
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF000[bloom] <- t_time_sk, RF001[min_max] <- t_time_sk
-|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=7,0,9 row-size=91B cardinality=298.71K
+|  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=7,0,9 row-size=91B cardinality=1.01K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 11(OPEN)
 |
-|--20:EXCHANGE [BROADCAST]
+|--21:EXCHANGE [HASH(t_time_sk)]
 |  |  mem-estimate=1.73MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=9 row-size=31B cardinality=57.60K
 |  |  in pipelines: 11(GETNEXT)
@@ -282,12 +283,19 @@ Per-Host Resources: mem-estimate=217.39MB mem-reservation=30.62MB thread-reserva
 |     tuple-ids=9 row-size=31B cardinality=57.60K
 |     in pipelines: 11(GETNEXT)
 |
+20:EXCHANGE [HASH(time_sk)]
+|  mem-estimate=221.15KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=7,0 row-size=60B cardinality=1.01K
+|  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
+|
+F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reservation=2 runtime-filters-memory=5.00MB
 12:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: sold_item_sk = i_item_sk
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF002[bloom] <- i_item_sk, RF003[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7,0 row-size=60B cardinality=298.71K
+|  tuple-ids=7,0 row-size=60B cardinality=1.01K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 00(OPEN)
 |
 |--19:EXCHANGE [BROADCAST]
@@ -344,10 +352,10 @@ Per-Host Resources: mem-estimate=217.39MB mem-reservation=30.62MB thread-reserva
 |  |     in pipelines: 09(GETNEXT)
 |  |
 |  08:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
 |     runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_sold_time_sk, RF003[min_max] -> tpcds_parquet.store_sales.ss_item_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_sold_time_sk, RF002[bloom] -> tpcds_parquet.store_sales.ss_item_sk, RF008[bloom] -> ss_sold_date_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
+|       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -433,67 +441,66 @@ Per-Host Resources: mem-estimate=217.39MB mem-reservation=30.62MB thread-reserva
    tuple-ids=1 row-size=20B cardinality=719.38K
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=100.75MB Threads=15
-Per-Host Resource Estimates: Memory=298MB
-F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=21.70MB mem-reservation=4.00MB thread-reservation=1
+Max Per-Host Resource Reservation: Memory=84.50MB Threads=18
+Per-Host Resource Estimates: Memory=279MB
+F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.37MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, t_hour, t_minute, sum(ext_price)
-|  mem-estimate=16.01MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-23:MERGING-EXCHANGE [UNPARTITIONED]
+24:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ext_price) DESC, i_brand_id ASC
-|  mem-estimate=5.69MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=11 row-size=56B cardinality=298.71K
+|  mem-estimate=373.98KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=11 row-size=56B cardinality=1.01K
 |  in pipelines: 15(GETNEXT)
 |
-F09:PLAN FRAGMENT [HASH(i_brand,i_brand_id,t_hour,t_minute)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=22.00MB mem-reservation=16.75MB thread-reservation=1
+F10:PLAN FRAGMENT [HASH(i_brand,i_brand_id,t_hour,t_minute)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-reservation=1
 15:SORT
 |  order by: sum(ext_price) DESC, i_brand_id ASC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=11 row-size=56B cardinality=298.71K
-|  in pipelines: 15(GETNEXT), 22(OPEN)
+|  tuple-ids=11 row-size=56B cardinality=1.01K
+|  in pipelines: 15(GETNEXT), 23(OPEN)
 |
-22:AGGREGATE [FINALIZE]
+23:AGGREGATE [FINALIZE]
 |  output: sum:merge(ext_price)
 |  group by: i_brand, i_brand_id, t_hour, t_minute
-|  mem-estimate=10.00MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=298.71K
-|  in pipelines: 22(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.01K
+|  in pipelines: 23(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
 |
-21:EXCHANGE [HASH(i_brand,i_brand_id,t_hour,t_minute)]
-|  mem-estimate=5.69MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=298.71K
+22:EXCHANGE [HASH(i_brand,i_brand_id,t_hour,t_minute)]
+|  mem-estimate=373.98KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.01K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
-F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB
-Per-Instance Resources: mem-estimate=58.00MB mem-reservation=13.00MB thread-reservation=1
+F09:PLAN FRAGMENT [HASH(time_sk)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=10.41MB mem-reservation=2.00MB thread-reservation=1
 14:AGGREGATE [STREAMING]
 |  output: sum(ext_price)
 |  group by: i_brand, i_brand_id, t_hour, t_minute
-|  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=298.71K
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.01K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
-13:HASH JOIN [INNER JOIN, BROADCAST]
+13:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: time_sk = t_time_sk
 |  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=7,0,9 row-size=91B cardinality=298.71K
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=7,0,9 row-size=91B cardinality=1.01K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 11(OPEN)
 |
-|--F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=12.23MB mem-reservation=10.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F12:PLAN FRAGMENT [HASH(time_sk)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=4.67MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: t_time_sk
 |  |  runtime filters: RF000[bloom] <- t_time_sk, RF001[min_max] <- t_time_sk
-|  |  mem-estimate=9.50MB mem-reservation=9.50MB spill-buffer=256.00KB thread-reservation=0
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  20:EXCHANGE [BROADCAST]
+|  21:EXCHANGE [HASH(t_time_sk)]
 |  |  mem-estimate=1.73MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=9 row-size=31B cardinality=57.60K
 |  |  in pipelines: 11(GETNEXT)
@@ -513,15 +520,23 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=13.00MB thread-rese
 |     tuple-ids=9 row-size=31B cardinality=57.60K
 |     in pipelines: 11(GETNEXT)
 |
+20:EXCHANGE [HASH(time_sk)]
+|  mem-estimate=422.54KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=7,0 row-size=60B cardinality=1.01K
+|  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
+|
+F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB
+Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1
 12:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=01
 |  hash predicates: sold_item_sk = i_item_sk
 |  fk/pk conjuncts: assumed fk/pk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7,0 row-size=60B cardinality=298.71K
+|  tuple-ids=7,0 row-size=60B cardinality=1.01K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 00(OPEN)
 |
-|--F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|--F13:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
@@ -562,7 +577,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=13.00MB thread-rese
 |  |  tuple-ids=5,6 row-size=32B cardinality=170.55K
 |  |  in pipelines: 08(GETNEXT), 09(OPEN)
 |  |
-|  |--F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |--F16:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=01
@@ -591,10 +606,10 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=13.00MB thread-rese
 |  |     in pipelines: 09(GETNEXT)
 |  |
 |  08:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
 |     runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_sold_time_sk, RF003[min_max] -> tpcds_parquet.store_sales.ss_item_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_sold_time_sk, RF002[bloom] -> tpcds_parquet.store_sales.ss_item_sk, RF008[bloom] -> ss_sold_date_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
+|       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -610,7 +625,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=13.00MB thread-rese
 |  |  tuple-ids=3,4 row-size=32B cardinality=85.31K
 |  |  in pipelines: 05(GETNEXT), 06(OPEN)
 |  |
-|  |--F14:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |--F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=01
@@ -657,7 +672,7 @@ Per-Instance Resources: mem-estimate=58.00MB mem-reservation=13.00MB thread-rese
 |  tuple-ids=1,2 row-size=32B cardinality=42.85K
 |  in pipelines: 02(GETNEXT), 03(OPEN)
 |
-|--F13:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|--F14:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
index e91b7df..f159253 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
@@ -85,7 +85,7 @@ PLAN-ROOT SINK
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
 |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
 |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=18,8,28,38 row-size=212B cardinality=589.03K
+|  tuple-ids=18,28,8,38 row-size=212B cardinality=43.59K
 |  in pipelines: 13(GETNEXT), 27(OPEN)
 |
 |--21:UNION
@@ -155,8 +155,81 @@ PLAN-ROOT SINK
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
+|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=18,28,8 row-size=168B cardinality=43.59K
+|  in pipelines: 13(GETNEXT), 06(OPEN)
+|
+|--00:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=44B cardinality=58.90K
+|  |  in pipelines: 06(GETNEXT)
+|  |
+|  06:AGGREGATE [FINALIZE]
+|  |  output: sum(ss_net_paid)
+|  |  group by: c_customer_id, c_first_name, c_last_name, d_year
+|  |  having: sum(ss_net_paid) > CAST(0 AS DECIMAL(3,0))
+|  |  mem-estimate=56.15MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=3 row-size=84B cardinality=58.90K
+|  |  in pipelines: 06(GETNEXT), 02(OPEN)
+|  |
+|  05:HASH JOIN [INNER JOIN]
+|  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  runtime filters: RF014[bloom] <- c_customer_sk, RF015[min_max] <- c_customer_sk
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=1,2,0 row-size=88B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |
+|  |--01:SCAN HDFS [tpcds_parquet.customer]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
+|  |     tuple-ids=0 row-size=68B cardinality=100.00K
+|  |     in pipelines: 01(GETNEXT)
+|  |
+|  04:HASH JOIN [INNER JOIN]
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  runtime filters: RF016[bloom] <- d_date_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |
+|  |--03:SCAN HDFS [tpcds_parquet.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
+|  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|  |     tuple-ids=2 row-size=8B cardinality=373
+|  |     in pipelines: 03(GETNEXT)
+|  |
+|  02:SCAN HDFS [tpcds_parquet.store_sales]
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk, RF014[bloom] -> ss_customer_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.95MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
+|     tuple-ids=1 row-size=12B cardinality=2.88M
+|     in pipelines: 02(GETNEXT)
+|
+28:HASH JOIN [INNER JOIN]
+|  hash predicates: customer_id = customer_id
+|  fk/pk conjuncts: assumed fk/pk
+|  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=18,8,28 row-size=168B cardinality=589.03K
+|  tuple-ids=18,28 row-size=124B cardinality=43.59K
 |  in pipelines: 13(GETNEXT), 20(OPEN)
 |
 |--14:UNION
@@ -175,7 +248,7 @@ PLAN-ROOT SINK
 |  19:HASH JOIN [INNER JOIN]
 |  |  hash predicates: c_customer_sk = ws_bill_customer_sk
 |  |  fk/pk conjuncts: c_customer_sk = ws_bill_customer_sk
-|  |  runtime filters: RF014[bloom] <- ws_bill_customer_sk, RF015[min_max] <- ws_bill_customer_sk
+|  |  runtime filters: RF010[bloom] <- ws_bill_customer_sk, RF011[min_max] <- ws_bill_customer_sk
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=24,25,26 row-size=88B cardinality=148.00K
 |  |  in pipelines: 15(GETNEXT), 16(OPEN)
@@ -183,7 +256,7 @@ PLAN-ROOT SINK
 |  |--18:HASH JOIN [INNER JOIN]
 |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
 |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
-|  |  |  runtime filters: RF016[bloom] <- d_date_sk, RF017[min_max] <- d_date_sk
+|  |  |  runtime filters: RF012[bloom] <- d_date_sk, RF013[min_max] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
 |  |  |  in pipelines: 16(GETNEXT), 17(OPEN)
@@ -203,7 +276,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  16:SCAN HDFS [tpcds_parquet.web_sales]
 |  |     HDFS partitions=1/1 files=2 size=45.09MB
-|  |     runtime filters: RF017[min_max] -> ws_sold_date_sk, RF016[bloom] -> ws_sold_date_sk
+|  |     runtime filters: RF013[min_max] -> ws_sold_date_sk, RF012[bloom] -> ws_sold_date_sk
 |  |     stored statistics:
 |  |       table: rows=719.38K size=45.09MB
 |  |       columns: all
@@ -214,7 +287,7 @@ PLAN-ROOT SINK
 |  |
 |  15:SCAN HDFS [tpcds_parquet.customer]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF015[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF014[bloom] -> c_customer_sk
+|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF011[min_max] -> c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF010[bloom] -> c_customer_sk
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
@@ -223,79 +296,6 @@ PLAN-ROOT SINK
 |     tuple-ids=24 row-size=68B cardinality=100.00K
 |     in pipelines: 15(GETNEXT)
 |
-28:HASH JOIN [INNER JOIN]
-|  hash predicates: customer_id = customer_id
-|  fk/pk conjuncts: assumed fk/pk
-|  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=18,8 row-size=124B cardinality=589.03K
-|  in pipelines: 13(GETNEXT), 06(OPEN)
-|
-|--00:UNION
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=44B cardinality=58.90K
-|  |  in pipelines: 06(GETNEXT)
-|  |
-|  06:AGGREGATE [FINALIZE]
-|  |  output: sum(ss_net_paid)
-|  |  group by: c_customer_id, c_first_name, c_last_name, d_year
-|  |  having: sum(ss_net_paid) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=56.15MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=84B cardinality=58.90K
-|  |  in pipelines: 06(GETNEXT), 02(OPEN)
-|  |
-|  05:HASH JOIN [INNER JOIN]
-|  |  hash predicates: ss_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  |  runtime filters: RF010[bloom] <- c_customer_sk, RF011[min_max] <- c_customer_sk
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=1,2,0 row-size=88B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 01(OPEN)
-|  |
-|  |--01:SCAN HDFS [tpcds_parquet.customer]
-|  |     HDFS partitions=1/1 files=1 size=5.49MB
-|  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
-|  |     stored statistics:
-|  |       table: rows=100.00K size=5.49MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
-|  |     mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=68B cardinality=100.00K
-|  |     in pipelines: 01(GETNEXT)
-|  |
-|  04:HASH JOIN [INNER JOIN]
-|  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF012[bloom] <- d_date_sk
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 03(OPEN)
-|  |
-|  |--03:SCAN HDFS [tpcds_parquet.date_dim]
-|  |     HDFS partitions=1/1 files=1 size=2.15MB
-|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     stored statistics:
-|  |       table: rows=73.05K size=2.15MB
-|  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
-|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
-|  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=8B cardinality=373
-|  |     in pipelines: 03(GETNEXT)
-|  |
-|  02:SCAN HDFS [tpcds_parquet.store_sales]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|     runtime filters: RF011[min_max] -> ss_customer_sk, RF012[bloom] -> ss_sold_date_sk, RF010[bloom] -> ss_customer_sk
-|     stored statistics:
-|       table: rows=2.88M size=200.96MB
-|       partitions: 1824/1824 rows=2.88M
-|       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
-|     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=1 row-size=12B cardinality=2.88M
-|     in pipelines: 02(GETNEXT)
-|
 07:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  tuple-ids=18 row-size=80B cardinality=589.03K
@@ -318,7 +318,7 @@ PLAN-ROOT SINK
 |
 |--08:SCAN HDFS [tpcds_parquet.customer]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id
+|     runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
@@ -349,10 +349,10 @@ PLAN-ROOT SINK
 |     in pipelines: 10(GETNEXT)
 |
 09:SCAN HDFS [tpcds_parquet.store_sales]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
    runtime filters: RF007[min_max] -> ss_customer_sk, RF008[bloom] -> ss_sold_date_sk, RF006[bloom] -> ss_customer_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -360,63 +360,63 @@ PLAN-ROOT SINK
    tuple-ids=11 row-size=12B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=258.62MB Threads=33
-Per-Host Resource Estimates: Memory=947MB
-F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+Max Per-Host Resource Reservation: Memory=240.75MB Threads=34
+Per-Host Resource Estimates: Memory=926MB
+F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-55:MERGING-EXCHANGE [UNPARTITIONED]
+56:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC
 |  limit: 100
 |  mem-estimate=22.00KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=40 row-size=64B cardinality=100
 |  in pipelines: 31(GETNEXT)
 |
-F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservation=1 runtime-filters-memory=3.00MB
+F18:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=15.89MB mem-reservation=9.62MB thread-reservation=1 runtime-filters-memory=2.00MB
 31:TOP-N [LIMIT=100]
 |  order by: customer_id ASC
 |  mem-estimate=6.25KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=40 row-size=64B cardinality=100
 |  in pipelines: 31(GETNEXT), 36(OPEN)
 |
-30:HASH JOIN [INNER JOIN, BROADCAST]
+30:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
 |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
-|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=18,8,28,38 row-size=212B cardinality=589.03K
-|  in pipelines: 36(GETNEXT), 53(OPEN)
+|  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=18,28,8,38 row-size=212B cardinality=43.59K
+|  in pipelines: 36(GETNEXT), 54(OPEN)
 |
-|--54:EXCHANGE [BROADCAST]
-|  |  mem-estimate=6.30MB mem-reservation=0B thread-reservation=0
+|--55:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=3.20MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=38 row-size=44B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT)
+|  |  in pipelines: 54(GETNEXT)
 |  |
-|  F23:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1
 |  21:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=38 row-size=44B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT)
+|  |  in pipelines: 54(GETNEXT)
 |  |
-|  53:AGGREGATE [FINALIZE]
+|  54:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
 |  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |  tuple-ids=37 row-size=84B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT), 23(OPEN)
+|  |  in pipelines: 54(GETNEXT), 23(OPEN)
 |  |
-|  52:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,d_year)]
+|  53:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,d_year)]
 |  |  mem-estimate=6.10MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=37 row-size=84B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F21:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  F22:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=26.52MB mem-reservation=17.50MB thread-reservation=1
 |  27:AGGREGATE [STREAMING]
 |  |  output: sum(ws_net_paid)
@@ -433,12 +433,12 @@ Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservat
 |  |  tuple-ids=35,36,34 row-size=88B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT), 22(OPEN)
 |  |
-|  |--51:EXCHANGE [HASH(c_customer_sk)]
+|  |--52:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=6.55MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=34 row-size=68B cardinality=100.00K
 |  |  |  in pipelines: 22(GETNEXT)
 |  |  |
-|  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=2
 |  |  22:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
@@ -450,12 +450,12 @@ Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservat
 |  |     tuple-ids=34 row-size=68B cardinality=100.00K
 |  |     in pipelines: 22(GETNEXT)
 |  |
-|  50:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  51:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=1.47MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=35,36 row-size=20B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F18:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F19:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=98.95MB mem-reservation=10.94MB thread-reservation=2 runtime-filters-memory=1.00MB
 |  25:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ws_sold_date_sk = d_date_sk
@@ -465,12 +465,12 @@ Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservat
 |  |  tuple-ids=35,36 row-size=20B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT), 24(OPEN)
 |  |
-|  |--49:EXCHANGE [BROADCAST]
+|  |--50:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=36 row-size=8B cardinality=373
 |  |  |  in pipelines: 24(GETNEXT)
 |  |  |
-|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
 |  |  24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -496,64 +496,64 @@ Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservat
 |     tuple-ids=35 row-size=12B cardinality=719.38K
 |     in pipelines: 23(GETNEXT)
 |
-29:HASH JOIN [INNER JOIN, BROADCAST]
+29:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=18,8,28 row-size=168B cardinality=589.03K
+|  tuple-ids=18,28,8 row-size=168B cardinality=43.59K
 |  in pipelines: 36(GETNEXT), 47(OPEN)
 |
-|--48:EXCHANGE [BROADCAST]
-|  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=44B cardinality=14.80K
+|--49:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=987.66KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=44B cardinality=58.90K
 |  |  in pipelines: 47(GETNEXT)
 |  |
-|  F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1
-|  14:UNION
+|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Resources: mem-estimate=44.26MB mem-reservation=34.00MB thread-reservation=1
+|  00:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=44B cardinality=14.80K
+|  |  tuple-ids=8 row-size=44B cardinality=58.90K
 |  |  in pipelines: 47(GETNEXT)
 |  |
 |  47:AGGREGATE [FINALIZE]
-|  |  output: sum:merge(ws_net_paid)
+|  |  output: sum:merge(ss_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
-|  |  having: sum(ws_net_paid) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=27 row-size=84B cardinality=14.80K
-|  |  in pipelines: 47(GETNEXT), 16(OPEN)
+|  |  having: sum(ss_net_paid) > CAST(0 AS DECIMAL(3,0))
+|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=3 row-size=84B cardinality=58.90K
+|  |  in pipelines: 47(GETNEXT), 02(OPEN)
 |  |
 |  46:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,d_year)]
-|  |  mem-estimate=6.10MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=27 row-size=84B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT)
+|  |  mem-estimate=10.26MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=84B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
 |  |
-|  F15:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=27.52MB mem-reservation=18.50MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  20:AGGREGATE [STREAMING]
-|  |  output: sum(ws_net_paid)
+|  F15:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
+|  Per-Host Resources: mem-estimate=33.13MB mem-reservation=22.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  06:AGGREGATE [STREAMING]
+|  |  output: sum(ss_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
-|  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=27 row-size=84B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT)
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=3 row-size=84B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
 |  |
-|  19:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash predicates: ws_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
+|  05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  |  runtime filters: RF014[bloom] <- c_customer_sk, RF015[min_max] <- c_customer_sk
-|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=25,26,24 row-size=88B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT), 15(OPEN)
+|  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=1,2,0 row-size=88B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT), 01(OPEN)
 |  |
 |  |--45:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=6.55MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=24 row-size=68B cardinality=100.00K
-|  |  |  in pipelines: 15(GETNEXT)
+|  |  |  tuple-ids=0 row-size=68B cardinality=100.00K
+|  |  |  in pipelines: 01(GETNEXT)
 |  |  |
 |  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=65.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB
-|  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
@@ -561,32 +561,32 @@ Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |     mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=24 row-size=68B cardinality=100.00K
-|  |     in pipelines: 15(GETNEXT)
+|  |     tuple-ids=0 row-size=68B cardinality=100.00K
+|  |     in pipelines: 01(GETNEXT)
 |  |
-|  44:EXCHANGE [HASH(ws_bill_customer_sk)]
-|  |  mem-estimate=1.47MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT)
+|  44:EXCHANGE [HASH(ss_customer_sk)]
+|  |  mem-estimate=3.83MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
 |  |
-|  F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=99.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB
-|  18:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: ws_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
-|  |  runtime filters: RF016[bloom] <- d_date_sk, RF017[min_max] <- d_date_sk
+|  F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  runtime filters: RF016[bloom] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |
 |  |--43:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=26 row-size=8B cardinality=373
-|  |  |  in pipelines: 17(GETNEXT)
+|  |  |  tuple-ids=2 row-size=8B cardinality=373
+|  |  |  in pipelines: 03(GETNEXT)
 |  |  |
 |  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
-|  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     stored statistics:
@@ -596,78 +596,86 @@ Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservat
 |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=26 row-size=8B cardinality=373
-|  |     in pipelines: 17(GETNEXT)
+|  |     tuple-ids=2 row-size=8B cardinality=373
+|  |     in pipelines: 03(GETNEXT)
 |  |
-|  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
-|     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF017[min_max] -> ws_sold_date_sk, RF015[min_max] -> ws_bill_customer_sk, RF016[bloom] -> ws_sold_date_sk, RF014[bloom] -> ws_bill_customer_sk
+|  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk, RF014[bloom] -> ss_customer_sk
 |     stored statistics:
-|       table: rows=719.38K size=45.09MB
+|       table: rows=2.88M size=200.95MB
+|       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=644.77K
-|     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=25 row-size=12B cardinality=719.38K
-|     in pipelines: 16(GETNEXT)
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
+|     tuple-ids=1 row-size=12B cardinality=2.88M
+|     in pipelines: 02(GETNEXT)
+|
+48:EXCHANGE [HASH(customer_id)]
+|  mem-estimate=2.10MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=18,28 row-size=124B cardinality=43.59K
+|  in pipelines: 36(GETNEXT)
 |
+F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=48.85MB mem-reservation=37.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 28:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=18,8 row-size=124B cardinality=589.03K
+|  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=18,28 row-size=124B cardinality=43.59K
 |  in pipelines: 36(GETNEXT), 41(OPEN)
 |
 |--42:EXCHANGE [BROADCAST]
-|  |  mem-estimate=2.61MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=44B cardinality=58.90K
+|  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=28 row-size=44B cardinality=14.80K
 |  |  in pipelines: 41(GETNEXT)
 |  |
-|  F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=44.26MB mem-reservation=34.00MB thread-reservation=1
-|  00:UNION
+|  F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Host Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1
+|  14:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=44B cardinality=58.90K
+|  |  tuple-ids=28 row-size=44B cardinality=14.80K
 |  |  in pipelines: 41(GETNEXT)
 |  |
 |  41:AGGREGATE [FINALIZE]
-|  |  output: sum:merge(ss_net_paid)
+|  |  output: sum:merge(ws_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
-|  |  having: sum(ss_net_paid) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=84B cardinality=58.90K
-|  |  in pipelines: 41(GETNEXT), 02(OPEN)
+|  |  having: sum(ws_net_paid) > CAST(0 AS DECIMAL(3,0))
+|  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  tuple-ids=27 row-size=84B cardinality=14.80K
+|  |  in pipelines: 41(GETNEXT), 16(OPEN)
 |  |
 |  40:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,d_year)]
-|  |  mem-estimate=10.26MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=84B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  mem-estimate=6.10MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=27 row-size=84B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT)
 |  |
-|  F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=33.13MB mem-reservation=22.75MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  06:AGGREGATE [STREAMING]
-|  |  output: sum(ss_net_paid)
+|  F09:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  Per-Host Resources: mem-estimate=27.52MB mem-reservation=18.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  20:AGGREGATE [STREAMING]
+|  |  output: sum(ws_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=84B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
+|  |  tuple-ids=27 row-size=84B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT)
 |  |
-|  05:HASH JOIN [INNER JOIN, PARTITIONED]
-|  |  hash predicates: ss_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  19:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash predicates: ws_bill_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
 |  |  runtime filters: RF010[bloom] <- c_customer_sk, RF011[min_max] <- c_customer_sk
-|  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  |  tuple-ids=1,2,0 row-size=88B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  tuple-ids=25,26,24 row-size=88B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT), 15(OPEN)
 |  |
 |  |--39:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=6.55MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0 row-size=68B cardinality=100.00K
-|  |  |  in pipelines: 01(GETNEXT)
+|  |  |  tuple-ids=24 row-size=68B cardinality=100.00K
+|  |  |  in pipelines: 15(GETNEXT)
 |  |  |
 |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=66.00MB mem-reservation=6.00MB thread-reservation=2 runtime-filters-memory=2.00MB
-|  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
@@ -675,32 +683,32 @@ Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |     mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=68B cardinality=100.00K
-|  |     in pipelines: 01(GETNEXT)
+|  |     tuple-ids=24 row-size=68B cardinality=100.00K
+|  |     in pipelines: 15(GETNEXT)
 |  |
-|  38:EXCHANGE [HASH(ss_customer_sk)]
-|  |  mem-estimate=3.83MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  38:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  mem-estimate=1.47MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT)
 |  |
-|  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=2.00MB
-|  04:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF012[bloom] <- d_date_sk
+|  F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Host Resources: mem-estimate=99.95MB mem-reservation=11.94MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  18:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  runtime filters: RF012[bloom] <- d_date_sk, RF013[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT), 17(OPEN)
 |  |
 |  |--37:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=8B cardinality=373
-|  |  |  in pipelines: 03(GETNEXT)
+|  |  |  tuple-ids=26 row-size=8B cardinality=373
+|  |  |  in pipelines: 17(GETNEXT)
 |  |  |
 |  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=2
-|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     stored statistics:
@@ -710,20 +718,19 @@ Per-Host Resources: mem-estimate=85.26MB mem-reservation=65.38MB thread-reservat
 |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=8B cardinality=373
-|  |     in pipelines: 03(GETNEXT)
+|  |     tuple-ids=26 row-size=8B cardinality=373
+|  |     in pipelines: 17(GETNEXT)
 |  |
-|  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|     runtime filters: RF011[min_max] -> ss_customer_sk, RF012[bloom] -> ss_sold_date_sk, RF010[bloom] -> ss_customer_sk
+|  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|     HDFS partitions=1/1 files=2 size=45.09MB
+|     runtime filters: RF013[min_max] -> ws_sold_date_sk, RF011[min_max] -> ws_bill_customer_sk, RF012[bloom] -> ws_sold_date_sk, RF010[bloom] -> ws_bill_customer_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
-|       partitions: 1824/1824 rows=2.88M
+|       table: rows=719.38K size=45.09MB
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
-|     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=1 row-size=12B cardinality=2.88M
-|     in pipelines: 02(GETNEXT)
+|     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
+|     tuple-ids=25 row-size=12B cardinality=719.38K
+|     in pipelines: 16(GETNEXT)
 |
 07:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -768,7 +775,7 @@ Per-Host Resources: mem-estimate=33.13MB mem-reservation=22.75MB thread-reservat
 |  Per-Host Resources: mem-estimate=67.00MB mem-reservation=7.00MB thread-reservation=2 runtime-filters-memory=3.00MB
 |  08:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id
+|     runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
@@ -813,10 +820,10 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati
 |     in pipelines: 10(GETNEXT)
 |
 09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
    runtime filters: RF007[min_max] -> ss_customer_sk, RF008[bloom] -> ss_sold_date_sk, RF006[bloom] -> ss_customer_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -824,71 +831,71 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati
    tuple-ids=11 row-size=12B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=306.75MB Threads=40
-Per-Host Resource Estimates: Memory=656MB
-F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+Max Per-Host Resource Reservation: Memory=267.38MB Threads=44
+Per-Host Resource Estimates: Memory=621MB
+F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
-55:MERGING-EXCHANGE [UNPARTITIONED]
+56:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC
 |  limit: 100
 |  mem-estimate=41.91KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=40 row-size=64B cardinality=100
 |  in pipelines: 31(GETNEXT)
 |
-F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-reservation=1
+F18:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=2.50MB mem-reservation=0B thread-reservation=1
 31:TOP-N [LIMIT=100]
 |  order by: customer_id ASC
 |  mem-estimate=6.25KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=40 row-size=64B cardinality=100
 |  in pipelines: 31(GETNEXT), 36(OPEN)
 |
-30:HASH JOIN [INNER JOIN, BROADCAST]
+30:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
 |  other predicates: CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END > CASE WHEN year_total > CAST(0 AS DECIMAL(3,0)) THEN year_total / year_total ELSE NULL END
-|  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
-|  tuple-ids=18,8,28,38 row-size=212B cardinality=589.03K
-|  in pipelines: 36(GETNEXT), 53(OPEN)
+|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=18,28,8,38 row-size=212B cardinality=43.59K
+|  in pipelines: 36(GETNEXT), 54(OPEN)
 |
-|--F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=41.30MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F26:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=7.07MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: customer_id
 |  |  runtime filters: RF000[bloom] <- customer_id, RF001[min_max] <- customer_id
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  |
-|  54:EXCHANGE [BROADCAST]
-|  |  mem-estimate=6.30MB mem-reservation=0B thread-reservation=0
+|  55:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=3.20MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=38 row-size=44B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT)
+|  |  in pipelines: 54(GETNEXT)
 |  |
-|  F23:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F24:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1
 |  21:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=38 row-size=44B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT)
+|  |  in pipelines: 54(GETNEXT)
 |  |
-|  53:AGGREGATE [FINALIZE]
+|  54:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
 |  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |  tuple-ids=37 row-size=84B cardinality=148.00K
-|  |  in pipelines: 53(GETNEXT), 23(OPEN)
+|  |  in pipelines: 54(GETNEXT), 23(OPEN)
 |  |
-|  52:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,d_year)]
+|  53:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,d_year)]
 |  |  mem-estimate=6.10MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=37 row-size=84B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F21:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  F22:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=11.47MB mem-reservation=9.00MB thread-reservation=1
 |  27:AGGREGATE [STREAMING]
 |  |  output: sum(ws_net_paid)
@@ -905,7 +912,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |  tuple-ids=35,36,34 row-size=88B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT), 22(OPEN)
 |  |
-|  |--F26:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |--F27:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=15.05MB mem-reservation=8.50MB thread-reservation=1
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
@@ -913,12 +920,12 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |  |  runtime filters: RF019[min_max] <- c_customer_sk
 |  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |  |
-|  |  51:EXCHANGE [HASH(c_customer_sk)]
+|  |  52:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=6.55MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=34 row-size=68B cardinality=100.00K
 |  |  |  in pipelines: 22(GETNEXT)
 |  |  |
-|  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1
 |  |  22:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
@@ -930,12 +937,12 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |     tuple-ids=34 row-size=68B cardinality=100.00K
 |  |     in pipelines: 22(GETNEXT)
 |  |
-|  50:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  51:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=1.47MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=35,36 row-size=20B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F18:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F19:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
 |  25:HASH JOIN [INNER JOIN, BROADCAST]
@@ -946,7 +953,7 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |  tuple-ids=35,36 row-size=20B cardinality=148.00K
 |  |  in pipelines: 23(GETNEXT), 24(OPEN)
 |  |
-|  |--F27:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |--F28:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
@@ -954,12 +961,12 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |  |  runtime filters: RF020[bloom] <- d_date_sk, RF021[min_max] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
-|  |  49:EXCHANGE [BROADCAST]
+|  |  50:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=36 row-size=8B cardinality=373
 |  |  |  in pipelines: 24(GETNEXT)
 |  |  |
-|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
 |  |  24:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
@@ -985,81 +992,81 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |     tuple-ids=35 row-size=12B cardinality=719.38K
 |     in pipelines: 23(GETNEXT)
 |
-29:HASH JOIN [INNER JOIN, BROADCAST]
+29:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=03
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=18,8,28 row-size=168B cardinality=589.03K
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=18,28,8 row-size=168B cardinality=43.59K
 |  in pipelines: 36(GETNEXT), 47(OPEN)
 |
-|--F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=7.46MB mem-reservation=6.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F29:PLAN FRAGMENT [HASH(customer_id)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: customer_id
 |  |  runtime filters: RF002[bloom] <- customer_id, RF003[min_max] <- customer_id
-|  |  mem-estimate=5.75MB mem-reservation=5.75MB spill-buffer=128.00KB thread-reservation=0
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  48:EXCHANGE [BROADCAST]
-|  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=44B cardinality=14.80K
+|  49:EXCHANGE [HASH(customer_id)]
+|  |  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=44B cardinality=58.90K
 |  |  in pipelines: 47(GETNEXT)
 |  |
-|  F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Instance Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1
-|  14:UNION
+|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-reservation=1
+|  00:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=44B cardinality=14.80K
+|  |  tuple-ids=8 row-size=44B cardinality=58.90K
 |  |  in pipelines: 47(GETNEXT)
 |  |
 |  47:AGGREGATE [FINALIZE]
-|  |  output: sum:merge(ws_net_paid)
+|  |  output: sum:merge(ss_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
-|  |  having: sum(ws_net_paid) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=27 row-size=84B cardinality=14.80K
-|  |  in pipelines: 47(GETNEXT), 16(OPEN)
+|  |  having: sum(ss_net_paid) > CAST(0 AS DECIMAL(3,0))
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=3 row-size=84B cardinality=58.90K
+|  |  in pipelines: 47(GETNEXT), 02(OPEN)
 |  |
 |  46:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,d_year)]
-|  |  mem-estimate=6.10MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=27 row-size=84B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT)
+|  |  mem-estimate=10.52MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=84B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
 |  |
-|  F15:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
-|  Per-Instance Resources: mem-estimate=11.47MB mem-reservation=9.00MB thread-reservation=1
-|  20:AGGREGATE [STREAMING]
-|  |  output: sum(ws_net_paid)
+|  F15:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|  Per-Instance Resources: mem-estimate=13.91MB mem-reservation=9.00MB thread-reservation=1
+|  06:AGGREGATE [STREAMING]
+|  |  output: sum(ss_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
 |  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=27 row-size=84B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT)
+|  |  tuple-ids=3 row-size=84B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
 |  |
-|  19:HASH JOIN [INNER JOIN, PARTITIONED]
+|  05:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=04
-|  |  hash predicates: ws_bill_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=25,26,24 row-size=88B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT), 15(OPEN)
+|  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=1,2,0 row-size=88B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT), 01(OPEN)
 |  |
-|  |--F29:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
-|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |--F30:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|  |  |  Per-Instance Resources: mem-estimate=10.43MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=03
 |  |  |  build expressions: c_customer_sk
 |  |  |  runtime filters: RF014[bloom] <- c_customer_sk, RF015[min_max] <- c_customer_sk
-|  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  |  |
 |  |  45:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=6.55MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=24 row-size=68B cardinality=100.00K
-|  |  |  in pipelines: 15(GETNEXT)
+|  |  |  tuple-ids=0 row-size=68B cardinality=100.00K
+|  |  |  in pipelines: 01(GETNEXT)
 |  |  |
 |  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
@@ -1067,41 +1074,41 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|  |     tuple-ids=24 row-size=68B cardinality=100.00K
-|  |     in pipelines: 15(GETNEXT)
+|  |     tuple-ids=0 row-size=68B cardinality=100.00K
+|  |     in pipelines: 01(GETNEXT)
 |  |
-|  44:EXCHANGE [HASH(ws_bill_customer_sk)]
-|  |  mem-estimate=1.47MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT)
+|  44:EXCHANGE [HASH(ss_customer_sk)]
+|  |  mem-estimate=3.91MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT)
 |  |
-|  F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
 |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
-|  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
-|  18:HASH JOIN [INNER JOIN, BROADCAST]
+|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1
+|  04:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=05
-|  |  hash predicates: ws_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
-|  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
+|  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |
-|  |--F30:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |--F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=03
 |  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF016[bloom] <- d_date_sk, RF017[min_max] <- d_date_sk
+|  |  |  runtime filters: RF016[bloom] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  43:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=26 row-size=8B cardinality=373
-|  |  |  in pipelines: 17(GETNEXT)
+|  |  |  tuple-ids=2 row-size=8B cardinality=373
+|  |  |  in pipelines: 03(GETNEXT)
 |  |  |
 |  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     stored statistics:
@@ -1111,95 +1118,103 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=26 row-size=8B cardinality=373
-|  |     in pipelines: 17(GETNEXT)
+|  |     tuple-ids=2 row-size=8B cardinality=373
+|  |     in pipelines: 03(GETNEXT)
 |  |
-|  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
-|     HDFS partitions=1/1 files=2 size=45.09MB
-|     runtime filters: RF017[min_max] -> ws_sold_date_sk, RF015[min_max] -> ws_bill_customer_sk, RF016[bloom] -> ws_sold_date_sk, RF014[bloom] -> ws_bill_customer_sk
+|  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk, RF014[bloom] -> ss_customer_sk
 |     stored statistics:
-|       table: rows=719.38K size=45.09MB
+|       table: rows=2.88M size=200.95MB
+|       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=644.77K
-|     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=25 row-size=12B cardinality=719.38K
-|     in pipelines: 16(GETNEXT)
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|     tuple-ids=1 row-size=12B cardinality=2.88M
+|     in pipelines: 02(GETNEXT)
+|
+48:EXCHANGE [HASH(customer_id)]
+|  mem-estimate=2.49MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=18,28 row-size=124B cardinality=43.59K
+|  in pipelines: 36(GETNEXT)
 |
+F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-reservation=1
 28:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=06
 |  hash predicates: customer_id = customer_id
 |  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=18,8 row-size=124B cardinality=589.03K
+|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=18,28 row-size=124B cardinality=43.59K
 |  in pipelines: 36(GETNEXT), 41(OPEN)
 |
-|--F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=20.75MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F32:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=7.46MB mem-reservation=6.75MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=06 plan-id=07 cohort-id=01
 |  |  build expressions: customer_id
 |  |  runtime filters: RF004[bloom] <- customer_id, RF005[min_max] <- customer_id
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=512.00KB thread-reservation=0
+|  |  mem-estimate=5.75MB mem-reservation=5.75MB spill-buffer=128.00KB thread-reservation=0
 |  |
 |  42:EXCHANGE [BROADCAST]
-|  |  mem-estimate=2.75MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=44B cardinality=58.90K
+|  |  mem-estimate=731.94KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=28 row-size=44B cardinality=14.80K
 |  |  in pipelines: 41(GETNEXT)
 |  |
-|  F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-reservation=1
-|  00:UNION
+|  F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Instance Resources: mem-estimate=16.10MB mem-reservation=8.50MB thread-reservation=1
+|  14:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=44B cardinality=58.90K
+|  |  tuple-ids=28 row-size=44B cardinality=14.80K
 |  |  in pipelines: 41(GETNEXT)
 |  |
 |  41:AGGREGATE [FINALIZE]
-|  |  output: sum:merge(ss_net_paid)
+|  |  output: sum:merge(ws_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
-|  |  having: sum(ss_net_paid) > CAST(0 AS DECIMAL(3,0))
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=3 row-size=84B cardinality=58.90K
-|  |  in pipelines: 41(GETNEXT), 02(OPEN)
+|  |  having: sum(ws_net_paid) > CAST(0 AS DECIMAL(3,0))
+|  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  tuple-ids=27 row-size=84B cardinality=14.80K
+|  |  in pipelines: 41(GETNEXT), 16(OPEN)
 |  |
 |  40:EXCHANGE [HASH(c_customer_id,c_first_name,c_last_name,d_year)]
-|  |  mem-estimate=10.52MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=84B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  mem-estimate=6.10MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=27 row-size=84B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT)
 |  |
-|  F09:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=13.91MB mem-reservation=9.00MB thread-reservation=1
-|  06:AGGREGATE [STREAMING]
-|  |  output: sum(ss_net_paid)
+|  F09:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  Per-Instance Resources: mem-estimate=11.47MB mem-reservation=9.00MB thread-reservation=1
+|  20:AGGREGATE [STREAMING]
+|  |  output: sum(ws_net_paid)
 |  |  group by: c_customer_id, c_first_name, c_last_name, d_year
 |  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=3 row-size=84B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  |  tuple-ids=27 row-size=84B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT)
 |  |
-|  05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  19:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=07
-|  |  hash predicates: ss_customer_sk = c_customer_sk
-|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=1,2,0 row-size=88B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 01(OPEN)
+|  |  hash predicates: ws_bill_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
+|  |  tuple-ids=25,26,24 row-size=88B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT), 15(OPEN)
 |  |
-|  |--F32:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-|  |  |  Per-Instance Resources: mem-estimate=10.43MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |--F33:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=04
 |  |  |  build expressions: c_customer_sk
 |  |  |  runtime filters: RF010[bloom] <- c_customer_sk, RF011[min_max] <- c_customer_sk
-|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |  |
 |  |  39:EXCHANGE [HASH(c_customer_sk)]
 |  |  |  mem-estimate=6.55MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0 row-size=68B cardinality=100.00K
-|  |  |  in pipelines: 01(GETNEXT)
+|  |  |  tuple-ids=24 row-size=68B cardinality=100.00K
+|  |  |  in pipelines: 15(GETNEXT)
 |  |  |
 |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  01:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  15:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
 |  |     stored statistics:
@@ -1207,41 +1222,41 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|  |     tuple-ids=0 row-size=68B cardinality=100.00K
-|  |     in pipelines: 01(GETNEXT)
+|  |     tuple-ids=24 row-size=68B cardinality=100.00K
+|  |     in pipelines: 15(GETNEXT)
 |  |
-|  38:EXCHANGE [HASH(ss_customer_sk)]
-|  |  mem-estimate=3.91MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT)
+|  38:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  mem-estimate=1.47MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT)
 |  |
-|  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
-|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=1
-|  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
+|  18:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=08
-|  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=1,2 row-size=20B cardinality=589.03K
-|  |  in pipelines: 02(GETNEXT), 03(OPEN)
+|  |  tuple-ids=25,26 row-size=20B cardinality=148.00K
+|  |  in pipelines: 16(GETNEXT), 17(OPEN)
 |  |
-|  |--F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |--F34:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=08 plan-id=09 cohort-id=04
 |  |  |  build expressions: d_date_sk
-|  |  |  runtime filters: RF012[bloom] <- d_date_sk
+|  |  |  runtime filters: RF012[bloom] <- d_date_sk, RF013[min_max] <- d_date_sk
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  37:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=8B cardinality=373
-|  |  |  in pipelines: 03(GETNEXT)
+|  |  |  tuple-ids=26 row-size=8B cardinality=373
+|  |  |  in pipelines: 17(GETNEXT)
 |  |  |
 |  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  17:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     stored statistics:
@@ -1251,20 +1266,19 @@ Per-Instance Resources: mem-estimate=27.52MB mem-reservation=17.00MB thread-rese
 |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2001 AS INT)
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=2 row-size=8B cardinality=373
-|  |     in pipelines: 03(GETNEXT)
+|  |     tuple-ids=26 row-size=8B cardinality=373
+|  |     in pipelines: 17(GETNEXT)
 |  |
-|  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|     runtime filters: RF011[min_max] -> ss_customer_sk, RF012[bloom] -> ss_sold_date_sk, RF010[bloom] -> ss_customer_sk
+|  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|     HDFS partitions=1/1 files=2 size=45.09MB
+|     runtime filters: RF013[min_max] -> ws_sold_date_sk, RF011[min_max] -> ws_bill_customer_sk, RF012[bloom] -> ws_sold_date_sk, RF010[bloom] -> ws_bill_customer_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.96MB
-|       partitions: 1824/1824 rows=2.88M
+|       table: rows=719.38K size=45.09MB
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
-|     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=1 row-size=12B cardinality=2.88M
-|     in pipelines: 02(GETNEXT)
+|     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
+|     tuple-ids=25 row-size=12B cardinality=719.38K
+|     in pipelines: 16(GETNEXT)
 |
 07:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -1300,7 +1314,7 @@ Per-Instance Resources: mem-estimate=13.91MB mem-reservation=9.00MB thread-reser
 |  tuple-ids=11,12,10 row-size=88B cardinality=589.03K
 |  in pipelines: 09(GETNEXT), 08(OPEN)
 |
-|--F34:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
+|--F35:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
 |  |  Per-Instance Resources: mem-estimate=10.43MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=09 plan-id=10 cohort-id=01
@@ -1318,7 +1332,7 @@ Per-Instance Resources: mem-estimate=13.91MB mem-reservation=9.00MB thread-reser
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1
 |  08:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=5.49MB
-|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id
+|     runtime filters: RF005[min_max] -> tpcds_parquet.customer.c_customer_id, RF001[min_max] -> tpcds_parquet.customer.c_customer_id, RF003[min_max] -> tpcds_parquet.customer.c_customer_id, RF004[bloom] -> tpcds_parquet.customer.c_customer_id, RF000[bloom] -> tpcds_parquet.customer.c_customer_id, RF002[bloom] -> tpcds_parquet.customer.c_customer_id
 |     stored statistics:
 |       table: rows=100.00K size=5.49MB
 |       columns: all
@@ -1343,7 +1357,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser
 |  tuple-ids=11,12 row-size=20B cardinality=589.03K
 |  in pipelines: 09(GETNEXT), 10(OPEN)
 |
-|--F35:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|--F36:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=10 plan-id=11 cohort-id=01
@@ -1372,10 +1386,10 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=1.00MB thread-reser
 |     in pipelines: 10(GETNEXT)
 |
 09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=200.95MB
    runtime filters: RF007[min_max] -> ss_customer_sk, RF008[bloom] -> ss_sold_date_sk, RF006[bloom] -> ss_customer_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/views.test b/testdata/workloads/functional-planner/queries/PlannerTest/views.test
index 7a9b5b0..b340626 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/views.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/views.test
@@ -163,121 +163,119 @@ functional.complex_view t3 where t1.id = t2.x and t2.x = t3.abc
 PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN]
-|  hash predicates: int_col = count(a.bigint_col)
-|  runtime filters: RF000 <- count(a.bigint_col)
+|  hash predicates: int_col = functional.alltypes.id
+|  runtime filters: RF000 <- functional.alltypes.id
 |  row-size=143B cardinality=730
 |
-|--06:TOP-N [LIMIT=100]
-|  |  order by: b.string_col ASC
-|  |  row-size=21B cardinality=1
-|  |
-|  05:AGGREGATE [FINALIZE]
-|  |  output: count(a.bigint_col)
-|  |  group by: b.string_col
-|  |  having: count(a.bigint_col) > 1
-|  |  row-size=21B cardinality=1
-|  |
-|  04:HASH JOIN [INNER JOIN]
-|  |  hash predicates: a.id = b.id
-|  |  runtime filters: RF004 <- b.id
-|  |  row-size=29B cardinality=1
+|--07:HASH JOIN [INNER JOIN]
+|  |  hash predicates: functional.alltypes.id = count(a.bigint_col)
+|  |  runtime filters: RF002 <- count(a.bigint_col)
+|  |  row-size=110B cardinality=1
 |  |
-|  |--03:SCAN HDFS [functional.alltypestiny b]
-|  |     HDFS partitions=4/4 files=4 size=460B
-|  |     row-size=17B cardinality=8
+|  |--06:TOP-N [LIMIT=100]
+|  |  |  order by: b.string_col ASC
+|  |  |  row-size=21B cardinality=1
+|  |  |
+|  |  05:AGGREGATE [FINALIZE]
+|  |  |  output: count(a.bigint_col)
+|  |  |  group by: b.string_col
+|  |  |  having: count(a.bigint_col) > 1
+|  |  |  row-size=21B cardinality=1
+|  |  |
+|  |  04:HASH JOIN [INNER JOIN]
+|  |  |  hash predicates: a.id = b.id
+|  |  |  runtime filters: RF004 <- b.id
+|  |  |  row-size=29B cardinality=1
+|  |  |
+|  |  |--03:SCAN HDFS [functional.alltypestiny b]
+|  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |     row-size=17B cardinality=8
+|  |  |
+|  |  02:SCAN HDFS [functional.alltypesagg a]
+|  |     HDFS partitions=11/11 files=11 size=814.73KB
+|  |     predicates: a.bigint_col < 50
+|  |     runtime filters: RF004 -> a.id
+|  |     row-size=12B cardinality=1.10K
 |  |
-|  02:SCAN HDFS [functional.alltypesagg a]
-|     HDFS partitions=11/11 files=11 size=814.73KB
-|     predicates: a.bigint_col < 50
-|     runtime filters: RF004 -> a.id
-|     row-size=12B cardinality=1.10K
-|
-07:HASH JOIN [INNER JOIN]
-|  hash predicates: functional.alltypes.id = int_col
-|  runtime filters: RF002 <- int_col
-|  row-size=122B cardinality=730
-|
-|--01:SCAN HDFS [functional.alltypes]
+|  00:SCAN HDFS [functional.alltypes]
 |     HDFS partitions=24/24 files=24 size=478.45KB
-|     predicates: functional.alltypes.int_col > 1
-|     runtime filters: RF000 -> int_col
-|     row-size=33B cardinality=730
+|     predicates: functional.alltypes.id > 1
+|     runtime filters: RF002 -> functional.alltypes.id
+|     row-size=89B cardinality=730
 |
-00:SCAN HDFS [functional.alltypes]
+01:SCAN HDFS [functional.alltypes]
    HDFS partitions=24/24 files=24 size=478.45KB
-   predicates: functional.alltypes.id > 1
-   runtime filters: RF000 -> functional.alltypes.id, RF002 -> functional.alltypes.id
-   row-size=89B cardinality=730
+   predicates: functional.alltypes.int_col > 1
+   runtime filters: RF000 -> int_col
+   row-size=33B cardinality=730
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
-16:EXCHANGE [UNPARTITIONED]
+15:EXCHANGE [UNPARTITIONED]
 |
-08:HASH JOIN [INNER JOIN, PARTITIONED]
-|  hash predicates: int_col = count(a.bigint_col)
-|  runtime filters: RF000 <- count(a.bigint_col)
+08:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: int_col = functional.alltypes.id
+|  runtime filters: RF000 <- functional.alltypes.id
 |  row-size=143B cardinality=730
 |
-|--15:EXCHANGE [HASH(count(a.bigint_col))]
-|  |
-|  14:MERGING-EXCHANGE [UNPARTITIONED]
-|  |  order by: b.string_col ASC
-|  |  limit: 100
-|  |
-|  06:TOP-N [LIMIT=100]
-|  |  order by: b.string_col ASC
-|  |  row-size=21B cardinality=1
+|--14:EXCHANGE [BROADCAST]
 |  |
-|  13:AGGREGATE [FINALIZE]
-|  |  output: count:merge(a.bigint_col)
-|  |  group by: b.string_col
-|  |  having: count(a.bigint_col) > 1
-|  |  row-size=21B cardinality=1
+|  07:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: functional.alltypes.id = count(a.bigint_col)
+|  |  runtime filters: RF002 <- count(a.bigint_col)
+|  |  row-size=110B cardinality=1
 |  |
-|  12:EXCHANGE [HASH(b.string_col)]
-|  |
-|  05:AGGREGATE [STREAMING]
-|  |  output: count(a.bigint_col)
-|  |  group by: b.string_col
-|  |  row-size=21B cardinality=1
-|  |
-|  04:HASH JOIN [INNER JOIN, BROADCAST]
-|  |  hash predicates: a.id = b.id
-|  |  runtime filters: RF004 <- b.id
-|  |  row-size=29B cardinality=1
-|  |
-|  |--11:EXCHANGE [BROADCAST]
+|  |--13:EXCHANGE [BROADCAST]
 |  |  |
-|  |  03:SCAN HDFS [functional.alltypestiny b]
-|  |     HDFS partitions=4/4 files=4 size=460B
-|  |     row-size=17B cardinality=8
-|  |
-|  02:SCAN HDFS [functional.alltypesagg a]
-|     HDFS partitions=11/11 files=11 size=814.73KB
-|     predicates: a.bigint_col < 50
-|     runtime filters: RF004 -> a.id
-|     row-size=12B cardinality=1.10K
-|
-07:HASH JOIN [INNER JOIN, PARTITIONED]
-|  hash predicates: functional.alltypes.id = int_col
-|  runtime filters: RF002 <- int_col
-|  row-size=122B cardinality=730
-|
-|--10:EXCHANGE [HASH(int_col)]
+|  |  12:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  |  order by: b.string_col ASC
+|  |  |  limit: 100
+|  |  |
+|  |  06:TOP-N [LIMIT=100]
+|  |  |  order by: b.string_col ASC
+|  |  |  row-size=21B cardinality=1
+|  |  |
+|  |  11:AGGREGATE [FINALIZE]
+|  |  |  output: count:merge(a.bigint_col)
+|  |  |  group by: b.string_col
+|  |  |  having: count(a.bigint_col) > 1
+|  |  |  row-size=21B cardinality=1
+|  |  |
+|  |  10:EXCHANGE [HASH(b.string_col)]
+|  |  |
+|  |  05:AGGREGATE [STREAMING]
+|  |  |  output: count(a.bigint_col)
+|  |  |  group by: b.string_col
+|  |  |  row-size=21B cardinality=1
+|  |  |
+|  |  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash predicates: a.id = b.id
+|  |  |  runtime filters: RF004 <- b.id
+|  |  |  row-size=29B cardinality=1
+|  |  |
+|  |  |--09:EXCHANGE [BROADCAST]
+|  |  |  |
+|  |  |  03:SCAN HDFS [functional.alltypestiny b]
+|  |  |     HDFS partitions=4/4 files=4 size=460B
+|  |  |     row-size=17B cardinality=8
+|  |  |
+|  |  02:SCAN HDFS [functional.alltypesagg a]
+|  |     HDFS partitions=11/11 files=11 size=814.73KB
+|  |     predicates: a.bigint_col < 50
+|  |     runtime filters: RF004 -> a.id
+|  |     row-size=12B cardinality=1.10K
 |  |
-|  01:SCAN HDFS [functional.alltypes]
+|  00:SCAN HDFS [functional.alltypes]
 |     HDFS partitions=24/24 files=24 size=478.45KB
-|     predicates: functional.alltypes.int_col > 1
-|     runtime filters: RF000 -> int_col
-|     row-size=33B cardinality=730
-|
-09:EXCHANGE [HASH(functional.alltypes.id)]
+|     predicates: functional.alltypes.id > 1
+|     runtime filters: RF002 -> functional.alltypes.id
+|     row-size=89B cardinality=730
 |
-00:SCAN HDFS [functional.alltypes]
+01:SCAN HDFS [functional.alltypes]
    HDFS partitions=24/24 files=24 size=478.45KB
-   predicates: functional.alltypes.id > 1
-   runtime filters: RF000 -> functional.alltypes.id, RF002 -> functional.alltypes.id
-   row-size=89B cardinality=730
+   predicates: functional.alltypes.int_col > 1
+   runtime filters: RF000 -> int_col
+   row-size=33B cardinality=730
 ====
 # Self-join of view to make sure the on clause is properly set
 # in the cloned view instances.