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:43 UTC

[impala] branch branch-4.0.1 updated (1950394 -> 80a1dc3)

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

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


    from 1950394  Prepare 4.0.0 RC7
     new ef7de39  IMPALA-10828: Impala 4.0 should pin to a specifit toolchain commit when building on ARM
     new b3a3149  IMPALA-10681: Improve inner join cardinality estimates
     new 8862719  IMPALA-7560: Set selectivity of Not-equal
     new 849ecb9  IMPALA-10810: Bump json-smart from 2.3 to 2.4.7
     new cb604bb  IMPALA-10825: fix impalad crashes when closing the retrying query
     new eb595e5  IMPALA-5476: Fix catalogd restart brings stale metadata
     new 80a1dc3  IMPALA-10714: Defer advancing read page until the buffer is attached

The 7 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/runtime/buffered-tuple-stream-test.cc       |  229 +-
 be/src/runtime/buffered-tuple-stream.cc            |   30 +-
 be/src/runtime/buffered-tuple-stream.h             |    4 +-
 be/src/runtime/query-driver.cc                     |   13 +-
 be/src/service/impala-server.cc                    |   59 +-
 bin/bootstrap_system.sh                            |    4 +-
 fe/pom.xml                                         |    4 +-
 .../apache/impala/analysis/BinaryPredicate.java    |   41 +-
 .../main/java/org/apache/impala/analysis/Expr.java |   15 +
 .../apache/impala/analysis/FunctionCallExpr.java   |    6 +
 .../java/org/apache/impala/planner/JoinNode.java   |  199 +-
 .../impala/analysis/ExprCardinalityTest.java       |   71 +-
 .../org/apache/impala/planner/CardinalityTest.java |    4 +-
 java/datagenerator/pom.xml                         |    2 +-
 java/executor-deps/pom.xml                         |    4 +-
 java/pom.xml                                       |    1 +
 java/shaded-deps/hive-exec/pom.xml                 |    4 +-
 java/test-hive-udfs/pom.xml                        |    2 +-
 java/yarn-extras/pom.xml                           |    2 +-
 .../queries/PlannerTest/analytic-fns.test          |    4 +-
 .../queries/PlannerTest/card-inner-join.test       |  136 ++
 .../queries/PlannerTest/card-scan.test             |    4 +-
 .../queries/PlannerTest/hbase.test                 |    8 +-
 .../queries/PlannerTest/inline-view-limit.test     |    4 +-
 .../queries/PlannerTest/join-order.test            |  294 +--
 .../queries/PlannerTest/joins.test                 |    8 +-
 .../queries/PlannerTest/kudu.test                  |    4 +-
 .../PlannerTest/partition-key-scans-default.test   |    2 +-
 .../queries/PlannerTest/partition-key-scans.test   |    2 +-
 .../queries/PlannerTest/predicate-propagation.test |    8 +-
 .../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/tpch-all.test              |   50 +-
 .../queries/PlannerTest/tpch-kudu.test             |    2 +-
 .../queries/PlannerTest/tpch-nested.test           |    8 +-
 .../queries/PlannerTest/tpch-views.test            |    2 +-
 .../queries/PlannerTest/views.test                 |  188 +-
 tests/custom_cluster/test_query_retries.py         |   26 +
 tests/custom_cluster/test_restart_services.py      |  104 +
 43 files changed, 4096 insertions(+), 3320 deletions(-)

[impala] 06/07: IMPALA-5476: Fix catalogd restart brings stale metadata

Posted by st...@apache.org.
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 eb595e5c27182a46b7cf110530c2e076b4f77081
Author: liuyao <54...@163.com>
AuthorDate: Sat Jul 3 19:04:58 2021 +0800

    IMPALA-5476: Fix catalogd restart brings stale metadata
    
    ImpaladCatalog#updateCatalog() doesn't trigger a full topic update
    request when detecting catalogServiceId changes. It just updates the
    local catalogServiceId and throws an exception to abort applying the
    DDL/DML results. This causes a problem when catalogd is restarted and
    the DDL/DML is executed on the restarted instance. In this case, only
    the local catalogServiceId is updated to the latest. The local catalog
    remains stale. Then when dealing with the following updates from
    statestore, the catalogServiceId always matches, so updates will be
    applied without exceptions. However, the catalog objects usually won't
    be updated since they have higher versions (from the old catalogd
    instance) than those in the update. This brings the local catalog out
    of sync until the catalog version of the new catalogd grows larger
    enough.
    
    Note that in dealing with the catalog updates from statestore, if the
    catalogServiceId unmatches, impalad will request a full topic update.
    See more in ImpalaServer::CatalogUpdateCallback().
    
    This patch fixes this issue by checking the catalogServiceId before
    invoking UpdateCatalogCache() of FE. If catalogServiceId doesn't match
    the one in the DDL/DML result, wait until it changes. The following
    update from statestore will change it and unblocks the DDL/DML thread.
    
    Testing
    
    add several tests in
    tests/custom_cluster/test_restart_services.py
    
    Change-Id: I9fe25f5a2a42fb432e306ef08ae35750c8f3c50c
    Reviewed-on: http://gerrit.cloudera.org:8080/17645
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/impala-server.cc               |  59 +++++++++++----
 tests/custom_cluster/test_restart_services.py | 104 ++++++++++++++++++++++++++
 2 files changed, 150 insertions(+), 13 deletions(-)

diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index a48f7fe..df5541f 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -2126,19 +2126,52 @@ Status ImpalaServer::ProcessCatalogUpdateResult(
       WaitForCatalogUpdateTopicPropagation(catalog_service_id);
     }
   } else {
-    CatalogUpdateResultIterator callback_ctx(catalog_update_result);
-    TUpdateCatalogCacheRequest update_req;
-    update_req.__set_is_delta(true);
-    update_req.__set_native_iterator_ptr(reinterpret_cast<int64_t>(&callback_ctx));
-    // The catalog version is updated in WaitForCatalogUpdate below. So we need a
-    // standalone field in the request to update the service ID without touching the
-    // catalog version.
-    update_req.__set_catalog_service_id(catalog_update_result.catalog_service_id);
-    // Apply the changes to the local catalog cache.
-    TUpdateCatalogCacheResponse resp;
-    Status status = exec_env_->frontend()->UpdateCatalogCache(update_req, &resp);
-    if (!status.ok()) LOG(ERROR) << status.GetDetail();
-    RETURN_IF_ERROR(status);
+    TUniqueId cur_service_id;
+    {
+      unique_lock<mutex> ver_lock(catalog_version_lock_);
+      cur_service_id = catalog_update_info_.catalog_service_id;
+      if (catalog_update_info_.catalog_service_id != catalog_service_id) {
+        LOG(INFO) << "Catalog service ID mismatch. Current ID: "
+            << PrintId(cur_service_id) << ". ID in response: "
+            << PrintId(catalog_service_id) << ". Catalogd may be restarted. Waiting for"
+            " new catalog update from statestore.";
+        // Catalog service ID has been changed, and impalad request a full topic update.
+        // When impalad completes the full topic update, it will exit this loop.
+        while (cur_service_id == catalog_update_info_.catalog_service_id) {
+          catalog_version_update_cv_.Wait(ver_lock);
+        }
+        cur_service_id = catalog_update_info_.catalog_service_id;
+      }
+    }
+
+    if (cur_service_id == catalog_service_id) {
+      CatalogUpdateResultIterator callback_ctx(catalog_update_result);
+      TUpdateCatalogCacheRequest update_req;
+      update_req.__set_is_delta(true);
+      update_req.__set_native_iterator_ptr(reinterpret_cast<int64_t>(&callback_ctx));
+      // The catalog version is updated in WaitForCatalogUpdate below. So we need a
+      // standalone field in the request to update the service ID without touching the
+      // catalog version.
+      update_req.__set_catalog_service_id(catalog_update_result.catalog_service_id);
+      // Apply the changes to the local catalog cache.
+      TUpdateCatalogCacheResponse resp;
+      Status status = exec_env_->frontend()->UpdateCatalogCache(update_req, &resp);
+      if (!status.ok()) LOG(ERROR) << status.GetDetail();
+      RETURN_IF_ERROR(status);
+    } else {
+      // We can't apply updates on another service id, because the local catalog is still
+      // inconsistent with the catalogd that executes the DDL. Catalogd may be restarted
+      // more than once inside a statestore update cycle. 'cur_service_id' could belong
+      // to 1) a stale update from the previous restarted catalogd, or 2) a newer update
+      // from next restarted catalogd. We are good to ignore the DDL result at the second
+      // case. However, in the first case clients may see stale catalog until the
+      // expected catalog topic update comes.
+      // TODO: handle the first case in IMPALA-10875.
+      LOG(WARNING) << "Ignoring catalog update result of catalog service ID: "
+          << PrintId(catalog_service_id) << ". The expected catalog service ID: "
+          << PrintId(catalog_service_id) << ". Current catalog service ID: "
+          << PrintId(cur_service_id) <<". Catalogd may be restarted more than once.";
+    }
     if (!wait_for_all_subscribers) return Status::OK();
     // Wait until we receive and process the catalog update that covers the effects
     // (catalog objects) of this operation.
diff --git a/tests/custom_cluster/test_restart_services.py b/tests/custom_cluster/test_restart_services.py
index 771c686..beccc27 100644
--- a/tests/custom_cluster/test_restart_services.py
+++ b/tests/custom_cluster/test_restart_services.py
@@ -163,6 +163,110 @@ class TestRestart(CustomClusterTestSuite):
     thread.join()
     self.wait_for_state(query_handle[0], QueryState.FINISHED, 30000)
 
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    statestored_args="--statestore_update_frequency_ms=5000")
+  def test_restart_catalogd(self):
+    self.execute_query_expect_success(self.client, "drop table if exists join_aa")
+    self.execute_query_expect_success(self.client, "create table join_aa(id int)")
+    # Make the catalog object version grow large enough
+    self.execute_query_expect_success(self.client, "invalidate metadata")
+
+    # No need to care whether the dll is executed successfully, it is just to make
+    # the local catalog catche of impalad out of sync
+    for i in range(0, 10):
+      try:
+        query = "alter table join_aa add columns (age" + str(i) + " int)"
+        self.execute_query_async(query)
+      except Exception, e:
+        LOG.info(str(e))
+      if i == 5:
+        self.cluster.catalogd.restart()
+
+    self.execute_query_expect_success(self.client,
+        "alter table join_aa add columns (name string)")
+    self.execute_query_expect_success(self.client, "select name from join_aa")
+    self.execute_query_expect_success(self.client, "drop table join_aa")
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    statestored_args="--statestore_update_frequency_ms=5000")
+  def test_restart_catalogd_sync_ddl(self):
+    self.execute_query_expect_success(self.client, "drop table if exists join_aa")
+    self.execute_query_expect_success(self.client, "create table join_aa(id int)")
+    # Make the catalog object version grow large enough
+    self.execute_query_expect_success(self.client, "invalidate metadata")
+    query_options = {"sync_ddl": "true"}
+
+    # No need to care whether the dll is executed successfully, it is just to make
+    # the local catalog catche of impalad out of sync
+    for i in range(0, 10):
+      try:
+        query = "alter table join_aa add columns (age" + str(i) + " int)"
+        self.execute_query_async(query, query_options)
+      except Exception, e:
+        LOG.info(str(e))
+      if i == 5:
+        self.cluster.catalogd.restart()
+
+    self.execute_query_expect_success(self.client,
+        "alter table join_aa add columns (name string)", query_options)
+    self.execute_query_expect_success(self.client, "select name from join_aa")
+    self.execute_query_expect_success(self.client, "drop table join_aa")
+
+  UPDATE_FREQUENCY_S = 10
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    statestored_args="--statestore_update_frequency_ms={frequency_ms}"
+    .format(frequency_ms=(UPDATE_FREQUENCY_S * 1000)))
+  def test_restart_catalogd_twice(self):
+    self.execute_query_expect_success(self.client, "drop table if exists join_aa")
+    self.cluster.catalogd.restart()
+    query = "create table join_aa(id int)"
+    query_handle = []
+
+    def execute_query_async():
+      query_handle.append(self.execute_query(query))
+
+    thread = threading.Thread(target=execute_query_async)
+    thread.start()
+    sleep(self.UPDATE_FREQUENCY_S - 5)
+    self.cluster.catalogd.restart()
+    thread.join()
+    self.execute_query_expect_success(self.client,
+        "alter table join_aa add columns (name string)")
+    self.execute_query_expect_success(self.client, "select name from join_aa")
+    self.execute_query_expect_success(self.client, "drop table join_aa")
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+      impalad_args="--use_local_catalog=true",
+      catalogd_args="--catalog_topic_mode=minimal",
+      statestored_args="--statestore_update_frequency_ms=5000")
+  def test_restart_catalogd_with_local_catalog(self):
+    self.execute_query_expect_success(self.client, "drop table if exists join_aa")
+    self.execute_query_expect_success(self.client, "create table join_aa(id int)")
+    # Make the catalog object version grow large enough
+    self.execute_query_expect_success(self.client, "invalidate metadata")
+
+    # No need to care whether the dll is executed successfully, it is just to make
+    # the local catalog catche of impalad out of sync
+    for i in range(0, 10):
+      try:
+        query = "alter table join_aa add columns (age" + str(i) + " int)"
+        self.execute_query_async(query)
+      except Exception, e:
+        LOG.info(str(e))
+      if i == 5:
+        self.cluster.catalogd.restart()
+
+    self.execute_query_expect_success(self.client,
+        "alter table join_aa add columns (name string)")
+    self.execute_query_expect_success(self.client, "select name from join_aa")
+    self.execute_query_expect_success(self.client, "select age0 from join_aa")
+    self.execute_query_expect_success(self.client, "drop table join_aa")
+
   SUBSCRIBER_TIMEOUT_S = 2
   CANCELLATION_GRACE_PERIOD_S = 5
 

[impala] 01/07: IMPALA-10828: Impala 4.0 should pin to a specifit toolchain commit when building on ARM

Posted by st...@apache.org.
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 ef7de393325a5e9343e24e90f565697b03f25aac
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Fri Sep 24 10:43:46 2021 +0800

    IMPALA-10828: Impala 4.0 should pin to a specifit toolchain commit when building on ARM
    
    While building on ARM platform, bin/bootstrap_system.sh will compile
    native-toolchain using the latest commit, which will cause build
    failures if we are building branches other than master.
    
    This patch addes a branch name (asf-impala-4.0) when cloning the
    native-toolchain repo. So we can build Impala 4.0 on ARM without
    manually modifying bin/bootstrap_system.sh.
    
    Tests:
     - Tested on ARM platform.
    
    Change-Id: Id353e0132b684a338e2c5454331be9bff9a8be14
---
 bin/bootstrap_system.sh | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/bin/bootstrap_system.sh b/bin/bootstrap_system.sh
index ec7919f..dc6db31 100755
--- a/bin/bootstrap_system.sh
+++ b/bin/bootstrap_system.sh
@@ -493,8 +493,8 @@ if [[ $ARCH_NAME == 'aarch64' ]]; then
   echo -e "\n$SET_TOOLCHAIN_HOME" >> "${IMPALA_HOME}/bin/impala-config-local.sh"
   eval "$SET_TOOLCHAIN_HOME"
   if ! [[ -d "$NATIVE_TOOLCHAIN_HOME" ]]; then
-    time -p git clone https://github.com/cloudera/native-toolchain/ \
-      "$NATIVE_TOOLCHAIN_HOME"
+    time -p git clone --single-branch -b asf-impala-4.0 \
+      https://github.com/cloudera/native-toolchain/ "$NATIVE_TOOLCHAIN_HOME"
   fi
   cd "$NATIVE_TOOLCHAIN_HOME"
   git pull

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

Posted by st...@apache.org.
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.

[impala] 03/07: IMPALA-7560: Set selectivity of Not-equal

Posted by st...@apache.org.
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 8862719d87ac5dc214985025463f002d41b15672
Author: liuyao <54...@163.com>
AuthorDate: Mon Apr 26 14:37:43 2021 +0800

    IMPALA-7560: Set selectivity of Not-equal
    
    Calculate binary predicate selectivity if one of the children is
    a slotref and the other children are all constant.
    eg. something like "col != 5", but not "2 * col != 10"
    
    selectivity = 1 - 1/ndv
    
    Testing:
    Modify the function testNeSelectivity() of the
    ExprCardinalityTest.java, change -1 to the correct value.
    
    Change-Id: Icd6f5945840ea2a8194d72aa440ddfa6915cbb3a
    Reviewed-on: http://gerrit.cloudera.org:8080/17344
    Reviewed-by: Qifan Chen <qc...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Zoltan Borok-Nagy <bo...@cloudera.com>
---
 .../apache/impala/analysis/BinaryPredicate.java    | 41 +++++++++++--
 .../impala/analysis/ExprCardinalityTest.java       | 71 +++++++++-------------
 .../org/apache/impala/planner/CardinalityTest.java |  4 +-
 .../queries/PlannerTest/analytic-fns.test          |  4 +-
 .../queries/PlannerTest/card-scan.test             |  4 +-
 .../queries/PlannerTest/hbase.test                 |  8 +--
 .../queries/PlannerTest/inline-view-limit.test     |  4 +-
 .../queries/PlannerTest/kudu.test                  |  4 +-
 .../queries/PlannerTest/predicate-propagation.test |  8 +--
 .../queries/PlannerTest/tpch-all.test              | 50 +++++++--------
 .../queries/PlannerTest/tpch-kudu.test             |  2 +-
 .../queries/PlannerTest/tpch-nested.test           |  8 +--
 .../queries/PlannerTest/tpch-views.test            |  2 +-
 13 files changed, 110 insertions(+), 100 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java b/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java
index 9092058..83ab86c 100644
--- a/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java
+++ b/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.impala.catalog.Db;
+import org.apache.impala.catalog.FeTable;
 import org.apache.impala.catalog.Function.CompareMode;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.Type;
@@ -237,14 +238,42 @@ public class BinaryPredicate extends Predicate {
     // TODO: Compute selectivity for nested predicates.
     // TODO: Improve estimation using histograms.
     Reference<SlotRef> slotRefRef = new Reference<SlotRef>();
-    if ((op_ == Operator.EQ || op_ == Operator.NOT_DISTINCT)
-        && isSingleColumnPredicate(slotRefRef, null)) {
-      long distinctValues = slotRefRef.getRef().getNumDistinctValues();
-      if (distinctValues > 0) {
-        selectivity_ = 1.0 / distinctValues;
-        selectivity_ = Math.max(0, Math.min(1, selectivity_));
+    if (!isSingleColumnPredicate(slotRefRef, null)) {
+      return;
+    }
+    long distinctValues = slotRefRef.getRef().getNumDistinctValues();
+    if (distinctValues < 0) {
+      // Lack of statistics to estimate the selectivity.
+      return;
+    } else if (distinctValues == 0 && (op_ == Operator.EQ || op_ == Operator.NE)) {
+      // If the table is empty, then distinctValues is 0. This case is equivalent
+      // to selectivity is 0.
+      selectivity_ = 0.0;
+      return;
+    }
+
+    if (op_ == Operator.EQ || op_ == Operator.NOT_DISTINCT) {
+      selectivity_ = 1.0 / distinctValues;
+    } else if (op_ == Operator.NE || op_ == Operator.DISTINCT_FROM) {
+      // For case <column> IS DISTINCT FROM NULL, all non-null values are true
+      if (Expr.IS_NULL_LITERAL.apply(getChild(1)) && op_ == Operator.DISTINCT_FROM) {
+        selectivity_ = 1.0;
+      } else {
+        selectivity_ = 1.0 - 1.0 / distinctValues;
       }
+      SlotDescriptor slotDesc = slotRefRef.getRef().getDesc();
+      if (slotDesc.getStats().hasNullsStats()) {
+        FeTable table = slotDesc.getParent().getTable();
+        if (table != null && table.getNumRows() > 0) {
+          long numRows = table.getNumRows();
+          selectivity_ *=
+              (double) (numRows - slotDesc.getStats().getNumNulls()) / numRows;
+        }
+      }
+    } else {
+      return;
     }
+    selectivity_ = Math.max(0, Math.min(1, selectivity_));
   }
 
   @Override
diff --git a/fe/src/test/java/org/apache/impala/analysis/ExprCardinalityTest.java b/fe/src/test/java/org/apache/impala/analysis/ExprCardinalityTest.java
index 061e4a9..efd0378 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ExprCardinalityTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ExprCardinalityTest.java
@@ -314,23 +314,16 @@ public class ExprCardinalityTest {
    */
   @Test
   public void testNeSelectivity() throws ImpalaException {
-    // Bug: No estimated selectivity for != (IMPALA-8039)
-    //verifySelectExpr("alltypes", "id != 10", 3, 1 - 1.0/7300);
-    verifySelectExpr("alltypes", "id != 10", 3, -1);
-    //verifySelectExpr("alltypes", "bool_col != true", 3, 1 - 1.0/2);
-    verifySelectExpr("alltypes", "bool_col != true", 3, -1);
-    //verifySelectExpr("alltypes", "int_col != 10", 3, 1 - 1.0/10);
-    verifySelectExpr("alltypes", "int_col != 10", 3, -1);
-
-    //verifySelectExpr("nullrows", "id != 'foo'", 3, 1 - 1.0/26);
-    verifySelectExpr("nullrows", "id != 'foo'", 3, -1);
-    // Bug: All nulls, so NDV should = 1, so Sel should be 1 - 1.0/1
-    //verifySelectExpr("nullrows", "null_str != 'foo'", 3, 1 - 1.0/1);
-    verifySelectExpr("nullrows", "null_str != 'foo'", 3, -1);
-    //verifySelectExpr("nullrows", "group_str != 'foo'", 3, 1 - 1.0/6);
-    verifySelectExpr("nullrows", "group_str != 'foo'", 3, -1);
-    //verifySelectExpr("nullrows", "some_nulls != 'foo'", 3, 1 - 1.0/7);
-    verifySelectExpr("nullrows", "some_nulls != 'foo'", 3, -1);
+    verifySelectExpr("alltypes", "id != 10", 3, 1 - 1.0/7300);
+    verifySelectExpr("alltypes", "bool_col != true", 3, 1 - 1.0/2);
+    verifySelectExpr("alltypes", "int_col != 10", 3, 1 - 1.0/10);
+    verifySelectExpr("nullrows", "id != 'foo'", 3, 1 - 1.0/26);
+    verifySelectExpr("nullrows", "null_str != 'foo'", 3, 1 - 1.0/1);
+    verifySelectExpr("nullrows", "group_str != 'foo'", 3, 1 - 1.0/6);
+    verifySelectExpr("nullrows", "some_nulls != 'foo'", 3, (1 - 1.0/6)*6/26);
+    // field has no statistics.
+    verifySelectExpr("emptytable", "field != 'foo'", 3, -1);
+    verifySelectExpr("emptytable", "f2 != 10", 3, 0.0);
 
     // Bug: Sel should default to 1 - good old 0.1
     verifySelectExpr("manynulls", "id != 10", 3, -1);
@@ -343,36 +336,28 @@ public class ExprCardinalityTest {
    */
   @Test
   public void testDistinctSelectivity() throws ImpalaException {
-    // BUG: IS DISTINCT has no selectivity
-    //verifySelectExpr("alltypes", "id is distinct from 10", 3, 1 - 1.0/7300);
-    verifySelectExpr("alltypes", "id is distinct from 10", 3, -1);
+
+    verifySelectExpr("alltypes", "id is distinct from 10", 3, 1 - 1.0/7300);
     // Bug: does not treat NULL specially
     // Bug: NDV sould be 2 since IS DISTINCT won't return NULL
     //verifySelectExpr("alltypes", "id is distinct from null", 2, 1);
-    verifySelectExpr("alltypes", "id is distinct from null", 3, -1);
-    //verifySelectExpr("alltypes", "bool_col is distinct from true", 3, 1 - 1.0/2);
-    verifySelectExpr("alltypes", "bool_col is distinct from true", 3, -1);
-    //verifySelectExpr("alltypes", "bool_col is distinct from null", 2, 1);
-    verifySelectExpr("alltypes", "bool_col is distinct from null", 3, -1);
-    //verifySelectExpr("alltypes", "int_col is distinct from 10", 3, 1 - 1.0/10);
-    verifySelectExpr("alltypes", "int_col is distinct from 10", 3, -1);
-    //verifySelectExpr("alltypes", "int_col is distinct from null", 2, 1);
-    verifySelectExpr("alltypes", "int_col is distinct from null", 3, -1);
-
-    //verifySelectExpr("nullrows", "id is distinct from 'foo'", 3, 1 - 1.0/26);
-    verifySelectExpr("nullrows", "id is distinct from 'foo'", 3, -1);
-    //verifySelectExpr("nullrows", "id is distinct from null", 2, 1);
-    verifySelectExpr("nullrows", "id is distinct from null", 3, -1);
-    // Bug: All nulls, so NDV should = 1, so Sel should be 1.0/1
-    //verifySelectExpr("nullrows", "null_str is distinct from 'foo'", 2, 1 - 1.0/1);
-    verifySelectExpr("nullrows", "null_str is distinct from 'foo'", 3, -1);
-    verifySelectExpr("nullrows", "null_str is distinct from null", 3, -1);
-    //verifySelectExpr("nullrows", "group_str is distinct from 'foo'", 3, 1 - 1.0/6);
-    verifySelectExpr("nullrows", "group_str is distinct from 'foo'", 3, -1);
-    //verifySelectExpr("nullrows", "group_str is distinct from null", 2, 0);
-    verifySelectExpr("nullrows", "group_str is distinct from null", 3, -1);
-    //verifySelectExpr("nullrows", "some_nulls is not distinct from 'foo'", 3, 1 - 1.0/7);
+    verifySelectExpr("alltypes", "id is distinct from null", 3, 1);
+    verifySelectExpr("alltypes", "bool_col is distinct from true", 3, 1 - 1.0/2);
+
+    verifySelectExpr("alltypes", "bool_col is distinct from null", 3, 1);
+    verifySelectExpr("alltypes", "int_col is distinct from 10", 3, 1 - 1.0/10);
+
+    verifySelectExpr("alltypes", "int_col is distinct from null", 3, 1);
+    verifySelectExpr("nullrows", "id is distinct from 'foo'", 3, 1 - 1.0/26);
+
+    verifySelectExpr("nullrows", "id is distinct from null", 3, 1);
+    verifySelectExpr("nullrows", "null_str is distinct from 'foo'", 3, 1 - 1.0/1);
+    verifySelectExpr("nullrows", "null_str is distinct from null", 3, 0);
+    verifySelectExpr("nullrows", "group_str is distinct from 'foo'", 3, 1 - 1.0/6);
+    verifySelectExpr("nullrows", "group_str is distinct from null", 3, 1);
+    verifySelectExpr("nullrows", "group_str is distinct from null", 3, 1);
     verifySelectExpr("nullrows", "some_nulls is not distinct from 'foo'", 3, 1.0/6);
+    verifySelectExpr("nullrows", "some_nulls is distinct from null", 3, 6.0/26.0);
 
     // Bug: Sel should default to 1 - good old 0.1
     verifySelectExpr("manynulls", "id is distinct from 10", 3, -1);
diff --git a/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java b/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java
index f634b28..91fc296 100644
--- a/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java
@@ -67,10 +67,8 @@ public class CardinalityTest extends PlannerTestBase {
     verifyCardinality(
         "SELECT id FROM functional.alltypes WHERE int_col = 1", 7300/10);
 
-    // Assume classic 0.1 selectivity for other operators
-    // IMPALA-7560 says this should be revised.
     verifyCardinality(
-        "SELECT id FROM functional.alltypes WHERE int_col != 1", 730);
+        "SELECT id FROM functional.alltypes WHERE int_col != 1", 6570);
 
     // IMPALA-7601 says the following should be revised.
     verifyCardinality(
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
index 593ac72..a01b401 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
@@ -2086,7 +2086,7 @@ PLAN-ROOT SINK
 |
 05:SELECT
 |  predicates: id IS NULL, tinyint_col != 5
-|  row-size=9B cardinality=730
+|  row-size=9B cardinality=699
 |
 00:UNION
 |  row-size=9B cardinality=7.30K
@@ -2124,7 +2124,7 @@ PLAN-ROOT SINK
 |
 05:SELECT
 |  predicates: id IS NULL, tinyint_col != 5
-|  row-size=9B cardinality=730
+|  row-size=9B cardinality=699
 |
 00:UNION
 |  row-size=9B cardinality=7.30K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/card-scan.test b/testdata/workloads/functional-planner/queries/PlannerTest/card-scan.test
index cf4c316..20aee39 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/card-scan.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/card-scan.test
@@ -156,8 +156,6 @@ PLAN-ROOT SINK
 # Not-equal, card = 1 - 1/ndv
 # Use smaller table so effect is clear
 # |T|=8, NDV=8
-# Bug: IMPALA-8039
-# Bug, expected cardinality ~7
 select *
 from functional.alltypestiny
 where id != 10
@@ -167,7 +165,7 @@ PLAN-ROOT SINK
 00:SCAN HDFS [functional.alltypestiny]
    HDFS partitions=4/4 files=4 size=460B
    predicates: id != 10
-   row-size=89B cardinality=1
+   row-size=89B cardinality=7
 ====
 # Inequality. No useful stats.
 # Bug: IMPALA-8037, Assumes sel = 0.1
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test b/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
index 5a26b0f..f829753 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
@@ -411,10 +411,10 @@ PLAN-ROOT SINK
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters:
-  d:string_col NOT_EQUAL '2'
   d:string_col GREATER_OR_EQUAL '4'
   d:date_string_col EQUAL '04/03/09'
-   predicates: string_col != '2', string_col >= '4', date_string_col = '04/03/09'
+  d:string_col NOT_EQUAL '2'
+   predicates: string_col >= '4', date_string_col = '04/03/09', string_col != '2'
    row-size=89B cardinality=1
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
@@ -423,10 +423,10 @@ PLAN-ROOT SINK
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters:
-  d:string_col NOT_EQUAL '2'
   d:string_col GREATER_OR_EQUAL '4'
   d:date_string_col EQUAL '04/03/09'
-   predicates: string_col != '2', string_col >= '4', date_string_col = '04/03/09'
+  d:string_col NOT_EQUAL '2'
+   predicates: string_col >= '4', date_string_col = '04/03/09', string_col != '2'
    row-size=89B cardinality=1
 ====
 # mix of predicates and functional_hbase. filters
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test b/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
index 266d528..514bf0d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
@@ -708,7 +708,7 @@ PLAN-ROOT SINK
 |
 02:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
-   predicates: functional.alltypessmall.id != 1, functional.alltypessmall.id > 10, functional.alltypessmall.id > 20, id != 2
+   predicates: functional.alltypessmall.id > 10, functional.alltypessmall.id > 20, functional.alltypessmall.id != 1, id != 2
    runtime filters: RF000 -> id
    row-size=4B cardinality=10
 ====
@@ -772,7 +772,7 @@ PLAN-ROOT SINK
 |
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
-   predicates: functional.alltypes.id != 2, functional.alltypes.id > 10, functional.alltypes.id > 20, id != 1
+   predicates: functional.alltypes.id > 10, functional.alltypes.id > 20, functional.alltypes.id != 2, id != 1
    runtime filters: RF000 -> id
    row-size=4B cardinality=730
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
index 824188a..6bdcc41 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
@@ -143,7 +143,7 @@ and zip > 1 and zip < 50
 PLAN-ROOT SINK
 |
 00:SCAN KUDU [functional_kudu.testtbl]
-   kudu predicates: id <= 20, id >= 10, zip < 50, zip <= 30, zip <= 5, zip > 1, zip >= 0, name = 'foo'
+   kudu predicates: name = 'foo', id <= 20, id >= 10, zip < 50, zip <= 30, zip <= 5, zip > 1, zip >= 0
    row-size=28B cardinality=0
 ---- SCANRANGELOCATIONS
 NODE 0:
@@ -154,7 +154,7 @@ PLAN-ROOT SINK
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN KUDU [functional_kudu.testtbl]
-   kudu predicates: id <= 20, id >= 10, zip < 50, zip <= 30, zip <= 5, zip > 1, zip >= 0, name = 'foo'
+   kudu predicates: name = 'foo', id <= 20, id >= 10, zip < 50, zip <= 30, zip <= 5, zip > 1, zip >= 0
    row-size=28B cardinality=0
 ====
 # Constant propagation works for Kudu
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test b/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
index b977b98..74ff2b4 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
@@ -585,7 +585,7 @@ PLAN-ROOT SINK
 |  01:SCAN HDFS [functional.alltypessmall]
 |     partition predicates: functional.alltypessmall.year = 2009, functional.alltypessmall.month <= 2
 |     HDFS partitions=2/4 files=2 size=3.16KB
-|     predicates: functional.alltypessmall.int_col != 5, id > 11
+|     predicates: id > 11, functional.alltypessmall.int_col != 5
 |     row-size=16B cardinality=5
 |
 00:SCAN HDFS [functional.alltypes a]
@@ -629,7 +629,7 @@ PLAN-ROOT SINK
 |  01:SCAN HDFS [functional.alltypessmall]
 |     partition predicates: functional.alltypessmall.year = 2009, functional.alltypessmall.month <= 2
 |     HDFS partitions=2/4 files=2 size=3.16KB
-|     predicates: functional.alltypessmall.int_col != 5, id > 11
+|     predicates:  id > 11, functional.alltypessmall.int_col != 5
 |     row-size=16B cardinality=5
 |
 00:SCAN HDFS [functional.alltypes a]
@@ -672,7 +672,7 @@ PLAN-ROOT SINK
 |  01:SCAN HDFS [functional.alltypessmall]
 |     partition predicates: functional.alltypessmall.year = 2009, functional.alltypessmall.month <= 2
 |     HDFS partitions=2/4 files=2 size=3.16KB
-|     predicates: functional.alltypessmall.int_col != 5, id > 11
+|     predicates: id > 11, functional.alltypessmall.int_col != 5
 |     row-size=16B cardinality=5
 |
 00:SCAN HDFS [functional.alltypes a]
@@ -709,7 +709,7 @@ PLAN-ROOT SINK
 |  01:SCAN HDFS [functional.alltypessmall]
 |     partition predicates: functional.alltypessmall.year = 2009, functional.alltypessmall.month <= 2
 |     HDFS partitions=2/4 files=2 size=3.16KB
-|     predicates: functional.alltypessmall.int_col != 5, id > 11
+|     predicates: id > 11, functional.alltypessmall.int_col != 5
 |     row-size=16B cardinality=5
 |
 00:SCAN HDFS [functional.alltypes a]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
index 917e5c1..e54f6a3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
@@ -3453,20 +3453,20 @@ PLAN-ROOT SINK
 |
 07:SORT
 |  order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 06:AGGREGATE [FINALIZE]
 |  output: count(ps_suppkey)
 |  group by: p_brand, p_type, p_size
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 05:AGGREGATE
 |  group by: p_brand, p_type, p_size, ps_suppkey
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 04:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
 |  hash predicates: ps_suppkey = s_suppkey
-|  row-size=81B cardinality=31.92K
+|  row-size=81B cardinality=31.49K
 |
 |--02:SCAN HDFS [tpch.supplier]
 |     HDFS partitions=1/1 files=1 size=1.33MB
@@ -3476,12 +3476,12 @@ PLAN-ROOT SINK
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: ps_partkey = p_partkey
 |  runtime filters: RF000 <- p_partkey
-|  row-size=81B cardinality=31.92K
+|  row-size=81B cardinality=31.49K
 |
 |--01:SCAN HDFS [tpch.part]
 |     HDFS partitions=1/1 files=1 size=22.83MB
-|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), p_brand != 'Brand#45', NOT p_type LIKE 'MEDIUM POLISHED%'
-|     row-size=65B cardinality=8.00K
+|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), NOT p_type LIKE 'MEDIUM POLISHED%', p_brand != 'Brand#45'
+|     row-size=65B cardinality=7.89K
 |
 00:SCAN HDFS [tpch.partsupp]
    HDFS partitions=1/1 files=1 size=112.71MB
@@ -3497,33 +3497,33 @@ PLAN-ROOT SINK
 |
 07:SORT
 |  order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 13:AGGREGATE [FINALIZE]
 |  output: count:merge(ps_suppkey)
 |  group by: p_brand, p_type, p_size
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 12:EXCHANGE [HASH(p_brand,p_type,p_size)]
 |
 06:AGGREGATE [STREAMING]
 |  output: count(ps_suppkey)
 |  group by: p_brand, p_type, p_size
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 11:AGGREGATE
 |  group by: p_brand, p_type, p_size, ps_suppkey
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 10:EXCHANGE [HASH(p_brand,p_type,p_size,ps_suppkey)]
 |
 05:AGGREGATE [STREAMING]
 |  group by: p_brand, p_type, p_size, ps_suppkey
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 04:HASH JOIN [NULL AWARE LEFT ANTI JOIN, BROADCAST]
 |  hash predicates: ps_suppkey = s_suppkey
-|  row-size=81B cardinality=31.92K
+|  row-size=81B cardinality=31.49K
 |
 |--09:EXCHANGE [BROADCAST]
 |  |
@@ -3535,14 +3535,14 @@ PLAN-ROOT SINK
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ps_partkey = p_partkey
 |  runtime filters: RF000 <- p_partkey
-|  row-size=81B cardinality=31.92K
+|  row-size=81B cardinality=31.49K
 |
 |--08:EXCHANGE [BROADCAST]
 |  |
 |  01:SCAN HDFS [tpch.part]
 |     HDFS partitions=1/1 files=1 size=22.83MB
-|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), p_brand != 'Brand#45', NOT p_type LIKE 'MEDIUM POLISHED%'
-|     row-size=65B cardinality=8.00K
+|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), NOT p_type LIKE 'MEDIUM POLISHED%', p_brand != 'Brand#45'
+|     row-size=65B cardinality=7.89K
 |
 00:SCAN HDFS [tpch.partsupp]
    HDFS partitions=1/1 files=1 size=112.71MB
@@ -3558,33 +3558,33 @@ PLAN-ROOT SINK
 |
 07:SORT
 |  order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 13:AGGREGATE [FINALIZE]
 |  output: count:merge(ps_suppkey)
 |  group by: p_brand, p_type, p_size
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 12:EXCHANGE [HASH(p_brand,p_type,p_size)]
 |
 06:AGGREGATE [STREAMING]
 |  output: count(ps_suppkey)
 |  group by: p_brand, p_type, p_size
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 11:AGGREGATE
 |  group by: p_brand, p_type, p_size, ps_suppkey
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 10:EXCHANGE [HASH(p_brand,p_type,p_size,ps_suppkey)]
 |
 05:AGGREGATE [STREAMING]
 |  group by: p_brand, p_type, p_size, ps_suppkey
-|  row-size=65B cardinality=31.92K
+|  row-size=65B cardinality=31.49K
 |
 04:HASH JOIN [NULL AWARE LEFT ANTI JOIN, BROADCAST]
 |  hash predicates: ps_suppkey = s_suppkey
-|  row-size=81B cardinality=31.92K
+|  row-size=81B cardinality=31.49K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
@@ -3599,7 +3599,7 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ps_partkey = p_partkey
-|  row-size=81B cardinality=31.92K
+|  row-size=81B cardinality=31.49K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
@@ -3610,8 +3610,8 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS [tpch.part]
 |     HDFS partitions=1/1 files=1 size=22.83MB
-|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), p_brand != 'Brand#45', NOT p_type LIKE 'MEDIUM POLISHED%'
-|     row-size=65B cardinality=8.00K
+|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), NOT p_type LIKE 'MEDIUM POLISHED%', p_brand != 'Brand#45'
+|     row-size=65B cardinality=7.89K
 |
 00:SCAN HDFS [tpch.partsupp]
    HDFS partitions=1/1 files=1 size=112.71MB
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
index b5f85ab..b851b4a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
@@ -1228,7 +1228,7 @@ PLAN-ROOT SINK
 |  row-size=89B cardinality=31.92K
 |
 |--01:SCAN KUDU [tpch_kudu.part]
-|     predicates: p_brand != 'Brand#45', NOT p_type LIKE 'MEDIUM POLISHED%'
+|     predicates: NOT p_type LIKE 'MEDIUM POLISHED%', p_brand != 'Brand#45'
 |     kudu predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9)
 |     row-size=73B cardinality=8.00K
 |
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
index 94759c8..c5a158d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
@@ -2307,8 +2307,8 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [tpch_nested_parquet.part p]
 |     HDFS partitions=1/1 files=1 size=6.24MB
-|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), p_brand != 'Brand#45', NOT p_type LIKE 'MEDIUM POLISHED%'
-|     row-size=65B cardinality=8.00K
+|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), NOT p_type LIKE 'MEDIUM POLISHED%', p_brand != 'Brand#45'
+|     row-size=65B cardinality=7.89K
 |
 01:SUBPLAN
 |  row-size=103B cardinality=10.00K
@@ -2368,8 +2368,8 @@ PLAN-ROOT SINK
 |  |
 |  05:SCAN HDFS [tpch_nested_parquet.part p]
 |     HDFS partitions=1/1 files=1 size=6.24MB
-|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), p_brand != 'Brand#45', NOT p_type LIKE 'MEDIUM POLISHED%'
-|     row-size=65B cardinality=8.00K
+|     predicates: p_size IN (49, 14, 23, 45, 19, 3, 36, 9), NOT p_type LIKE 'MEDIUM POLISHED%', p_brand != 'Brand#45'
+|     row-size=65B cardinality=7.89K
 |
 01:SUBPLAN
 |  row-size=103B cardinality=10.00K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
index adc56aa..ff599c0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
@@ -1222,7 +1222,7 @@ PLAN-ROOT SINK
 |
 |--01:SCAN HDFS [tpch.part]
 |     HDFS partitions=1/1 files=1 size=22.83MB
-|     predicates: tpch.part.p_size IN (49, 14, 23, 45, 19, 3, 36, 9), tpch.part.p_brand != 'Brand#45', NOT tpch.part.p_type LIKE 'MEDIUM POLISHED%'
+|     predicates: tpch.part.p_size IN (49, 14, 23, 45, 19, 3, 36, 9), NOT tpch.part.p_type LIKE 'MEDIUM POLISHED%', tpch.part.p_brand != 'Brand#45'
 |     row-size=65B cardinality=8.00K
 |
 00:SCAN HDFS [tpch.partsupp]

[impala] 07/07: IMPALA-10714: Defer advancing read page until the buffer is attached

Posted by st...@apache.org.
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 80a1dc33d33538af609a0f1dd761363a06d279b8
Author: Riza Suminto <ri...@cloudera.com>
AuthorDate: Thu Sep 16 15:12:22 2021 -0700

    IMPALA-10714: Defer advancing read page until the buffer is attached
    
    If a BufferedTupleStream is a read-write stream and set up with
    attach_on_read = true, BufferedTupleStream::NextReadPage() expects that
    the page buffer is attached to the output row batch before advancing the
    read iterator. However, BufferedTupleStream::GetNextInternal() will not
    attach the page buffer of a fully read page if it is a read-write page.
    
    Consider the following scenario:
    1. Only 1 page left in stream. This is a read-write page.
    2. GetNext() has fully read the page, but does NOT attach the buffer to
       output row batch because it is a read-write page.
    3. Stream writer insert more rows, but the read-write page can not fit
       any more rows. Therefore, new pages are created.
    4. Stream writer call UnpinStream().
    5. UnpinStream() call NextReadPage(), which in turn will fail the
       assertion "read_iter->read_page_->attached_to_output_batch".
    
    BufferedTupleStream::UnpinStream() need to defer advancing the read page
    if this situation happens.
    
    This patch adds BE test StreamStateTest.UnpinFullyExhaustedReadPage that
    simulates the corner case. This patch also moves BE test
    DeferAdvancingReadPage and ShortDebugString into class StreamStateTest
    to reduce friend class declaration in buffered-tuple-stream.h
    
    Testing:
    - Run and pass BE test StreamStateTest.UnpinFullyExhaustedReadPage.
    
    Change-Id: I586ed72ba01cc3f28b0dcb1e202b3ca32a6c3b83
    Reviewed-on: http://gerrit.cloudera.org:8080/17853
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/runtime/buffered-tuple-stream-test.cc | 229 ++++++++++++++++++++++++++-
 be/src/runtime/buffered-tuple-stream.cc      |  30 ++--
 be/src/runtime/buffered-tuple-stream.h       |   4 +-
 3 files changed, 244 insertions(+), 19 deletions(-)

diff --git a/be/src/runtime/buffered-tuple-stream-test.cc b/be/src/runtime/buffered-tuple-stream-test.cc
index dfdfeba..ad34f5e 100644
--- a/be/src/runtime/buffered-tuple-stream-test.cc
+++ b/be/src/runtime/buffered-tuple-stream-test.cc
@@ -569,6 +569,38 @@ class ArrayTupleStreamTest : public SimpleTupleStreamTest {
   }
 };
 
+/// Test internal stream state under certain corner cases.
+class StreamStateTest : public SimpleTupleStreamTest {
+ protected:
+  // Test that UnpinStream defers advancing the read page when all rows from the read
+  // page are attached to a returned RowBatch but got not enough reservation.
+  void TestDeferAdvancingReadPage();
+
+  // Test unpinning a read-write stream when the read page has been fully exhausted but
+  // its buffer is not attached yet to the output row batch.
+  void TestUnpinAfterFullStreamRead(
+      bool read_write, bool attach_on_read, bool refill_before_unpin);
+
+  // Fill up the stream by repeatedly inserting write_batch into the stream until it is
+  // full. Return number of rows successfully inserted into the stream.
+  // Stream must be in pinned mode.
+  Status FillUpStream(
+      BufferedTupleStream* stream, RowBatch* write_batch, int64_t& num_inserted);
+
+  // Read out the stream until eos is reached. Return number of rows successfully read.
+  Status ReadOutStream(
+      BufferedTupleStream* stream, RowBatch* read_batch, int64_t& num_read);
+
+  // Verify that page count, pinned bytes, and unpinned bytes of the stream match the
+  // expectation.
+  void VerifyStreamState(BufferedTupleStream* stream, int num_page, int num_pinned_page,
+      int num_unpinned_page, int buffer_size);
+
+  // Test that stream's debug string is capped only for the first
+  // BufferedTupleStream::MAX_PAGE_ITER_DEBUG.
+  void TestShortDebugString();
+};
+
 // Basic API test. No data should be going to disk.
 TEST_F(SimpleTupleStreamTest, Basic) {
   Init(numeric_limits<int64_t>::max());
@@ -1297,9 +1329,7 @@ TEST_F(SimpleTupleStreamTest, UnpinReadPage) {
   write_batch->Reset();
 }
 
-// Test that UnpinStream defer advancing the read page when all rows from the read page
-// are attached to a returned RowBatch but got not enough reservation.
-TEST_F(SimpleTupleStreamTest, DeferAdvancingReadPage) {
+void StreamStateTest::TestDeferAdvancingReadPage() {
   int num_rows = 1024;
   int buffer_size = 4 * 1024;
   // Only give 2 * buffer_size for the stream initial read and write page reservation.
@@ -1315,7 +1345,7 @@ TEST_F(SimpleTupleStreamTest, DeferAdvancingReadPage) {
     // and the output batch has NOT been reset.
     BufferedTupleStream stream(
         runtime_state_, int_desc_, &client_, buffer_size, buffer_size);
-    ASSERT_OK(stream.Init("SimpleTupleStreamTest::DeferAdvancingReadPage", true));
+    ASSERT_OK(stream.Init("StreamStateTest::DeferAdvancingReadPage", true));
     ASSERT_OK(stream.PrepareForReadWrite(true, &got_reservation));
     ASSERT_TRUE(got_reservation);
 
@@ -1367,6 +1397,126 @@ TEST_F(SimpleTupleStreamTest, DeferAdvancingReadPage) {
   write_batch->Reset();
 }
 
+void StreamStateTest::TestUnpinAfterFullStreamRead(
+    bool read_write, bool attach_on_read, bool refill_before_unpin) {
+  DCHECK(read_write || !refill_before_unpin)
+      << "Only read-write stream support refilling stream after full read.";
+
+  int num_rows = 1024;
+  int buffer_size = 4 * 1024;
+  int max_num_pages = 4;
+  Init(max_num_pages * buffer_size);
+
+  bool got_reservation;
+  RowBatch* write_batch = CreateIntBatch(0, num_rows, false);
+
+  {
+    BufferedTupleStream stream(
+        runtime_state_, int_desc_, &client_, buffer_size, buffer_size);
+    ASSERT_OK(stream.Init("StreamStateTest::TestUnpinAfterFullStreamRead", true));
+    if (read_write) {
+      ASSERT_OK(stream.PrepareForReadWrite(attach_on_read, &got_reservation));
+    } else {
+      ASSERT_OK(stream.PrepareForWrite(&got_reservation));
+    }
+    ASSERT_TRUE(got_reservation);
+    RowBatch read_batch(int_desc_, num_rows, &tracker_);
+    int64_t num_rows_written = 0;
+    int64_t num_rows_read = 0;
+
+    // Add rows into the stream until the stream is full.
+    ASSERT_OK(FillUpStream(&stream, write_batch, num_rows_written));
+    int num_pages = max_num_pages;
+    ASSERT_EQ(stream.pages_.size(), num_pages);
+    ASSERT_FALSE(stream.has_read_write_page());
+
+    // Read the entire rows out of the stream.
+    if (!read_write) {
+      ASSERT_OK(stream.PrepareForRead(attach_on_read, &got_reservation));
+      ASSERT_TRUE(got_reservation);
+    }
+    ASSERT_OK(ReadOutStream(&stream, &read_batch, num_rows_read));
+    if (attach_on_read) num_pages = 1;
+    ASSERT_EQ(stream.pages_.size(), num_pages);
+    ASSERT_EQ(stream.has_read_write_page(), read_write);
+
+    if (read_write && refill_before_unpin) {
+      // Fill the stream until it is full again.
+      ASSERT_OK(FillUpStream(&stream, write_batch, num_rows_written));
+      num_pages = max_num_pages;
+      ASSERT_EQ(stream.pages_.size(), num_pages);
+      ASSERT_EQ(stream.has_read_write_page(), !attach_on_read);
+    }
+
+    // Verify that the read page has been fully read before unpinning the stream.
+    ASSERT_EQ(
+        stream.read_it_.read_page_rows_returned_, stream.read_it_.read_page_->num_rows);
+    // read_page_ should NOT be attached to output batch unless stream is in read-only and
+    // attach_on_read mode.
+    bool attached = !read_write && attach_on_read;
+    ASSERT_EQ(stream.read_it_.read_page_->attached_to_output_batch, attached);
+
+    // Verify stream state before UnpinStream.
+    int num_pinned_pages = num_pages;
+    ASSERT_TRUE(stream.is_pinned());
+    if (attached) {
+      // In a pinned + read-only + attach_on_read stream, a fully exhausted read page is
+      // automatically unpinned and destroyed, but not yet removed from stream.pages_
+      // until the next GetNext() or UnpinStream() call.
+      ASSERT_EQ(stream.pages_.size(), 1);
+      num_pinned_pages = 0;
+    }
+    VerifyStreamState(&stream, num_pages, num_pinned_pages, 0, buffer_size);
+
+    // Unpin the stream.
+    ASSERT_OK(stream.UnpinStream(BufferedTupleStream::UNPIN_ALL_EXCEPT_CURRENT));
+    ASSERT_FALSE(stream.is_pinned());
+
+    // Verify stream state after UnpinStream. num_pages should remain unchanged after
+    // UnpinStream() except for the case of read-only + attach_on_read stream.
+    if (read_write) {
+      if (attach_on_read) {
+        if (refill_before_unpin) {
+          num_pinned_pages = 2;
+          ASSERT_TRUE(stream.pages_.begin()->is_pinned());
+          ASSERT_TRUE(stream.pages_.back().is_pinned());
+        } else {
+          num_pinned_pages = 1;
+          ASSERT_TRUE(stream.pages_.back().is_pinned());
+        }
+      } else {
+        num_pinned_pages = 1;
+        ASSERT_TRUE(stream.pages_.back().is_pinned());
+      }
+    } else {
+      if (attach_on_read) {
+        num_pages = 0;
+      }
+      num_pinned_pages = 0;
+    }
+    int num_unpinned_pages = num_pages - num_pinned_pages;
+    VerifyStreamState(
+        &stream, num_pages, num_pinned_pages, num_unpinned_pages, buffer_size);
+
+    if (read_write) {
+      // Additionally, test that write and read operation still work in read-write
+      // stream after UnpinStream.
+      Status status;
+      ASSERT_OK(ReadOutStream(&stream, &read_batch, num_rows_read));
+      for (int i = 0; i < write_batch->num_rows(); ++i) {
+        EXPECT_TRUE(stream.AddRow(write_batch->GetRow(i), &status));
+        ASSERT_OK(status);
+      }
+      ASSERT_OK(ReadOutStream(&stream, &read_batch, num_rows_read));
+      ASSERT_EQ(write_batch->num_rows(), num_rows_read);
+    }
+
+    stream.Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+    read_batch.Reset();
+  }
+  write_batch->Reset();
+}
+
 // Test writing to a stream (AddRow and UnpinStream), even though attached pages have not
 // been released yet.
 TEST_F(SimpleTupleStreamTest, WriteAfterReadAttached) {
@@ -1527,7 +1677,7 @@ TEST_F(SimpleTupleStreamTest, ConcurrentReaders) {
   stream.Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
 }
 
-TEST_F(SimpleTupleStreamTest, ShortDebugString) {
+void StreamStateTest::TestShortDebugString() {
   Init(BUFFER_POOL_LIMIT);
 
   int num_batches = 50;
@@ -1539,7 +1689,7 @@ TEST_F(SimpleTupleStreamTest, ShortDebugString) {
 
   BufferedTupleStream stream(
       runtime_state_, desc, &client_, default_page_len, max_page_len);
-  ASSERT_OK(stream.Init("SimpleTupleStreamTest::ShortDebugString", true));
+  ASSERT_OK(stream.Init("StreamStateTest::ShortDebugString", true));
   bool got_write_reservation;
   ASSERT_OK(stream.PrepareForWrite(&got_write_reservation));
   ASSERT_TRUE(got_write_reservation);
@@ -2022,6 +2172,73 @@ TEST_F(ArrayTupleStreamTest, TestComputeRowSize) {
 
   stream.Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
 }
+
+Status StreamStateTest::FillUpStream(
+    BufferedTupleStream* stream, RowBatch* write_batch, int64_t& num_inserted) {
+  DCHECK(stream->is_pinned());
+  int64_t idx = 0;
+  Status status;
+  num_inserted = 0;
+  while (stream->AddRow(write_batch->GetRow(idx), &status)) {
+    RETURN_IF_ERROR(status);
+    idx = (idx + 1) % write_batch->num_rows();
+    num_inserted++;
+  }
+  return status;
+}
+
+Status StreamStateTest::ReadOutStream(
+    BufferedTupleStream* stream, RowBatch* read_batch, int64_t& num_read) {
+  bool eos = false;
+  num_read = 0;
+  do {
+    read_batch->Reset();
+    RETURN_IF_ERROR(stream->GetNext(read_batch, &eos));
+    num_read += read_batch->num_rows();
+  } while (!eos);
+  return Status::OK();
+}
+
+void StreamStateTest::VerifyStreamState(BufferedTupleStream* stream, int num_page,
+    int num_pinned_page, int num_unpinned_page, int buffer_size) {
+  ASSERT_EQ(stream->pages_.size(), num_page);
+  ASSERT_EQ(stream->num_pages_, num_page);
+  ASSERT_EQ(stream->BytesPinned(false), buffer_size * num_pinned_page);
+  ASSERT_EQ(stream->bytes_unpinned(), buffer_size * num_unpinned_page);
+  stream->CheckConsistencyFull(stream->read_it_);
+}
+
+TEST_F(StreamStateTest, DeferAdvancingReadPage) {
+  TestDeferAdvancingReadPage();
+}
+
+TEST_F(StreamStateTest, UnpinFullyExhaustedReadPageOnReadWriteStreamNoAttachRefill) {
+  TestUnpinAfterFullStreamRead(true, false, true);
+}
+
+TEST_F(StreamStateTest, UnpinFullyExhaustedReadPageOnReadWriteStreamNoAttachNoRefill) {
+  TestUnpinAfterFullStreamRead(true, false, false);
+}
+
+TEST_F(StreamStateTest, UnpinFullyExhaustedReadPageOnReadWriteStreamAttachRefill) {
+  TestUnpinAfterFullStreamRead(true, true, true);
+}
+
+TEST_F(StreamStateTest, UnpinFullyExhaustedReadPageOnReadWriteStreamAttachNoRefill) {
+  TestUnpinAfterFullStreamRead(true, true, false);
+}
+
+TEST_F(StreamStateTest, UnpinFullyExhaustedReadPageOnReadOnlyStreamAttach) {
+  TestUnpinAfterFullStreamRead(false, true, false);
+}
+
+TEST_F(StreamStateTest, UnpinFullyExhaustedReadPageOnReadOnlyStreamNoAttach) {
+  TestUnpinAfterFullStreamRead(false, false, false);
+}
+
+TEST_F(StreamStateTest, ShortDebugString) {
+  TestShortDebugString();
+}
 }
 
 int main(int argc, char** argv) {
diff --git a/be/src/runtime/buffered-tuple-stream.cc b/be/src/runtime/buffered-tuple-stream.cc
index 01edc84..4330696 100644
--- a/be/src/runtime/buffered-tuple-stream.cc
+++ b/be/src/runtime/buffered-tuple-stream.cc
@@ -719,16 +719,24 @@ Status BufferedTupleStream::UnpinStream(UnpinMode mode) {
     bool defer_advancing_read_page = false;
     if (&*read_it_.read_page_ != write_page_ && read_it_.read_page_ != pages_.end()
         && read_it_.read_page_rows_returned_ == read_it_.read_page_->num_rows) {
-      if (read_it_.read_page_->attached_to_output_batch) {
-        if (num_pages_ <= 2) {
-          // NextReadPage will attempt to save default_page_len_ into write reservation if
-          // the stream ended up with only 1 read/write page after advancing the read
-          // page. This can potentially lead to negative unused reservation if the reader
-          // has not freed the row batch where the read page buffer is attached to. We
-          // defer advancing the read page until the next GetNext() call by the reader
-          // (see IMPALA-10584).
-          defer_advancing_read_page = true;
-        }
+      if (has_write_iterator_ && read_it_.attach_on_read_
+          && (num_pages_ <= 2 || !read_it_.read_page_->attached_to_output_batch)) {
+        // In a read-write stream + attach_on_read mode, there are cases where we should
+        // NOT advance the read page even though the page has been fully exhausted:
+        //
+        // 1. Stream has exactly 2 pages: 1 read and 1 write.
+        //    NextReadPage() will attempt to save default_page_len_ into write
+        //    reservation if the stream ended up with only 1 read/write page after
+        //    advancing the read page. This can potentially lead to negative unused
+        //    reservation if the reader has not freed the row batch where the read page
+        //    buffer is attached to (see IMPALA-10584).
+        // 2. Read page buffer has not been attached yet to the output row batch.
+        //    The previous GetNext() would not attach the read page buffer to the output
+        //    row batch if it was a read-write page (see IMPALA-10714).
+        //
+        // We defer advancing the read page for these cases until the next GetNext()
+        // call by the reader.
+        defer_advancing_read_page = true;
       }
 
       if (!defer_advancing_read_page) {
@@ -742,7 +750,7 @@ Status BufferedTupleStream::UnpinStream(UnpinMode mode) {
     std::list<Page>::iterator it = pages_.begin();
     if (defer_advancing_read_page) {
       // We skip advancing the read page earlier, so the first page must be a read page
-      // and attached_to_output_batch is true. We should keep the first page pinned. The
+      // and the reader has not done reading it. We should keep the first page pinned. The
       // next GetNext() call is the one who will be responsible to unpin the first page.
       DCHECK(read_it_.read_page_ == pages_.begin());
       ++it;
diff --git a/be/src/runtime/buffered-tuple-stream.h b/be/src/runtime/buffered-tuple-stream.h
index bba6479..8546414 100644
--- a/be/src/runtime/buffered-tuple-stream.h
+++ b/be/src/runtime/buffered-tuple-stream.h
@@ -484,6 +484,7 @@ class BufferedTupleStream {
 
    private:
     friend class BufferedTupleStream;
+    friend class StreamStateTest;
 
     /// True if the read iterator is currently valid
     bool valid_ = false;
@@ -560,8 +561,7 @@ class BufferedTupleStream {
 
  private:
   DISALLOW_COPY_AND_ASSIGN(BufferedTupleStream);
-  friend class SimpleTupleStreamTest_ShortDebugString_Test;
-  friend class SimpleTupleStreamTest_DeferAdvancingReadPage_Test;
+  friend class StreamStateTest;
 
   /// Runtime state instance used to check for cancellation. Not owned.
   RuntimeState* const state_;

[impala] 04/07: IMPALA-10810: Bump json-smart from 2.3 to 2.4.7

Posted by st...@apache.org.
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 849ecb9eb9e0d37f7341f98a52bffe95350b9663
Author: Zoltan Borok-Nagy <bo...@cloudera.com>
AuthorDate: Tue Jul 20 14:39:04 2021 +0200

    IMPALA-10810: Bump json-smart from 2.3 to 2.4.7
    
    I noticed that our json-smart dependency is stale and we could
    pick up a newer version.
    
    This patch upgrades it to 2.4.7 which is the newest version at
    the time of writing.
    
    Change-Id: I6b43f606f40e172aa267b55c564fa64d68515bd5
    Reviewed-on: http://gerrit.cloudera.org:8080/17702
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 fe/pom.xml                         | 4 ++--
 java/datagenerator/pom.xml         | 2 +-
 java/executor-deps/pom.xml         | 4 ++--
 java/pom.xml                       | 1 +
 java/shaded-deps/hive-exec/pom.xml | 4 ++--
 java/test-hive-udfs/pom.xml        | 2 +-
 java/yarn-extras/pom.xml           | 2 +-
 7 files changed, 10 insertions(+), 9 deletions(-)

diff --git a/fe/pom.xml b/fe/pom.xml
index 8a3840e..7a3d22a 100644
--- a/fe/pom.xml
+++ b/fe/pom.xml
@@ -33,11 +33,11 @@ under the License.
   <name>Apache Impala Query Engine Frontend</name>
 
   <dependencies>
-    <!-- Pin json-smart dependency to version 2.3 -->
+    <!-- Pin json-smart dependency. -->
     <dependency>
       <groupId>net.minidev</groupId>
       <artifactId>json-smart</artifactId>
-      <version>2.3</version>
+      <version>${json-smart.version}</version>
     </dependency>
 
     <dependency>
diff --git a/java/datagenerator/pom.xml b/java/datagenerator/pom.xml
index b14a6e4..dcc1a1d 100644
--- a/java/datagenerator/pom.xml
+++ b/java/datagenerator/pom.xml
@@ -38,7 +38,7 @@ under the License.
     <dependency>
       <groupId>net.minidev</groupId>
       <artifactId>json-smart</artifactId>
-      <version>2.3</version>
+      <version>${json-smart.version}</version>
     </dependency>
 
     <dependency>
diff --git a/java/executor-deps/pom.xml b/java/executor-deps/pom.xml
index 50dc413..502a023 100644
--- a/java/executor-deps/pom.xml
+++ b/java/executor-deps/pom.xml
@@ -45,11 +45,11 @@ under the License.
 
   <dependencies>
 
-    <!-- Pin json-smart dependency to version 2.3 -->
+    <!-- Pin json-smart dependency. -->
     <dependency>
       <groupId>net.minidev</groupId>
       <artifactId>json-smart</artifactId>
-      <version>2.3</version>
+      <version>${json-smart.version}</version>
     </dependency>
 
     <dependency>
diff --git a/java/pom.xml b/java/pom.xml
index 7f9498d..702b367 100644
--- a/java/pom.xml
+++ b/java/pom.xml
@@ -71,6 +71,7 @@ under the License.
     <xmlsec.version>2.2.1</xmlsec.version>
     <bcprov-jdk15on.version>1.64</bcprov-jdk15on.version>
     <springframework.version>4.3.29.RELEASE</springframework.version>
+    <json-smart.version>2.4.7</json-smart.version>
   </properties>
 
   <repositories>
diff --git a/java/shaded-deps/hive-exec/pom.xml b/java/shaded-deps/hive-exec/pom.xml
index 95030c6..088c496 100644
--- a/java/shaded-deps/hive-exec/pom.xml
+++ b/java/shaded-deps/hive-exec/pom.xml
@@ -35,12 +35,12 @@ the same dependencies
   <packaging>jar</packaging>
 
   <dependencies>
-    <!-- Pin json-smart dependency to version 2.3
+    <!-- Pin json-smart dependency.
          (this is not included in the shaded jar) -->
     <dependency>
       <groupId>net.minidev</groupId>
       <artifactId>json-smart</artifactId>
-      <version>2.3</version>
+      <version>${json-smart.version}</version>
     </dependency>
 
     <dependency>
diff --git a/java/test-hive-udfs/pom.xml b/java/test-hive-udfs/pom.xml
index 6ef7f24..d4f9c03 100644
--- a/java/test-hive-udfs/pom.xml
+++ b/java/test-hive-udfs/pom.xml
@@ -42,7 +42,7 @@ under the License.
     <dependency>
       <groupId>net.minidev</groupId>
       <artifactId>json-smart</artifactId>
-      <version>2.3</version>
+      <version>${json-smart.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
diff --git a/java/yarn-extras/pom.xml b/java/yarn-extras/pom.xml
index 68ba84d..1fded2a 100644
--- a/java/yarn-extras/pom.xml
+++ b/java/yarn-extras/pom.xml
@@ -37,7 +37,7 @@
     <dependency>
       <groupId>net.minidev</groupId>
       <artifactId>json-smart</artifactId>
-      <version>2.3</version>
+      <version>${json-smart.version}</version>
     </dependency>
 
     <dependency>

[impala] 05/07: IMPALA-10825: fix impalad crashes when closing the retrying query

Posted by st...@apache.org.
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 cb604bbd3ac6bb14c0aa994a6f743069d8ce7029
Author: xqhe <he...@126.com>
AuthorDate: Tue Jul 27 17:34:45 2021 +0800

    IMPALA-10825: fix impalad crashes when closing the retrying query
    
    The crash happens when canceling the retrying query in web UI.
    The canceling action will call ImpalaServer#UnregisterQuery.
    The QueryDriver will be null if the query has already been unregistered.
    
    Testing:
     Add test in tests/custom_cluster/test_query_retries.py and manually
    tested 100 times to make sure that there was no Impalad crash
    
    Change-Id: I3b9a2cccbfbdca00b099e0f8d5f2d4bcb4d0a8c3
    Reviewed-on: http://gerrit.cloudera.org:8080/17729
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/runtime/query-driver.cc             | 13 ++++++++++---
 tests/custom_cluster/test_query_retries.py | 26 ++++++++++++++++++++++++++
 2 files changed, 36 insertions(+), 3 deletions(-)

diff --git a/be/src/runtime/query-driver.cc b/be/src/runtime/query-driver.cc
index db8c3bd..c5d979c 100644
--- a/be/src/runtime/query-driver.cc
+++ b/be/src/runtime/query-driver.cc
@@ -174,14 +174,21 @@ void QueryDriver::TryQueryRetry(
         << Substitute("Cannot retry a that has already been retried query_id = $0",
             PrintId(query_id));
 
-    // Update the state and then schedule the retry asynchronously.
-    client_request_state_->MarkAsRetrying(*error);
-
     // Another reference to this QueryDriver (via the shared_ptr) needs to be created and
     // passed to the thread so that a valid shared_ptr exists while the thread is running.
     // Otherwise it is possible that the user cancels the query and this QueryDriver gets
     // deleted by the shared_ptr.
+    DebugActionNoFail(FLAGS_debug_actions, "RETRY_DELAY_GET_QUERY_DRIVER");
     shared_ptr<QueryDriver> query_driver = parent_server_->GetQueryDriver(query_id);
+    if (query_driver.get() == nullptr) {
+      VLOG_QUERY << Substitute(
+          "Skipping retry of query_id=$0 because it has already been unregistered",
+          PrintId(query_id));
+      return;
+    }
+
+    // Update the state and then schedule the retry asynchronously.
+    client_request_state_->MarkAsRetrying(*error);
 
     // Launch the query retry in a separate thread, 'was_retried' is set to true
     // if the query retry was successfully launched.
diff --git a/tests/custom_cluster/test_query_retries.py b/tests/custom_cluster/test_query_retries.py
index 5a6e8a4..04bcd89 100644
--- a/tests/custom_cluster/test_query_retries.py
+++ b/tests/custom_cluster/test_query_retries.py
@@ -747,6 +747,32 @@ class TestQueryRetries(CustomClusterTestSuite):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
+      impalad_args="--debug_actions=RETRY_DELAY_GET_QUERY_DRIVER:SLEEP@2000",
+      statestored_args="--statestore_heartbeat_frequency_ms=60000")
+  def test_retry_query_close_before_getting_query_driver(self):
+    """Trigger a query retry, and then close the retried query before getting
+    the query driver. Validate that it doesn't crash the impalad.
+    Set a really high statestore heartbeat frequency so that killed impalads are not
+    removed from the cluster membership."""
+
+    # Kill an impalad, and run a query. The query should be retried.
+    self.cluster.impalads[1].kill()
+    query = "select count(*) from tpch_parquet.lineitem"
+    handle = self.execute_query_async(query,
+        query_options={'retry_failed_queries': 'true'})
+
+    time.sleep(1)
+    # close the query
+    self.client.close_query(handle)
+
+    time.sleep(2)
+    impala_service = self.cluster.get_first_impalad().service
+    self.assert_eventually(60, 0.1,
+        lambda: impala_service.get_num_in_flight_queries() == 0,
+        lambda: "in-flight queries: %d" % impala_service.get_num_in_flight_queries())
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
       impalad_args="--debug_actions=QUERY_RETRY_SET_RESULT_CACHE:FAIL",
       statestored_args="--statestore_heartbeat_frequency_ms=60000")
   def test_retry_query_result_cacheing_failed(self):