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/07/14 01:49:13 UTC

[impala] 02/03: IMPALA-10766: Better selectivity for =, not distinct

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

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

commit 4c5fa0591706ec1399a6b92ab10e7028ad159aef
Author: liuyao <54...@163.com>
AuthorDate: Fri Jun 25 17:35:24 2021 +0800

    IMPALA-10766: Better selectivity for =,not distinct
    
    For = :
    If the right side is null, then selectivity is 0.
    If the left side is null, null should be excluded when calculating
    selectivity.
    
    For is not distinct from :
    If the right side is null, non null should be excluded when calculating
    selectivity, and only null should be included.
    If the left side is null and the right side is not null, null should be
    excluded when calculating selectivity, including part of non-null.
    
    Testing :
    Change the UT, modify the selectivity calculation error, add two new
    cases column != null and column = null
    
    Change-Id: Ib8ec62f2355a7036125cc0d261b790644b9f4b60
    Reviewed-on: http://gerrit.cloudera.org:8080/17637
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Qifan Chen <qc...@cloudera.com>
---
 .../apache/impala/analysis/BinaryPredicate.java    |  40 +++-
 .../impala/analysis/ExprCardinalityTest.java       |  43 ++--
 .../org/apache/impala/planner/CardinalityTest.java |   6 +-
 .../queries/PlannerTest/card-inner-join.test       |   4 +-
 .../queries/PlannerTest/fk-pk-join-detection.test  |  10 +-
 .../queries/PlannerTest/joins.test                 |  28 +--
 .../queries/PlannerTest/tpcds/tpcds-q08.test       |  24 +-
 .../queries/PlannerTest/tpcds/tpcds-q13.test       |  56 ++---
 .../queries/PlannerTest/tpcds/tpcds-q16.test       |  62 ++---
 .../queries/PlannerTest/tpcds/tpcds-q19.test       |  80 +++----
 .../queries/PlannerTest/tpcds/tpcds-q24a.test      | 122 +++++-----
 .../queries/PlannerTest/tpcds/tpcds-q24b.test      | 122 +++++-----
 .../queries/PlannerTest/tpcds/tpcds-q30.test       |  32 +--
 .../queries/PlannerTest/tpcds/tpcds-q33.test       | 126 +++++-----
 .../queries/PlannerTest/tpcds/tpcds-q42.test       |  40 ++--
 .../queries/PlannerTest/tpcds/tpcds-q44.test       |  24 +-
 .../queries/PlannerTest/tpcds/tpcds-q48.test       |  52 ++---
 .../queries/PlannerTest/tpcds/tpcds-q52.test       |  44 ++--
 .../queries/PlannerTest/tpcds/tpcds-q55.test       |  44 ++--
 .../queries/PlannerTest/tpcds/tpcds-q56.test       | 214 ++++++++---------
 .../queries/PlannerTest/tpcds/tpcds-q60.test       | 216 ++++++++---------
 .../queries/PlannerTest/tpcds/tpcds-q61.test       | 104 ++++-----
 .../queries/PlannerTest/tpcds/tpcds-q66.test       | 104 ++++-----
 .../queries/PlannerTest/tpcds/tpcds-q71.test       |  68 +++---
 .../queries/PlannerTest/tpcds/tpcds-q75.test       | 258 ++++++++++-----------
 .../queries/PlannerTest/tpcds/tpcds-q80.test       |  84 +++----
 .../queries/PlannerTest/tpcds/tpcds-q81.test       |  48 ++--
 .../queries/PlannerTest/tpcds/tpcds-q84.test       |  64 ++---
 .../queries/PlannerTest/tpcds/tpcds-q85.test       |  48 ++--
 .../queries/PlannerTest/tpcds/tpcds-q91.test       |  82 +++----
 .../queries/PlannerTest/tpcds/tpcds-q94.test       |  72 +++---
 .../queries/PlannerTest/tpcds/tpcds-q95.test       |  68 +++---
 32 files changed, 1203 insertions(+), 1186 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 83ab86c..0eb56c7 100644
--- a/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java
+++ b/fe/src/main/java/org/apache/impala/analysis/BinaryPredicate.java
@@ -241,13 +241,16 @@ public class BinaryPredicate extends Predicate {
     if (!isSingleColumnPredicate(slotRefRef, null)) {
       return;
     }
+    boolean rChildIsNull = Expr.IS_NULL_LITERAL.apply(getChild(1));
     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)) {
+    } else if ((distinctValues == 0 && (op_ == Operator.EQ || op_ == Operator.NE))
+        || (rChildIsNull && (op_ == Operator.EQ || op_ == Operator.NE))) {
       // If the table is empty, then distinctValues is 0. This case is equivalent
       // to selectivity is 0.
+      // For case <column> = NULL or <column> != NULL, all values are false
       selectivity_ = 0.0;
       return;
     }
@@ -256,23 +259,38 @@ public class BinaryPredicate extends Predicate {
       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) {
+      if (op_ == Operator.DISTINCT_FROM && rChildIsNull) {
         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;
     }
+
+    SlotDescriptor slotDesc = slotRefRef.getRef().getDesc();
+    if (slotDesc.getStats().hasNullsStats()) {
+      FeTable table = slotDesc.getParent().getTable();
+      if (table != null && table.getNumRows() > 0) {
+        long numRows = table.getNumRows();
+        long numNulls = slotDesc.getStats().getNumNulls();
+        if (op_ == Operator.EQ || op_ == Operator.NE
+            || (op_ == Operator.DISTINCT_FROM && rChildIsNull)
+            || (op_ == Operator.NOT_DISTINCT && !rChildIsNull)) {
+          // For =, !=, "is distinct from null" and "is not distinct from non-null",
+          // all null values are false.
+          selectivity_ *= (double) (numRows - numNulls) / numRows;
+        } else if (op_ == Operator.NOT_DISTINCT && rChildIsNull) {
+          // For is not distinct from null, only null values are true
+          selectivity_ = numNulls / numRows;
+        } else if (op_ == Operator.DISTINCT_FROM && !rChildIsNull) {
+          // For is distinct from not-null, null values are true, So need to add it
+          selectivity_ = selectivity_ * (double) (numRows - numNulls) / numRows +
+              numNulls / numRows;
+        }
+      }
+    }
+
     selectivity_ = Math.max(0, Math.min(1, selectivity_));
   }
 
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 efd0378..3bd408a 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ExprCardinalityTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ExprCardinalityTest.java
@@ -263,11 +263,11 @@ public class ExprCardinalityTest {
     verifySelectExpr("alltypes", "int_col = 10", 3, 1.0/10);
 
     verifySelectExpr("nullrows", "id = 'foo'", 3, 1.0/26);
-    verifySelectExpr("nullrows", "null_str = 'foo'", 3, 1.0/1);
+    verifySelectExpr("nullrows", "null_str = 'foo'", 3, 0);
     verifySelectExpr("nullrows", "group_str = 'foo'", 3, 1.0/6);
-    // Bug: nulls should count to NDV
-    //verifySelectExpr("nullrows", "some_nulls = 'foo'", 3, 1.0/7);
-    verifySelectExpr("nullrows", "some_nulls = 'foo'", 3, 1.0/6);
+
+    verifySelectExpr("nullrows", "some_nulls = 'foo'", 3, 1.0/6 * 6/26);
+    verifySelectExpr("nullrows", "some_nulls = null", 3, 0);
 
     // Bug: Sel should default to good old 0.1
     verifySelectExpr("manynulls", "id = 10", 3, -1);
@@ -282,28 +282,25 @@ public class ExprCardinalityTest {
   @Test
   public void testNotDistinctSelectivity() throws ImpalaException {
     verifySelectExpr("alltypes", "id is not distinct from 10", 3, 1.0/7300);
-    // Bug: does not treat NULL specially
-    // Bug: NDV sould be 2 since IS NOT DISTINCT won't return NULL
-    //verifySelectExpr("alltypes", "id is not distinct from null", 2, 0);
-    verifySelectExpr("alltypes", "id is not distinct from null", 3, 1.0/7300);
+
+    verifySelectExpr("alltypes", "id is not distinct from null", 3, 0);
     verifySelectExpr("alltypes", "bool_col is not distinct from true", 3, 1.0/2);
-    //verifySelectExpr("alltypes", "bool_col is not distinct from null", 2, 0);
-    verifySelectExpr("alltypes", "bool_col is not distinct from null", 3, 1.0/2);
+    verifySelectExpr("alltypes", "bool_col is not distinct from null", 3, 0);
     verifySelectExpr("alltypes", "int_col is not distinct from 10", 3, 1.0/10);
-    //verifySelectExpr("alltypes", "int_col is not distinct from null", 2, 0);
-    verifySelectExpr("alltypes", "int_col is not distinct from null", 3, 1.0/10);
+
+    verifySelectExpr("alltypes", "int_col is not distinct from null", 3, 0);
 
     verifySelectExpr("nullrows", "id is not distinct from 'foo'", 3, 1.0/26);
-    //verifySelectExpr("nullrows", "id is not distinct from null", 2, 0);
-    verifySelectExpr("nullrows", "id is not distinct from null", 3, 1.0/26);
-    // Bug: All nulls, so NDV should = 1, so Sel should be 1.0/1
-    verifySelectExpr("nullrows", "null_str is not distinct from 'foo'", 3, 1.0/1);
+
+    verifySelectExpr("nullrows", "id is not distinct from null", 3, 0);
+
+    verifySelectExpr("nullrows", "null_str is not distinct from 'foo'", 3, 0);
     verifySelectExpr("nullrows", "null_str is not distinct from null", 3, 1.0/1);
     verifySelectExpr("nullrows", "group_str is not distinct from 'foo'", 3, 1.0/6);
-    //verifySelectExpr("nullrows", "group_str is not distinct from null", 2, 1);
-    verifySelectExpr("nullrows", "group_str is not distinct from null", 3, 1.0/6);
-    //verifySelectExpr("nullrows", "some_nulls is not distinct from 'foo'", 3, 1.0/7);
-    verifySelectExpr("nullrows", "some_nulls is not distinct from 'foo'", 3, 1.0/6);
+
+    verifySelectExpr("nullrows", "group_str is not distinct from null", 3, 0);
+
+    verifySelectExpr("nullrows", "some_nulls is not distinct from 'foo'", 3, 1.0/6*6/26);
 
     // Bug: Sel should default to good old 0.1
     verifySelectExpr("manynulls", "id is not distinct from 10", 3, -1);
@@ -321,6 +318,7 @@ public class ExprCardinalityTest {
     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);
+    verifySelectExpr("nullrows", "some_nulls != null", 3, 0);
     // field has no statistics.
     verifySelectExpr("emptytable", "field != 'foo'", 3, -1);
     verifySelectExpr("emptytable", "f2 != 10", 3, 0.0);
@@ -351,12 +349,13 @@ public class ExprCardinalityTest {
     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);
+    // For is distinct from non-null, all null values are true
+    verifySelectExpr("nullrows", "null_str is distinct from 'foo'", 3, 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 not distinct from 'foo'", 3, 1.0/6*6/26);
     verifySelectExpr("nullrows", "some_nulls is distinct from null", 3, 6.0/26.0);
 
     // Bug: Sel should default to 1 - good old 0.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 91fc296..328b897 100644
--- a/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java
@@ -115,9 +115,11 @@ public class CardinalityTest extends PlannerTestBase {
     // f repeats for 5 rows, so NDV=7, 26/7 =~ 4
     verifyCardinality("SELECT null_int FROM functional.nullrows WHERE group_str = 'x'",
         4);
-    // null_str is all nulls, NDV = 1, selectivity = 1/1, cardinality = 26
+    // null_str is all nulls, NDV = 1, selectivity =0, cardinality = 0
+    // PlanNode#applySelectivity, If cardinality is 0, set cardinality to 1
+    // IMPALA-8647: don't round cardinality down to zero for safety.
     verifyCardinality(
-          "SELECT null_int FROM functional.nullrows WHERE null_str = 'x'", 26);
+          "SELECT null_int FROM functional.nullrows WHERE null_str = 'x'", 1);
   }
 
   @Test
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 f609ec8..783c276 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/card-inner-join.test
@@ -946,7 +946,7 @@ PLAN-ROOT SINK
 |  hash predicates: a.id = b.id
 |  other predicates: a.int_col + b.int_col > 30
 |  runtime filters: RF000 <- b.id
-|  row-size=20B cardinality=113
+|  row-size=20B cardinality=111
 |
 |--01:SCAN HDFS [functional.alltypesnopart b]
 |     partitions=1/1 files=0 size=0B
@@ -957,7 +957,7 @@ PLAN-ROOT SINK
    partitions=11/11 files=11 size=814.73KB
    predicates: a.smallint_col = 10
    runtime filters: RF000 -> a.id
-   row-size=10B cardinality=113
+   row-size=10B cardinality=111
 ====
 # Join on a computed column
 # Assumes Cartesian product * 0.1
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
index 720d423..75eb17b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
@@ -15,7 +15,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF000[bloom] <- c_customer_sk
 |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,1 row-size=319B cardinality=529.70K
+|  tuple-ids=0,1 row-size=319B cardinality=511.62K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.customer]
@@ -26,7 +26,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=1 row-size=219B cardinality=16.67K
+|     tuple-ids=1 row-size=219B cardinality=16.10K
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -70,7 +70,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=1 row-size=219B cardinality=16.67K
+|     tuple-ids=1 row-size=219B cardinality=16.10K
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -102,7 +102,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF000[bloom] <- c_customer_sk
 |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0N,1 row-size=319B cardinality=529.70K
+|  tuple-ids=0N,1 row-size=319B cardinality=511.62K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.customer]
@@ -113,7 +113,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=1 row-size=219B cardinality=16.67K
+|     tuple-ids=1 row-size=219B cardinality=16.10K
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
index f2adf2b..c4431d0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
@@ -175,7 +175,7 @@ PLAN-ROOT SINK
    HDFS partitions=5/11 files=5 size=372.38KB
    predicates: a.tinyint_col = 15
    runtime filters: RF000 -> a.id, RF001 -> a.int_col
-   row-size=95B cardinality=556
+   row-size=95B cardinality=455
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
@@ -202,7 +202,7 @@ PLAN-ROOT SINK
    HDFS partitions=5/11 files=5 size=372.38KB
    predicates: a.tinyint_col = 15
    runtime filters: RF000 -> a.id, RF001 -> a.int_col
-   row-size=95B cardinality=556
+   row-size=95B cardinality=455
 ====
 # same as before, with 3 tables;
 # non-eq join predicates are evaluated at the correct join node
@@ -228,7 +228,7 @@ PLAN-ROOT SINK
 |
 |--03:HASH JOIN [FULL OUTER JOIN]
 |  |  hash predicates: a.id = b.id, a.int_col = b.int_col
-|  |  row-size=184B cardinality=561
+|  |  row-size=184B cardinality=460
 |  |
 |  |--01:SCAN HDFS [functional.alltypessmall b]
 |  |     partition predicates: b.`month` > 2
@@ -240,7 +240,7 @@ PLAN-ROOT SINK
 |     partition predicates: a.`day` >= 6
 |     HDFS partitions=5/11 files=5 size=372.38KB
 |     predicates: a.tinyint_col = 15
-|     row-size=95B cardinality=556
+|     row-size=95B cardinality=455
 |
 02:SCAN HDFS [functional.alltypesaggnonulls c]
    partition predicates: c.`day` < 3
@@ -268,7 +268,7 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
 |  hash predicates: a.id = b.id, a.int_col = b.int_col
-|  row-size=184B cardinality=561
+|  row-size=184B cardinality=460
 |
 |--06:EXCHANGE [HASH(b.id,b.int_col)]
 |  |
@@ -286,7 +286,7 @@ PLAN-ROOT SINK
    HDFS partitions=5/11 files=5 size=372.38KB
    predicates: a.tinyint_col = 15
    runtime filters: RF000 -> a.id
-   row-size=95B cardinality=556
+   row-size=95B cardinality=455
 ====
 # equi join with constants in the on clause are not supported
 select a.id, b.id from
@@ -417,7 +417,7 @@ PLAN-ROOT SINK
    HDFS partitions=5/11 files=5 size=372.38KB
    predicates: a.tinyint_col = 15
    runtime filters: RF000 -> a.int_col, RF001 -> a.id
-   row-size=95B cardinality=556
+   row-size=95B cardinality=455
 ---- SCANRANGELOCATIONS
 NODE 0:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypesagg/year=2010/month=1/day=10/100110.txt 0:76263
@@ -452,7 +452,7 @@ PLAN-ROOT SINK
    HDFS partitions=5/11 files=5 size=372.38KB
    predicates: a.tinyint_col = 15
    runtime filters: RF000 -> a.int_col, RF001 -> a.id
-   row-size=95B cardinality=556
+   row-size=95B cardinality=455
 ====
 # hbase-hdfs join with scan filtering (bogus)
 select *
@@ -480,7 +480,7 @@ PLAN-ROOT SINK
    HDFS partitions=5/11 files=5 size=372.38KB
    predicates: a.tinyint_col = 15
    runtime filters: RF000 -> a.int_col, RF001 -> a.id
-   row-size=95B cardinality=556
+   row-size=95B cardinality=455
 ---- SCANRANGELOCATIONS
 NODE 0:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypesagg/year=2010/month=1/day=10/100110.txt 0:76263
@@ -2533,7 +2533,7 @@ PLAN-ROOT SINK
 |
 06:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: a.bigint_col > c.bigint_col
-|  row-size=192B cardinality=1.22K
+|  row-size=192B cardinality=1.00K
 |
 |--02:NESTED LOOP JOIN [INNER JOIN]
 |  |  predicates: a.id <= b.id
@@ -2550,7 +2550,7 @@ PLAN-ROOT SINK
 |
 05:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: c.int_col != d.int_col
-|  row-size=99B cardinality=1.22K
+|  row-size=99B cardinality=1.00K
 |
 |--04:SCAN HDFS [functional.alltypes d]
 |     HDFS partitions=24/24 files=24 size=478.45KB
@@ -2559,7 +2559,7 @@ PLAN-ROOT SINK
 03:SCAN HDFS [functional.alltypesagg c]
    HDFS partitions=11/11 files=11 size=814.73KB
    predicates: c.tinyint_col = 10
-   row-size=95B cardinality=1.22K
+   row-size=95B cardinality=1.00K
 ====
 # Inner, left-outer, left-semi and left-anti joins with non-equi join
 # predicates
@@ -2761,7 +2761,7 @@ PLAN-ROOT SINK
 |
 |--03:HASH JOIN [FULL OUTER JOIN]
 |  |  hash predicates: a.id IS NOT DISTINCT FROM b.id, a.int_col = b.int_col
-|  |  row-size=184B cardinality=561
+|  |  row-size=184B cardinality=460
 |  |
 |  |--01:SCAN HDFS [functional.alltypessmall b]
 |  |     partition predicates: b.`month` > 2
@@ -2773,7 +2773,7 @@ PLAN-ROOT SINK
 |     partition predicates: a.`day` >= 6
 |     HDFS partitions=5/11 files=5 size=372.38KB
 |     predicates: a.tinyint_col = 15
-|     row-size=95B cardinality=556
+|     row-size=95B cardinality=455
 |
 02:SCAN HDFS [functional.alltypesaggnonulls c]
    partition predicates: c.`day` < 3
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
index f7290e3..dfe3d66 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
@@ -126,7 +126,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: none
 |  |  |  runtime filters: RF006[bloom] <- c_current_addr_sk, RF007[min_max] <- c_current_addr_sk
 |  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  |  |  tuple-ids=6,7 row-size=38B cardinality=51.30K
+|  |  |  tuple-ids=6,7 row-size=38B cardinality=49.54K
 |  |  |  in pipelines: 05(GETNEXT), 06(OPEN)
 |  |  |
 |  |  |--06:SCAN HDFS [tpcds_parquet.customer]
@@ -139,7 +139,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: c_preferred_cust_flag = 'Y'
 |  |  |     parquet dictionary predicates: c_preferred_cust_flag = 'Y'
 |  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=7 row-size=17B cardinality=50.00K
+|  |  |     tuple-ids=7 row-size=17B cardinality=48.29K
 |  |  |     in pipelines: 06(GETNEXT)
 |  |  |
 |  |  05:SCAN HDFS [tpcds_parquet.customer_address]
@@ -302,7 +302,7 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat
 |  |  |  in pipelines: 05(GETNEXT)
 |  |  |
 |  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Host Resources: mem-estimate=48.58MB mem-reservation=8.25MB thread-reservation=2 runtime-filters-memory=1.00MB
+|  |  Per-Host Resources: mem-estimate=48.55MB mem-reservation=8.25MB thread-reservation=2 runtime-filters-memory=1.00MB
 |  |  08:AGGREGATE [STREAMING]
 |  |  |  output: count(*)
 |  |  |  group by: substring(ca_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT))
@@ -315,12 +315,12 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat
 |  |  |  fk/pk conjuncts: none
 |  |  |  runtime filters: RF006[bloom] <- c_current_addr_sk, RF007[min_max] <- c_current_addr_sk
 |  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  |  |  tuple-ids=6,7 row-size=38B cardinality=51.30K
+|  |  |  tuple-ids=6,7 row-size=38B cardinality=49.54K
 |  |  |  in pipelines: 05(GETNEXT), 06(OPEN)
 |  |  |
 |  |  |--19:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=851.08KB mem-reservation=0B thread-reservation=0
-|  |  |  |  tuple-ids=7 row-size=17B cardinality=50.00K
+|  |  |  |  mem-estimate=822.64KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=7 row-size=17B cardinality=48.29K
 |  |  |  |  in pipelines: 06(GETNEXT)
 |  |  |  |
 |  |  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -335,7 +335,7 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat
 |  |  |     parquet statistics predicates: c_preferred_cust_flag = 'Y'
 |  |  |     parquet dictionary predicates: c_preferred_cust_flag = 'Y'
 |  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=7 row-size=17B cardinality=50.00K
+|  |  |     tuple-ids=7 row-size=17B cardinality=48.29K
 |  |  |     in pipelines: 06(GETNEXT)
 |  |  |
 |  |  05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM]
@@ -556,11 +556,11 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  |  |  hash predicates: ca_address_sk = c_current_addr_sk
 |  |  |  fk/pk conjuncts: none
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  |  |  tuple-ids=6,7 row-size=38B cardinality=51.30K
+|  |  |  tuple-ids=6,7 row-size=38B cardinality=49.54K
 |  |  |  in pipelines: 05(GETNEXT), 06(OPEN)
 |  |  |
 |  |  |--F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  |  |  Per-Instance Resources: mem-estimate=11.33MB mem-reservation=10.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  Per-Instance Resources: mem-estimate=11.30MB mem-reservation=10.50MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: c_current_addr_sk
@@ -568,8 +568,8 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  |  |  |  mem-estimate=9.50MB mem-reservation=9.50MB spill-buffer=256.00KB thread-reservation=0
 |  |  |  |
 |  |  |  19:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=851.08KB mem-reservation=0B thread-reservation=0
-|  |  |  |  tuple-ids=7 row-size=17B cardinality=50.00K
+|  |  |  |  mem-estimate=822.64KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=7 row-size=17B cardinality=48.29K
 |  |  |  |  in pipelines: 06(GETNEXT)
 |  |  |  |
 |  |  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -584,7 +584,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  |  |     parquet statistics predicates: c_preferred_cust_flag = 'Y'
 |  |  |     parquet dictionary predicates: c_preferred_cust_flag = 'Y'
 |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |  |     tuple-ids=7 row-size=17B cardinality=50.00K
+|  |  |     tuple-ids=7 row-size=17B cardinality=48.29K
 |  |  |     in pipelines: 06(GETNEXT)
 |  |  |
 |  |  05:SCAN HDFS [tpcds_parquet.customer_address, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
index 891583c..bcd85dd 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
@@ -67,7 +67,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF000[bloom] <- s_store_sk, RF001[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=2,0,4,5,3,1 row-size=142B cardinality=2.55K
+|  tuple-ids=2,0,4,5,3,1 row-size=142B cardinality=2.51K
 |  in pipelines: 02(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds_parquet.store]
@@ -86,7 +86,7 @@ PLAN-ROOT SINK
 |  other predicates: cd_education_status = 'Advanced Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), cd_marital_status = 'M' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), hd_dep_count = CAST(3 AS INT) OR cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), cd_education_status = 'Advanced Degree' OR hd_dep_count = CAST(1 AS INT) OR hd_dep_count = CAST(1 AS INT), hd_dep_ [...]
 |  runtime filters: RF002[bloom] <- ss_cdemo_sk, RF003[min_max] <- ss_cdemo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=2,0,4,5,3 row-size=138B cardinality=2.55K
+|  tuple-ids=2,0,4,5,3 row-size=138B cardinality=2.51K
 |  in pipelines: 02(GETNEXT), 00(OPEN)
 |
 |--08:HASH JOIN [INNER JOIN]
@@ -95,7 +95,7 @@ PLAN-ROOT SINK
 |  |  other predicates: hd_dep_count IN (CAST(3 AS INT), CAST(1 AS INT)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), hd_dep_count IN (CAST(3 AS INT), CAST(1 AS INT)) OR ss_sales_price >= CAST(150.00 AS DECIMAL(5,2)), hd_dep_count = CAST(3 AS INT) OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR hd_dep_count = CAST(1 AS INT), hd_dep_count = CAST(3 AS INT) OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), hd_dep_count = CAST(3 A [...]
 |  |  runtime filters: RF004[bloom] <- hd_demo_sk, RF005[min_max] <- hd_demo_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,4,5,3 row-size=99B cardinality=3.05K
+|  |  tuple-ids=0,4,5,3 row-size=99B cardinality=3.01K
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--03:SCAN HDFS [tpcds_parquet.household_demographics]
@@ -116,7 +116,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  runtime filters: RF006[bloom] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,4,5 row-size=91B cardinality=10.43K
+|  |  tuple-ids=0,4,5 row-size=91B cardinality=10.26K
 |  |  in pipelines: 00(GETNEXT), 05(OPEN)
 |  |
 |  |--05:SCAN HDFS [tpcds_parquet.date_dim]
@@ -138,7 +138,7 @@ PLAN-ROOT SINK
 |  |  other predicates: ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ss_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit >=  [...]
 |  |  runtime filters: RF008[bloom] <- ca_address_sk, RF009[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,4 row-size=83B cardinality=50.98K
+|  |  tuple-ids=0,4 row-size=83B cardinality=50.19K
 |  |  in pipelines: 00(GETNEXT), 04(OPEN)
 |  |
 |  |--04:SCAN HDFS [tpcds_parquet.customer_address]
@@ -151,7 +151,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_state IN ('TX', 'OH', 'TX', 'OR', 'NM', 'KY', 'VA', 'TX', 'MS'), ca_country = 'United States'
 |  |     parquet dictionary predicates: ca_state IN ('TX', 'OH', 'TX', 'OR', 'NM', 'KY', 'VA', 'TX', 'MS'), ca_country = 'United States'
 |  |     mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=4 row-size=43B cardinality=8.82K
+|  |     tuple-ids=4 row-size=43B cardinality=8.69K
 |  |     in pipelines: 04(GETNEXT)
 |  |
 |  00:SCAN HDFS [tpcds_parquet.store_sales]
@@ -212,7 +212,7 @@ Per-Host Resources: mem-estimate=19.61MB mem-reservation=12.44MB thread-reservat
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF000[bloom] <- s_store_sk, RF001[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,4,5,3,2,1 row-size=142B cardinality=2.55K
+|  tuple-ids=0,4,5,3,2,1 row-size=142B cardinality=2.51K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--17:EXCHANGE [BROADCAST]
@@ -238,7 +238,7 @@ Per-Host Resources: mem-estimate=19.61MB mem-reservation=12.44MB thread-reservat
 |  other predicates: cd_education_status = 'Advanced Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), cd_marital_status = 'M' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), hd_dep_count = CAST(3 AS INT) OR cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), cd_education_status = 'Advanced Degree' OR hd_dep_count = CAST(1 AS INT) OR hd_dep_count = CAST(1 AS INT), hd_dep_ [...]
 |  runtime filters: RF002[bloom] <- cd_demo_sk, RF003[min_max] <- cd_demo_sk
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,4,5,3,2 row-size=138B cardinality=2.55K
+|  tuple-ids=0,4,5,3,2 row-size=138B cardinality=2.51K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--16:EXCHANGE [HASH(cd_demo_sk)]
@@ -260,8 +260,8 @@ Per-Host Resources: mem-estimate=19.61MB mem-reservation=12.44MB thread-reservat
 |     in pipelines: 02(GETNEXT)
 |
 15:EXCHANGE [HASH(ss_cdemo_sk)]
-|  mem-estimate=443.39KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,4,5,3 row-size=99B cardinality=3.05K
+|  mem-estimate=441.87KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,4,5,3 row-size=99B cardinality=3.01K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -272,7 +272,7 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva
 |  other predicates: hd_dep_count IN (CAST(3 AS INT), CAST(1 AS INT)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), hd_dep_count IN (CAST(3 AS INT), CAST(1 AS INT)) OR ss_sales_price >= CAST(150.00 AS DECIMAL(5,2)), hd_dep_count = CAST(3 AS INT) OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR hd_dep_count = CAST(1 AS INT), hd_dep_count = CAST(3 AS INT) OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), hd_dep_count = CAST(3 AS I [...]
 |  runtime filters: RF004[bloom] <- hd_demo_sk, RF005[min_max] <- hd_demo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,4,5,3 row-size=99B cardinality=3.05K
+|  tuple-ids=0,4,5,3 row-size=99B cardinality=3.01K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--14:EXCHANGE [BROADCAST]
@@ -300,7 +300,7 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF006[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,4,5 row-size=91B cardinality=10.43K
+|  tuple-ids=0,4,5 row-size=91B cardinality=10.26K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--13:EXCHANGE [BROADCAST]
@@ -329,12 +329,12 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva
 |  other predicates: ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ss_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit >= CAS [...]
 |  runtime filters: RF008[bloom] <- ca_address_sk, RF009[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,4 row-size=83B cardinality=50.98K
+|  tuple-ids=0,4 row-size=83B cardinality=50.19K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--12:EXCHANGE [BROADCAST]
-|  |  mem-estimate=417.54KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=4 row-size=43B cardinality=8.82K
+|  |  mem-estimate=411.83KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=4 row-size=43B cardinality=8.69K
 |  |  in pipelines: 04(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -349,7 +349,7 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva
 |     parquet statistics predicates: ca_state IN ('TX', 'OH', 'TX', 'OR', 'NM', 'KY', 'VA', 'TX', 'MS'), ca_country = 'United States'
 |     parquet dictionary predicates: ca_state IN ('TX', 'OH', 'TX', 'OR', 'NM', 'KY', 'VA', 'TX', 'MS'), ca_country = 'United States'
 |     mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=4 row-size=43B cardinality=8.82K
+|     tuple-ids=4 row-size=43B cardinality=8.69K
 |     in pipelines: 04(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
@@ -367,7 +367,7 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=56.83MB Threads=16
-Per-Host Resource Estimates: Memory=168MB
+Per-Host Resource Estimates: Memory=167MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
@@ -386,7 +386,7 @@ PLAN-ROOT SINK
 |  in pipelines: 11(GETNEXT)
 |
 F05:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=804.39KB mem-reservation=0B thread-reservation=1
+Per-Instance Resources: mem-estimate=802.87KB mem-reservation=0B thread-reservation=1
 11:AGGREGATE
 |  output: avg(CAST(ss_quantity AS BIGINT)), avg(ss_ext_sales_price), avg(ss_ext_wholesale_cost), sum(ss_ext_wholesale_cost)
 |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -398,7 +398,7 @@ Per-Instance Resources: mem-estimate=804.39KB mem-reservation=0B thread-reservat
 |  hash predicates: ss_store_sk = s_store_sk
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,4,5,3,2,1 row-size=142B cardinality=2.55K
+|  tuple-ids=0,4,5,3,2,1 row-size=142B cardinality=2.51K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--F08:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=3
@@ -432,7 +432,7 @@ Per-Instance Resources: mem-estimate=804.39KB mem-reservation=0B thread-reservat
 |  fk/pk conjuncts: ss_cdemo_sk = cd_demo_sk
 |  other predicates: cd_education_status = 'Advanced Degree' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), cd_marital_status = 'M' OR cd_marital_status = 'S' AND cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), hd_dep_count = CAST(3 AS INT) OR cd_education_status = 'College' OR hd_dep_count = CAST(1 AS INT), cd_education_status = 'Advanced Degree' OR hd_dep_count = CAST(1 AS INT) OR hd_dep_count = CAST(1 AS INT), hd_dep_ [...]
 |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,4,5,3,2 row-size=138B cardinality=2.55K
+|  tuple-ids=0,4,5,3,2 row-size=138B cardinality=2.51K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--F09:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=6
@@ -462,8 +462,8 @@ Per-Instance Resources: mem-estimate=804.39KB mem-reservation=0B thread-reservat
 |     in pipelines: 02(GETNEXT)
 |
 15:EXCHANGE [HASH(ss_cdemo_sk)]
-|  mem-estimate=788.39KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,4,5,3 row-size=99B cardinality=3.05K
+|  mem-estimate=786.87KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,4,5,3 row-size=99B cardinality=3.01K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -475,7 +475,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  fk/pk conjuncts: ss_hdemo_sk = hd_demo_sk
 |  other predicates: hd_dep_count IN (CAST(3 AS INT), CAST(1 AS INT)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), hd_dep_count IN (CAST(3 AS INT), CAST(1 AS INT)) OR ss_sales_price >= CAST(150.00 AS DECIMAL(5,2)), hd_dep_count = CAST(3 AS INT) OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR hd_dep_count = CAST(1 AS INT), hd_dep_count = CAST(3 AS INT) OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), hd_dep_count = CAST(3 AS I [...]
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,4,5,3 row-size=99B cardinality=3.05K
+|  tuple-ids=0,4,5,3 row-size=99B cardinality=3.01K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -511,7 +511,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.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=0,4,5 row-size=91B cardinality=10.43K
+|  tuple-ids=0,4,5 row-size=91B cardinality=10.26K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -548,7 +548,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  other predicates: ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ss_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('TX', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit >= CAS [...]
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,4 row-size=83B cardinality=50.98K
+|  tuple-ids=0,4 row-size=83B cardinality=50.19K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -560,8 +560,8 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  12:EXCHANGE [BROADCAST]
-|  |  mem-estimate=417.54KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=4 row-size=43B cardinality=8.82K
+|  |  mem-estimate=411.83KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=4 row-size=43B cardinality=8.69K
 |  |  in pipelines: 04(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -576,7 +576,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |     parquet statistics predicates: ca_state IN ('TX', 'OH', 'TX', 'OR', 'NM', 'KY', 'VA', 'TX', 'MS'), ca_country = 'United States'
 |     parquet dictionary predicates: ca_state IN ('TX', 'OH', 'TX', 'OR', 'NM', 'KY', 'VA', 'TX', 'MS'), ca_country = 'United States'
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=4 row-size=43B cardinality=8.82K
+|     tuple-ids=4 row-size=43B cardinality=8.69K
 |     in pipelines: 04(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
index 29a5446..637e5d6 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
@@ -48,13 +48,13 @@ PLAN-ROOT SINK
 |  output: sum(cs_ext_ship_cost), sum(cs_net_profit)
 |  group by: cs_order_number
 |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=8 row-size=40B cardinality=30.02K
+|  tuple-ids=8 row-size=40B cardinality=29.10K
 |  in pipelines: 11(GETNEXT), 04(OPEN)
 |
 10:HASH JOIN [LEFT ANTI JOIN]
 |  hash predicates: cs1.cs_order_number = cr1.cr_order_number
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  in pipelines: 04(GETNEXT), 05(OPEN)
 |
 |--05:SCAN HDFS [tpcds_parquet.catalog_returns cr1]
@@ -72,7 +72,7 @@ PLAN-ROOT SINK
 |  other join predicates: cs1.cs_warehouse_sk != cs2.cs_warehouse_sk
 |  runtime filters: RF000[bloom] <- cs1.cs_order_number, RF001[min_max] <- cs1.cs_order_number
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 |--08:HASH JOIN [INNER JOIN]
@@ -80,7 +80,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs1.cs_call_center_sk = cc_call_center_sk
 |  |  runtime filters: RF002[bloom] <- cc_call_center_sk, RF003[min_max] <- cc_call_center_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--03:SCAN HDFS [tpcds_parquet.call_center]
@@ -101,7 +101,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs1.cs_ship_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=0,2,1 row-size=76B cardinality=30.02K
+|  |  tuple-ids=0,2,1 row-size=76B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |
 |  |--01:SCAN HDFS [tpcds_parquet.date_dim]
@@ -122,7 +122,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs1.cs_ship_addr_sk = ca_address_sk
 |  |  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2 row-size=50B cardinality=30.02K
+|  |  tuple-ids=0,2 row-size=50B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |
 |  |--02:SCAN HDFS [tpcds_parquet.customer_address]
@@ -135,7 +135,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_state = 'GA'
 |  |     parquet dictionary predicates: ca_state = 'GA'
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=18B cardinality=980
+|  |     tuple-ids=2 row-size=18B cardinality=950
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  00:SCAN HDFS [tpcds_parquet.catalog_sales cs1]
@@ -197,13 +197,13 @@ Per-Host Resources: mem-estimate=24.17MB mem-reservation=10.56MB thread-reservat
 |  output: sum(cs_ext_ship_cost), sum(cs_net_profit)
 |  group by: cs_order_number
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=40B cardinality=30.02K
+|  tuple-ids=8 row-size=40B cardinality=29.10K
 |  in pipelines: 11(GETNEXT), 04(OPEN)
 |
 10:HASH JOIN [LEFT ANTI JOIN, PARTITIONED]
 |  hash predicates: cs1.cs_order_number = cr1.cr_order_number
 |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  in pipelines: 04(GETNEXT), 05(OPEN)
 |
 |--19:EXCHANGE [HASH(cr1.cr_order_number)]
@@ -228,22 +228,22 @@ Per-Host Resources: mem-estimate=24.17MB mem-reservation=10.56MB thread-reservat
 |  other join predicates: cs1.cs_warehouse_sk != cs2.cs_warehouse_sk
 |  runtime filters: RF000[bloom] <- cs1.cs_order_number, RF001[min_max] <- cs1.cs_order_number
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 |--18:EXCHANGE [HASH(cs1.cs_order_number)]
-|  |  mem-estimate=1.41MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  |  mem-estimate=1.37MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=345.08MB mem-reservation=24.81MB thread-reservation=2 runtime-filters-memory=3.00MB
+|  Per-Host Resources: mem-estimate=345.07MB mem-reservation=24.81MB thread-reservation=2 runtime-filters-memory=3.00MB
 |  08:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: cs1.cs_call_center_sk = cc_call_center_sk
 |  |  fk/pk conjuncts: cs1.cs_call_center_sk = cc_call_center_sk
 |  |  runtime filters: RF002[bloom] <- cc_call_center_sk, RF003[min_max] <- cc_call_center_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--16:EXCHANGE [BROADCAST]
@@ -271,7 +271,7 @@ Per-Host Resources: mem-estimate=24.17MB mem-reservation=10.56MB thread-reservat
 |  |  fk/pk conjuncts: cs1.cs_ship_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=0,2,1 row-size=76B cardinality=30.02K
+|  |  tuple-ids=0,2,1 row-size=76B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |
 |  |--15:EXCHANGE [BROADCAST]
@@ -299,12 +299,12 @@ Per-Host Resources: mem-estimate=24.17MB mem-reservation=10.56MB thread-reservat
 |  |  fk/pk conjuncts: cs1.cs_ship_addr_sk = ca_address_sk
 |  |  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2 row-size=50B cardinality=30.02K
+|  |  tuple-ids=0,2 row-size=50B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |
 |  |--14:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=38.28KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=18B cardinality=980
+|  |  |  mem-estimate=37.11KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=2 row-size=18B cardinality=950
 |  |  |  in pipelines: 02(GETNEXT)
 |  |  |
 |  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -319,7 +319,7 @@ Per-Host Resources: mem-estimate=24.17MB mem-reservation=10.56MB thread-reservat
 |  |     parquet statistics predicates: ca_state = 'GA'
 |  |     parquet dictionary predicates: ca_state = 'GA'
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=18B cardinality=980
+|  |     tuple-ids=2 row-size=18B cardinality=950
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  00:SCAN HDFS [tpcds_parquet.catalog_sales cs1, RANDOM]
@@ -388,14 +388,14 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |  output: sum(cs_ext_ship_cost), sum(cs_net_profit)
 |  group by: cs_order_number
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=40B cardinality=30.02K
+|  tuple-ids=8 row-size=40B cardinality=29.10K
 |  in pipelines: 11(GETNEXT), 04(OPEN)
 |
 10:HASH JOIN [LEFT ANTI JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: cs1.cs_order_number = cr1.cr_order_number
 |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  in pipelines: 04(GETNEXT), 05(OPEN)
 |
 |--F08:PLAN FRAGMENT [HASH(cs1.cs_order_number)] hosts=3 instances=3
@@ -427,11 +427,11 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |  hash predicates: cs2.cs_order_number = cs1.cs_order_number
 |  other join predicates: cs1.cs_warehouse_sk != cs2.cs_warehouse_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 |--F09:PLAN FRAGMENT [HASH(cs1.cs_order_number)] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=5.28MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=5.25MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: cs1.cs_order_number
@@ -439,8 +439,8 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  |
 |  18:EXCHANGE [HASH(cs1.cs_order_number)]
-|  |  mem-estimate=1.41MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  |  mem-estimate=1.37MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -451,7 +451,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |  |  hash predicates: cs1.cs_call_center_sk = cc_call_center_sk
 |  |  fk/pk conjuncts: cs1.cs_call_center_sk = cc_call_center_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=30.02K
+|  |  tuple-ids=0,2,1,3 row-size=109B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -487,7 +487,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |  |  hash predicates: cs1.cs_ship_date_sk = d_date_sk
 |  |  fk/pk conjuncts: cs1.cs_ship_date_sk = d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,1 row-size=76B cardinality=30.02K
+|  |  tuple-ids=0,2,1 row-size=76B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |
 |  |--F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -523,7 +523,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |  |  hash predicates: cs1.cs_ship_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: cs1.cs_ship_addr_sk = ca_address_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2 row-size=50B cardinality=30.02K
+|  |  tuple-ids=0,2 row-size=50B cardinality=29.10K
 |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |
 |  |--F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -535,8 +535,8 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  14:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=38.28KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=18B cardinality=980
+|  |  |  mem-estimate=37.11KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=2 row-size=18B cardinality=950
 |  |  |  in pipelines: 02(GETNEXT)
 |  |  |
 |  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -551,7 +551,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |  |     parquet statistics predicates: ca_state = 'GA'
 |  |     parquet dictionary predicates: ca_state = 'GA'
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=2 row-size=18B cardinality=980
+|  |     tuple-ids=2 row-size=18B cardinality=950
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  00:SCAN HDFS [tpcds_parquet.catalog_sales cs1, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
index 29440e9..8039982 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
@@ -53,7 +53,7 @@ PLAN-ROOT SINK
 |  output: sum(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id, i_manufact_id, i_manufact
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=76B cardinality=1.73K
+|  tuple-ids=6 row-size=76B cardinality=1.72K
 |  in pipelines: 11(GETNEXT), 04(OPEN)
 |
 10:HASH JOIN [INNER JOIN]
@@ -62,7 +62,7 @@ PLAN-ROOT SINK
 |  other predicates: substr(ca_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT)) != substr(s_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT))
 |  runtime filters: RF000[bloom] <- s_store_sk, RF001[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,3,1,2,0,5 row-size=158B cardinality=1.73K
+|  tuple-ids=4,3,1,2,0,5 row-size=158B cardinality=1.72K
 |  in pipelines: 04(GETNEXT), 05(OPEN)
 |
 |--05:SCAN HDFS [tpcds_parquet.store]
@@ -80,7 +80,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ca_address_sk = c_current_addr_sk
 |  runtime filters: RF002[bloom] <- c_current_addr_sk, RF003[min_max] <- c_current_addr_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,3,1,2,0 row-size=137B cardinality=1.73K
+|  tuple-ids=4,3,1,2,0 row-size=137B cardinality=1.72K
 |  in pipelines: 04(GETNEXT), 03(OPEN)
 |
 |--08:HASH JOIN [INNER JOIN]
@@ -88,7 +88,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: c_customer_sk = ss_customer_sk
 |  |  runtime filters: RF004[bloom] <- ss_customer_sk, RF005[min_max] <- ss_customer_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=3,1,2,0 row-size=116B cardinality=1.73K
+|  |  tuple-ids=3,1,2,0 row-size=116B cardinality=1.72K
 |  |  in pipelines: 03(GETNEXT), 01(OPEN)
 |  |
 |  |--07:HASH JOIN [INNER JOIN]
@@ -96,7 +96,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  |  runtime filters: RF006[bloom] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=1,2,0 row-size=108B cardinality=1.73K
+|  |  |  tuple-ids=1,2,0 row-size=108B cardinality=1.72K
 |  |  |  in pipelines: 01(GETNEXT), 00(OPEN)
 |  |  |
 |  |  |--00:SCAN HDFS [tpcds_parquet.date_dim]
@@ -117,7 +117,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  |  runtime filters: RF008[bloom] <- i_item_sk, RF009[min_max] <- i_item_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=1,2 row-size=96B cardinality=29.12K
+|  |  |  tuple-ids=1,2 row-size=96B cardinality=28.96K
 |  |  |  in pipelines: 01(GETNEXT), 02(OPEN)
 |  |  |
 |  |  |--02:SCAN HDFS [tpcds_parquet.item]
@@ -130,7 +130,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: i_manager_id = CAST(7 AS INT)
 |  |  |     parquet dictionary predicates: i_manager_id = CAST(7 AS INT)
 |  |  |     mem-estimate=96.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=2 row-size=72B cardinality=182
+|  |  |     tuple-ids=2 row-size=72B cardinality=181
 |  |  |     in pipelines: 02(GETNEXT)
 |  |  |
 |  |  01:SCAN HDFS [tpcds_parquet.store_sales]
@@ -194,12 +194,12 @@ Per-Host Resources: mem-estimate=10.27MB mem-reservation=1.94MB thread-reservati
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id, i_manufact_id, i_manufact
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=76B cardinality=1.73K
+|  tuple-ids=6 row-size=76B cardinality=1.72K
 |  in pipelines: 21(GETNEXT), 01(OPEN)
 |
 20:EXCHANGE [HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]
-|  mem-estimate=281.04KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=76B cardinality=1.73K
+|  mem-estimate=280.82KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=76B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F06:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=3
@@ -208,7 +208,7 @@ Per-Host Resources: mem-estimate=18.30MB mem-reservation=8.81MB thread-reservati
 |  output: sum(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id, i_manufact_id, i_manufact
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=76B cardinality=1.73K
+|  tuple-ids=6 row-size=76B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 10:HASH JOIN [INNER JOIN, BROADCAST]
@@ -217,7 +217,7 @@ Per-Host Resources: mem-estimate=18.30MB mem-reservation=8.81MB thread-reservati
 |  other predicates: substr(ca_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT)) != substr(s_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT))
 |  runtime filters: RF000[bloom] <- s_store_sk, RF001[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0,3,4,5 row-size=158B cardinality=1.73K
+|  tuple-ids=1,2,0,3,4,5 row-size=158B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--19:EXCHANGE [BROADCAST]
@@ -242,7 +242,7 @@ Per-Host Resources: mem-estimate=18.30MB mem-reservation=8.81MB thread-reservati
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  runtime filters: RF002[bloom] <- ca_address_sk, RF003[min_max] <- ca_address_sk
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=1,2,0,3,4 row-size=137B cardinality=1.73K
+|  tuple-ids=1,2,0,3,4 row-size=137B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--18:EXCHANGE [HASH(ca_address_sk)]
@@ -263,8 +263,8 @@ Per-Host Resources: mem-estimate=18.30MB mem-reservation=8.81MB thread-reservati
 |     in pipelines: 04(GETNEXT)
 |
 17:EXCHANGE [HASH(c_current_addr_sk)]
-|  mem-estimate=459.50KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=1,2,0,3 row-size=116B cardinality=1.73K
+|  mem-estimate=459.16KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=1,2,0,3 row-size=116B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
@@ -274,7 +274,7 @@ Per-Host Resources: mem-estimate=5.06MB mem-reservation=3.88MB thread-reservatio
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=1,2,0,3 row-size=116B cardinality=1.73K
+|  tuple-ids=1,2,0,3 row-size=116B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--16:EXCHANGE [HASH(c_customer_sk)]
@@ -296,8 +296,8 @@ Per-Host Resources: mem-estimate=5.06MB mem-reservation=3.88MB thread-reservatio
 |     in pipelines: 03(GETNEXT)
 |
 15:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=419.01KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=1,2,0 row-size=108B cardinality=1.73K
+|  mem-estimate=418.69KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=1,2,0 row-size=108B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -307,7 +307,7 @@ Per-Host Resources: mem-estimate=71.92MB mem-reservation=9.88MB thread-reservati
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF006[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0 row-size=108B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=108B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--14:EXCHANGE [BROADCAST]
@@ -335,12 +335,12 @@ Per-Host Resources: mem-estimate=71.92MB mem-reservation=9.88MB thread-reservati
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF008[bloom] <- i_item_sk, RF009[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=96B cardinality=29.12K
+|  tuple-ids=1,2 row-size=96B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--13:EXCHANGE [BROADCAST]
-|  |  mem-estimate=26.14KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=72B cardinality=182
+|  |  mem-estimate=26.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=72B cardinality=181
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -355,7 +355,7 @@ Per-Host Resources: mem-estimate=71.92MB mem-reservation=9.88MB thread-reservati
 |     parquet statistics predicates: i_manager_id = CAST(7 AS INT)
 |     parquet dictionary predicates: i_manager_id = CAST(7 AS INT)
 |     mem-estimate=96.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=2 row-size=72B cardinality=182
+|     tuple-ids=2 row-size=72B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
@@ -397,12 +397,12 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id, i_manufact_id, i_manufact
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=76B cardinality=1.73K
+|  tuple-ids=6 row-size=76B cardinality=1.72K
 |  in pipelines: 21(GETNEXT), 01(OPEN)
 |
 20:EXCHANGE [HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]
-|  mem-estimate=519.66KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=76B cardinality=1.73K
+|  mem-estimate=519.44KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=76B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F06:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=6
@@ -411,7 +411,7 @@ Per-Instance Resources: mem-estimate=10.83MB mem-reservation=2.00MB thread-reser
 |  output: sum(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id, i_manufact_id, i_manufact
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=76B cardinality=1.73K
+|  tuple-ids=6 row-size=76B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 10:HASH JOIN [INNER JOIN, BROADCAST]
@@ -420,7 +420,7 @@ Per-Instance Resources: mem-estimate=10.83MB mem-reservation=2.00MB thread-reser
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  other predicates: substr(ca_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT)) != substr(s_zip, CAST(1 AS BIGINT), CAST(5 AS BIGINT))
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0,3,4,5 row-size=158B cardinality=1.73K
+|  tuple-ids=1,2,0,3,4,5 row-size=158B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--F10:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=3
@@ -453,7 +453,7 @@ Per-Instance Resources: mem-estimate=10.83MB mem-reservation=2.00MB thread-reser
 |  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=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0,3,4 row-size=137B cardinality=1.73K
+|  tuple-ids=1,2,0,3,4 row-size=137B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--F11:PLAN FRAGMENT [HASH(c_current_addr_sk)] hosts=3 instances=6
@@ -482,18 +482,18 @@ Per-Instance Resources: mem-estimate=10.83MB mem-reservation=2.00MB thread-reser
 |     in pipelines: 04(GETNEXT)
 |
 17:EXCHANGE [HASH(c_current_addr_sk)]
-|  mem-estimate=854.12KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=1,2,0,3 row-size=116B cardinality=1.73K
+|  mem-estimate=853.78KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=1,2,0,3 row-size=116B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=777.63KB mem-reservation=0B thread-reservation=1
+Per-Instance Resources: mem-estimate=777.31KB mem-reservation=0B thread-reservation=1
 08:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=02
 |  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=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0,3 row-size=116B cardinality=1.73K
+|  tuple-ids=1,2,0,3 row-size=116B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--F12:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
@@ -524,8 +524,8 @@ Per-Instance Resources: mem-estimate=777.63KB mem-reservation=0B thread-reservat
 |     in pipelines: 03(GETNEXT)
 |
 15:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=777.63KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=1,2,0 row-size=108B cardinality=1.73K
+|  mem-estimate=777.31KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=1,2,0 row-size=108B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -536,7 +536,7 @@ Per-Instance Resources: mem-estimate=16.00MB 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=1,2,0 row-size=108B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=108B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--F13:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -572,7 +572,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=96B cardinality=29.12K
+|  tuple-ids=1,2 row-size=96B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F14:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -584,8 +584,8 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  13:EXCHANGE [BROADCAST]
-|  |  mem-estimate=26.14KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=72B cardinality=182
+|  |  mem-estimate=26.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=72B cardinality=181
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -600,7 +600,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |     parquet statistics predicates: i_manager_id = CAST(7 AS INT)
 |     parquet dictionary predicates: i_manager_id = CAST(7 AS INT)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|     tuple-ids=2 row-size=72B cardinality=182
+|     tuple-ids=2 row-size=72B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
index fe7802c..ccdd930 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
@@ -63,13 +63,13 @@ PLAN-ROOT SINK
 27:SORT
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT), 12(OPEN)
 |
 26:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: sum(netpaid) > CAST(0.05 AS DECIMAL(2,2)) * avg(netpaid)
 |  mem-estimate=16B mem-reservation=0B thread-reservation=0
-|  tuple-ids=8,19 row-size=84B cardinality=7.69K
+|  tuple-ids=8,19 row-size=84B cardinality=7.65K
 |  in pipelines: 12(GETNEXT), 25(OPEN)
 |
 |--25:AGGREGATE [FINALIZE]
@@ -198,14 +198,14 @@ PLAN-ROOT SINK
 |  output: sum(sum(ss_net_paid))
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 12(GETNEXT), 11(OPEN)
 |
 11:AGGREGATE [FINALIZE]
 |  output: sum(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 11(GETNEXT), 05(OPEN)
 |
 10:HASH JOIN [INNER JOIN]
@@ -214,7 +214,7 @@ PLAN-ROOT SINK
 |  other predicates: c_birth_country != upper(ca_country)
 |  runtime filters: RF000[bloom] <- c_current_addr_sk, RF001[bloom] <- s_zip, RF002[min_max] <- c_current_addr_sk, RF003[min_max] <- s_zip
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=5,4,1,0,3,2 row-size=290B cardinality=7.69K
+|  tuple-ids=5,4,1,0,3,2 row-size=290B cardinality=7.65K
 |  in pipelines: 05(GETNEXT), 04(OPEN)
 |
 |--09:HASH JOIN [INNER JOIN]
@@ -222,7 +222,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: c_customer_sk = ss_customer_sk
 |  |  runtime filters: RF004[bloom] <- ss_customer_sk, RF005[min_max] <- ss_customer_sk
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=4,1,0,3,2 row-size=230B cardinality=7.69K
+|  |  tuple-ids=4,1,0,3,2 row-size=230B cardinality=7.65K
 |  |  in pipelines: 04(GETNEXT), 01(OPEN)
 |  |
 |  |--08:HASH JOIN [INNER JOIN]
@@ -230,7 +230,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: sr_item_sk = ss_item_sk, sr_ticket_number = ss_ticket_number
 |  |  |  runtime filters: RF006[bloom] <- ss_item_sk, RF007[bloom] <- ss_ticket_number, RF008[min_max] <- ss_item_sk, RF009[min_max] <- ss_ticket_number
 |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  |  tuple-ids=1,0,3,2 row-size=165B cardinality=7.69K
+|  |  |  tuple-ids=1,0,3,2 row-size=165B cardinality=7.65K
 |  |  |  in pipelines: 01(GETNEXT), 00(OPEN)
 |  |  |
 |  |  |--07:HASH JOIN [INNER JOIN]
@@ -238,7 +238,7 @@ PLAN-ROOT SINK
 |  |  |  |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  |  |  |  runtime filters: RF010[bloom] <- s_store_sk, RF011[min_max] <- s_store_sk
 |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  |  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  |  |  |  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  |  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |  |  |
 |  |  |  |--02:SCAN HDFS [tpcds_parquet.store]
@@ -259,7 +259,7 @@ PLAN-ROOT SINK
 |  |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  |  |  runtime filters: RF012[bloom] <- i_item_sk, RF013[min_max] <- i_item_sk
 |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  |  tuple-ids=0,3 row-size=94B cardinality=31.36K
+|  |  |  |  tuple-ids=0,3 row-size=94B cardinality=31.20K
 |  |  |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |  |  |
 |  |  |  |--03:SCAN HDFS [tpcds_parquet.item]
@@ -272,7 +272,7 @@ PLAN-ROOT SINK
 |  |  |  |     parquet statistics predicates: tpcds_parquet.item.i_color = 'peach'
 |  |  |  |     parquet dictionary predicates: tpcds_parquet.item.i_color = 'peach'
 |  |  |  |     mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |  |     tuple-ids=3 row-size=66B cardinality=196
+|  |  |  |     tuple-ids=3 row-size=66B cardinality=195
 |  |  |  |     in pipelines: 03(GETNEXT)
 |  |  |  |
 |  |  |  00:SCAN HDFS [tpcds_parquet.store_sales]
@@ -330,8 +330,8 @@ PLAN-ROOT SINK
 |
 50:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
-|  mem-estimate=387.40KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  mem-estimate=386.54KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT)
 |
 F10:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name)] hosts=3 instances=3
@@ -339,13 +339,13 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat
 27:SORT
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT), 39(OPEN)
 |
 26:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  predicates: sum(netpaid) > CAST(0.05 AS DECIMAL(2,2)) * avg(netpaid)
 |  mem-estimate=16B mem-reservation=0B thread-reservation=0
-|  tuple-ids=8,19 row-size=84B cardinality=7.69K
+|  tuple-ids=8,19 row-size=84B cardinality=7.65K
 |  in pipelines: 39(GETNEXT), 48(OPEN)
 |
 |--49:EXCHANGE [BROADCAST]
@@ -543,12 +543,12 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat
 |  output: sum:merge(netpaid)
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 39(GETNEXT), 37(OPEN)
 |
 38:EXCHANGE [HASH(c_last_name,c_first_name,s_store_name)]
-|  mem-estimate=387.40KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  mem-estimate=386.54KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 37(GETNEXT)
 |
 F09:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)] hosts=3 instances=3
@@ -557,19 +557,19 @@ Per-Host Resources: mem-estimate=20.84MB mem-reservation=3.94MB thread-reservati
 |  output: sum(sum(ss_net_paid))
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 37(GETNEXT)
 |
 37:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 37(GETNEXT), 00(OPEN)
 |
 36:EXCHANGE [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)]
-|  mem-estimate=860.24KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  mem-estimate=858.29KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F08:PLAN FRAGMENT [HASH(c_current_addr_sk,s_zip)] hosts=3 instances=3
@@ -578,7 +578,7 @@ Per-Host Resources: mem-estimate=19.09MB mem-reservation=6.88MB thread-reservati
 |  output: sum(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 10:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -587,7 +587,7 @@ Per-Host Resources: mem-estimate=19.09MB mem-reservation=6.88MB thread-reservati
 |  other predicates: c_birth_country != upper(ca_country)
 |  runtime filters: RF000[bloom] <- ca_address_sk, RF001[bloom] <- ca_zip, RF002[min_max] <- ca_address_sk, RF003[min_max] <- ca_zip
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1,4,5 row-size=290B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4,5 row-size=290B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--35:EXCHANGE [HASH(ca_address_sk,ca_zip)]
@@ -609,7 +609,7 @@ Per-Host Resources: mem-estimate=19.09MB mem-reservation=6.88MB thread-reservati
 |
 34:EXCHANGE [HASH(c_current_addr_sk,s_zip)]
 |  mem-estimate=1.29MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F06:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
@@ -619,7 +619,7 @@ Per-Host Resources: mem-estimate=12.92MB mem-reservation=5.75MB thread-reservati
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
 |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--33:EXCHANGE [HASH(c_customer_sk)]
@@ -641,8 +641,8 @@ Per-Host Resources: mem-estimate=12.92MB mem-reservation=5.75MB thread-reservati
 |     in pipelines: 04(GETNEXT)
 |
 32:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=956.98KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.69K
+|  mem-estimate=954.88KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=3
@@ -652,7 +652,7 @@ Per-Host Resources: mem-estimate=15.87MB mem-reservation=10.50MB thread-reservat
 |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  runtime filters: RF006[bloom] <- sr_item_sk, RF007[bloom] <- sr_ticket_number, RF008[min_max] <- sr_item_sk, RF009[min_max] <- sr_ticket_number
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.69K
+|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--31:EXCHANGE [HASH(sr_item_sk,sr_ticket_number)]
@@ -673,8 +673,8 @@ Per-Host Resources: mem-estimate=15.87MB mem-reservation=10.50MB thread-reservat
 |     in pipelines: 01(GETNEXT)
 |
 30:EXCHANGE [HASH(ss_item_sk,ss_ticket_number)]
-|  mem-estimate=991.19KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  mem-estimate=988.61KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -684,7 +684,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF010[bloom] <- s_store_sk, RF011[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--29:EXCHANGE [BROADCAST]
@@ -713,12 +713,12 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF012[bloom] <- i_item_sk, RF013[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=94B cardinality=31.36K
+|  tuple-ids=0,3 row-size=94B cardinality=31.20K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--28:EXCHANGE [BROADCAST]
-|  |  mem-estimate=26.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=66B cardinality=196
+|  |  mem-estimate=25.87KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=66B cardinality=195
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -734,7 +734,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat
 |     parquet statistics predicates: tpcds_parquet.item.i_color = 'peach'
 |     parquet dictionary predicates: tpcds_parquet.item.i_color = 'peach'
 |     mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=3 row-size=66B cardinality=196
+|     tuple-ids=3 row-size=66B cardinality=195
 |     in pipelines: 03(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
@@ -759,8 +759,8 @@ PLAN-ROOT SINK
 |
 50:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
-|  mem-estimate=604.05KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  mem-estimate=603.18KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT)
 |
 F10:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name)] hosts=3 instances=6
@@ -768,14 +768,14 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese
 27:SORT
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT), 39(OPEN)
 |
 26:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  join table id: 00
 |  predicates: sum(netpaid) > CAST(0.05 AS DECIMAL(2,2)) * avg(netpaid)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=8,19 row-size=84B cardinality=7.69K
+|  tuple-ids=8,19 row-size=84B cardinality=7.65K
 |  in pipelines: 39(GETNEXT), 48(OPEN)
 |
 |--F20:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name)] hosts=3 instances=3
@@ -1020,12 +1020,12 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese
 |  output: sum:merge(netpaid)
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 39(GETNEXT), 37(OPEN)
 |
 38:EXCHANGE [HASH(c_last_name,c_first_name,s_store_name)]
-|  mem-estimate=604.05KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  mem-estimate=603.18KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 37(GETNEXT)
 |
 F09:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)] hosts=3 instances=6
@@ -1034,28 +1034,28 @@ Per-Instance Resources: mem-estimate=21.30MB mem-reservation=3.94MB thread-reser
 |  output: sum(sum(ss_net_paid))
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 37(GETNEXT)
 |
 37:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 37(GETNEXT), 00(OPEN)
 |
 36:EXCHANGE [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)]
 |  mem-estimate=1.30MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F08:PLAN FRAGMENT [HASH(c_current_addr_sk,s_zip)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=12.03MB mem-reservation=2.00MB thread-reservation=1
+Per-Instance Resources: mem-estimate=12.02MB mem-reservation=2.00MB thread-reservation=1
 11:AGGREGATE [STREAMING]
 |  output: sum(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 10:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1064,7 +1064,7 @@ Per-Instance Resources: mem-estimate=12.03MB mem-reservation=2.00MB thread-reser
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  other predicates: c_birth_country != upper(ca_country)
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1,4,5 row-size=290B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4,5 row-size=290B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--F26:PLAN FRAGMENT [HASH(c_current_addr_sk,s_zip)] hosts=3 instances=6
@@ -1093,18 +1093,18 @@ Per-Instance Resources: mem-estimate=12.03MB mem-reservation=2.00MB thread-reser
 |     in pipelines: 05(GETNEXT)
 |
 34:EXCHANGE [HASH(c_current_addr_sk,s_zip)]
-|  mem-estimate=2.03MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.69K
+|  mem-estimate=2.02MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F06:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=1.47MB mem-reservation=0B thread-reservation=1
+Per-Instance Resources: mem-estimate=1.46MB mem-reservation=0B thread-reservation=1
 09: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=0,3,2,1,4 row-size=230B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--F27:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
@@ -1135,8 +1135,8 @@ Per-Instance Resources: mem-estimate=1.47MB mem-reservation=0B thread-reservatio
 |     in pipelines: 04(GETNEXT)
 |
 32:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=1.47MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.69K
+|  mem-estimate=1.46MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=6
@@ -1146,7 +1146,7 @@ Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservatio
 |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.69K
+|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--F28:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=6
@@ -1176,7 +1176,7 @@ Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservatio
 |
 30:EXCHANGE [HASH(ss_item_sk,ss_ticket_number)]
 |  mem-estimate=1.44MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1187,7 +1187,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: ss_store_sk = s_store_sk
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1225,7 +1225,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=94B cardinality=31.36K
+|  tuple-ids=0,3 row-size=94B cardinality=31.20K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1237,8 +1237,8 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  28:EXCHANGE [BROADCAST]
-|  |  mem-estimate=26.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=66B cardinality=196
+|  |  mem-estimate=25.87KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=66B cardinality=195
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1255,7 +1255,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |     parquet statistics predicates: tpcds_parquet.item.i_color = 'peach'
 |     parquet dictionary predicates: tpcds_parquet.item.i_color = 'peach'
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=3 row-size=66B cardinality=196
+|     tuple-ids=3 row-size=66B cardinality=195
 |     in pipelines: 03(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
index f3016e4..59ff1c3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
@@ -63,13 +63,13 @@ PLAN-ROOT SINK
 27:SORT
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT), 12(OPEN)
 |
 26:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: sum(netpaid) > CAST(0.05 AS DECIMAL(2,2)) * avg(netpaid)
 |  mem-estimate=16B mem-reservation=0B thread-reservation=0
-|  tuple-ids=8,19 row-size=84B cardinality=7.69K
+|  tuple-ids=8,19 row-size=84B cardinality=7.65K
 |  in pipelines: 12(GETNEXT), 25(OPEN)
 |
 |--25:AGGREGATE [FINALIZE]
@@ -198,14 +198,14 @@ PLAN-ROOT SINK
 |  output: sum(sum(ss_net_paid))
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 12(GETNEXT), 11(OPEN)
 |
 11:AGGREGATE [FINALIZE]
 |  output: sum(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 11(GETNEXT), 05(OPEN)
 |
 10:HASH JOIN [INNER JOIN]
@@ -214,7 +214,7 @@ PLAN-ROOT SINK
 |  other predicates: c_birth_country != upper(ca_country)
 |  runtime filters: RF000[bloom] <- c_current_addr_sk, RF001[bloom] <- s_zip, RF002[min_max] <- c_current_addr_sk, RF003[min_max] <- s_zip
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=5,4,1,0,3,2 row-size=290B cardinality=7.69K
+|  tuple-ids=5,4,1,0,3,2 row-size=290B cardinality=7.65K
 |  in pipelines: 05(GETNEXT), 04(OPEN)
 |
 |--09:HASH JOIN [INNER JOIN]
@@ -222,7 +222,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: c_customer_sk = ss_customer_sk
 |  |  runtime filters: RF004[bloom] <- ss_customer_sk, RF005[min_max] <- ss_customer_sk
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=4,1,0,3,2 row-size=230B cardinality=7.69K
+|  |  tuple-ids=4,1,0,3,2 row-size=230B cardinality=7.65K
 |  |  in pipelines: 04(GETNEXT), 01(OPEN)
 |  |
 |  |--08:HASH JOIN [INNER JOIN]
@@ -230,7 +230,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: sr_item_sk = ss_item_sk, sr_ticket_number = ss_ticket_number
 |  |  |  runtime filters: RF006[bloom] <- ss_item_sk, RF007[bloom] <- ss_ticket_number, RF008[min_max] <- ss_item_sk, RF009[min_max] <- ss_ticket_number
 |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  |  tuple-ids=1,0,3,2 row-size=165B cardinality=7.69K
+|  |  |  tuple-ids=1,0,3,2 row-size=165B cardinality=7.65K
 |  |  |  in pipelines: 01(GETNEXT), 00(OPEN)
 |  |  |
 |  |  |--07:HASH JOIN [INNER JOIN]
@@ -238,7 +238,7 @@ PLAN-ROOT SINK
 |  |  |  |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  |  |  |  runtime filters: RF010[bloom] <- s_store_sk, RF011[min_max] <- s_store_sk
 |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  |  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  |  |  |  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  |  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |  |  |
 |  |  |  |--02:SCAN HDFS [tpcds_parquet.store]
@@ -259,7 +259,7 @@ PLAN-ROOT SINK
 |  |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  |  |  runtime filters: RF012[bloom] <- i_item_sk, RF013[min_max] <- i_item_sk
 |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  |  tuple-ids=0,3 row-size=94B cardinality=31.36K
+|  |  |  |  tuple-ids=0,3 row-size=94B cardinality=31.20K
 |  |  |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |  |  |
 |  |  |  |--03:SCAN HDFS [tpcds_parquet.item]
@@ -272,7 +272,7 @@ PLAN-ROOT SINK
 |  |  |  |     parquet statistics predicates: tpcds_parquet.item.i_color = 'saddle'
 |  |  |  |     parquet dictionary predicates: tpcds_parquet.item.i_color = 'saddle'
 |  |  |  |     mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |  |     tuple-ids=3 row-size=66B cardinality=196
+|  |  |  |     tuple-ids=3 row-size=66B cardinality=195
 |  |  |  |     in pipelines: 03(GETNEXT)
 |  |  |  |
 |  |  |  00:SCAN HDFS [tpcds_parquet.store_sales]
@@ -330,8 +330,8 @@ PLAN-ROOT SINK
 |
 50:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
-|  mem-estimate=387.40KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  mem-estimate=386.54KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT)
 |
 F10:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name)] hosts=3 instances=3
@@ -339,13 +339,13 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat
 27:SORT
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT), 39(OPEN)
 |
 26:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  predicates: sum(netpaid) > CAST(0.05 AS DECIMAL(2,2)) * avg(netpaid)
 |  mem-estimate=16B mem-reservation=0B thread-reservation=0
-|  tuple-ids=8,19 row-size=84B cardinality=7.69K
+|  tuple-ids=8,19 row-size=84B cardinality=7.65K
 |  in pipelines: 39(GETNEXT), 48(OPEN)
 |
 |--49:EXCHANGE [BROADCAST]
@@ -543,12 +543,12 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat
 |  output: sum:merge(netpaid)
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 39(GETNEXT), 37(OPEN)
 |
 38:EXCHANGE [HASH(c_last_name,c_first_name,s_store_name)]
-|  mem-estimate=387.40KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  mem-estimate=386.54KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 37(GETNEXT)
 |
 F09:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)] hosts=3 instances=3
@@ -557,19 +557,19 @@ Per-Host Resources: mem-estimate=20.84MB mem-reservation=3.94MB thread-reservati
 |  output: sum(sum(ss_net_paid))
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 37(GETNEXT)
 |
 37:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 37(GETNEXT), 00(OPEN)
 |
 36:EXCHANGE [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)]
-|  mem-estimate=860.24KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  mem-estimate=858.29KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F08:PLAN FRAGMENT [HASH(c_current_addr_sk,s_zip)] hosts=3 instances=3
@@ -578,7 +578,7 @@ Per-Host Resources: mem-estimate=19.09MB mem-reservation=6.88MB thread-reservati
 |  output: sum(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 10:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -587,7 +587,7 @@ Per-Host Resources: mem-estimate=19.09MB mem-reservation=6.88MB thread-reservati
 |  other predicates: c_birth_country != upper(ca_country)
 |  runtime filters: RF000[bloom] <- ca_address_sk, RF001[bloom] <- ca_zip, RF002[min_max] <- ca_address_sk, RF003[min_max] <- ca_zip
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1,4,5 row-size=290B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4,5 row-size=290B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--35:EXCHANGE [HASH(ca_address_sk,ca_zip)]
@@ -609,7 +609,7 @@ Per-Host Resources: mem-estimate=19.09MB mem-reservation=6.88MB thread-reservati
 |
 34:EXCHANGE [HASH(c_current_addr_sk,s_zip)]
 |  mem-estimate=1.29MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F06:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
@@ -619,7 +619,7 @@ Per-Host Resources: mem-estimate=12.92MB mem-reservation=5.75MB thread-reservati
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
 |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--33:EXCHANGE [HASH(c_customer_sk)]
@@ -641,8 +641,8 @@ Per-Host Resources: mem-estimate=12.92MB mem-reservation=5.75MB thread-reservati
 |     in pipelines: 04(GETNEXT)
 |
 32:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=956.98KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.69K
+|  mem-estimate=954.88KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=3
@@ -652,7 +652,7 @@ Per-Host Resources: mem-estimate=15.87MB mem-reservation=10.50MB thread-reservat
 |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  runtime filters: RF006[bloom] <- sr_item_sk, RF007[bloom] <- sr_ticket_number, RF008[min_max] <- sr_item_sk, RF009[min_max] <- sr_ticket_number
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.69K
+|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--31:EXCHANGE [HASH(sr_item_sk,sr_ticket_number)]
@@ -673,8 +673,8 @@ Per-Host Resources: mem-estimate=15.87MB mem-reservation=10.50MB thread-reservat
 |     in pipelines: 01(GETNEXT)
 |
 30:EXCHANGE [HASH(ss_item_sk,ss_ticket_number)]
-|  mem-estimate=991.19KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  mem-estimate=988.61KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -684,7 +684,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF010[bloom] <- s_store_sk, RF011[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--29:EXCHANGE [BROADCAST]
@@ -713,12 +713,12 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF012[bloom] <- i_item_sk, RF013[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=94B cardinality=31.36K
+|  tuple-ids=0,3 row-size=94B cardinality=31.20K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--28:EXCHANGE [BROADCAST]
-|  |  mem-estimate=26.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=66B cardinality=196
+|  |  mem-estimate=25.87KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=66B cardinality=195
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -734,7 +734,7 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat
 |     parquet statistics predicates: tpcds_parquet.item.i_color = 'saddle'
 |     parquet dictionary predicates: tpcds_parquet.item.i_color = 'saddle'
 |     mem-estimate=96.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=3 row-size=66B cardinality=196
+|     tuple-ids=3 row-size=66B cardinality=195
 |     in pipelines: 03(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
@@ -759,8 +759,8 @@ PLAN-ROOT SINK
 |
 50:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
-|  mem-estimate=604.05KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  mem-estimate=603.18KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT)
 |
 F10:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name)] hosts=3 instances=6
@@ -768,14 +768,14 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese
 27:SORT
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=21 row-size=68B cardinality=7.69K
+|  tuple-ids=21 row-size=68B cardinality=7.65K
 |  in pipelines: 27(GETNEXT), 39(OPEN)
 |
 26:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  join table id: 00
 |  predicates: sum(netpaid) > CAST(0.05 AS DECIMAL(2,2)) * avg(netpaid)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=8,19 row-size=84B cardinality=7.69K
+|  tuple-ids=8,19 row-size=84B cardinality=7.65K
 |  in pipelines: 39(GETNEXT), 48(OPEN)
 |
 |--F20:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name)] hosts=3 instances=3
@@ -1020,12 +1020,12 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese
 |  output: sum:merge(netpaid)
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 39(GETNEXT), 37(OPEN)
 |
 38:EXCHANGE [HASH(c_last_name,c_first_name,s_store_name)]
-|  mem-estimate=604.05KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  mem-estimate=603.18KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 37(GETNEXT)
 |
 F09:PLAN FRAGMENT [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)] hosts=3 instances=6
@@ -1034,28 +1034,28 @@ Per-Instance Resources: mem-estimate=21.30MB mem-reservation=3.94MB thread-reser
 |  output: sum(sum(ss_net_paid))
 |  group by: c_last_name, c_first_name, s_store_name
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=68B cardinality=7.69K
+|  tuple-ids=8 row-size=68B cardinality=7.65K
 |  in pipelines: 37(GETNEXT)
 |
 37:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 37(GETNEXT), 00(OPEN)
 |
 36:EXCHANGE [HASH(c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size)]
 |  mem-estimate=1.30MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F08:PLAN FRAGMENT [HASH(c_current_addr_sk,s_zip)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=12.03MB mem-reservation=2.00MB thread-reservation=1
+Per-Instance Resources: mem-estimate=12.02MB mem-reservation=2.00MB thread-reservation=1
 11:AGGREGATE [STREAMING]
 |  output: sum(ss_net_paid)
 |  group by: c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, i_current_price, i_manager_id, i_units, i_size
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=154B cardinality=7.69K
+|  tuple-ids=6 row-size=154B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 10:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1064,7 +1064,7 @@ Per-Instance Resources: mem-estimate=12.03MB mem-reservation=2.00MB thread-reser
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  other predicates: c_birth_country != upper(ca_country)
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1,4,5 row-size=290B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4,5 row-size=290B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--F26:PLAN FRAGMENT [HASH(c_current_addr_sk,s_zip)] hosts=3 instances=6
@@ -1093,18 +1093,18 @@ Per-Instance Resources: mem-estimate=12.03MB mem-reservation=2.00MB thread-reser
 |     in pipelines: 05(GETNEXT)
 |
 34:EXCHANGE [HASH(c_current_addr_sk,s_zip)]
-|  mem-estimate=2.03MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.69K
+|  mem-estimate=2.02MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F06:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=1.47MB mem-reservation=0B thread-reservation=1
+Per-Instance Resources: mem-estimate=1.46MB mem-reservation=0B thread-reservation=1
 09: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=0,3,2,1,4 row-size=230B cardinality=7.69K
+|  tuple-ids=0,3,2,1,4 row-size=230B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--F27:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
@@ -1135,8 +1135,8 @@ Per-Instance Resources: mem-estimate=1.47MB mem-reservation=0B thread-reservatio
 |     in pipelines: 04(GETNEXT)
 |
 32:EXCHANGE [HASH(ss_customer_sk)]
-|  mem-estimate=1.47MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.69K
+|  mem-estimate=1.46MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.65K
 |  in pipelines: 00(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=6
@@ -1146,7 +1146,7 @@ Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservatio
 |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.69K
+|  tuple-ids=0,3,2,1 row-size=165B cardinality=7.65K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--F28:PLAN FRAGMENT [HASH(ss_item_sk,ss_ticket_number)] hosts=3 instances=6
@@ -1176,7 +1176,7 @@ Per-Instance Resources: mem-estimate=1.44MB mem-reservation=0B thread-reservatio
 |
 30:EXCHANGE [HASH(ss_item_sk,ss_ticket_number)]
 |  mem-estimate=1.44MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1187,7 +1187,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: ss_store_sk = s_store_sk
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,2 row-size=149B cardinality=10.46K
+|  tuple-ids=0,3,2 row-size=149B cardinality=10.40K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1225,7 +1225,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=94B cardinality=31.36K
+|  tuple-ids=0,3 row-size=94B cardinality=31.20K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1237,8 +1237,8 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  28:EXCHANGE [BROADCAST]
-|  |  mem-estimate=26.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=66B cardinality=196
+|  |  mem-estimate=25.87KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=66B cardinality=195
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1255,7 +1255,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=4.00MB thread-reser
 |     parquet statistics predicates: tpcds_parquet.item.i_color = 'saddle'
 |     parquet dictionary predicates: tpcds_parquet.item.i_color = 'saddle'
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=3 row-size=66B cardinality=196
+|     tuple-ids=3 row-size=66B cardinality=195
 |     in pipelines: 03(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
index 8f3f595..4f9731e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
@@ -68,7 +68,7 @@ PLAN-ROOT SINK
 |  other join predicates: sum(wr_return_amt) > avg(ctr_total_return) * CAST(1.2 AS DECIMAL(2,1))
 |  runtime filters: RF000[bloom] <- ctr2.ctr_state, RF001[min_max] <- ctr2.ctr_state
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6,5,3 row-size=255B cardinality=3.26K
+|  tuple-ids=6,5,3 row-size=255B cardinality=3.16K
 |  in pipelines: 07(GETNEXT), 14(OPEN)
 |
 |--14:AGGREGATE [FINALIZE]
@@ -141,7 +141,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: none
 |  runtime filters: RF002[bloom] <- wr_returning_customer_sk, RF003[min_max] <- wr_returning_customer_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6,5,3 row-size=255B cardinality=3.26K
+|  tuple-ids=6,5,3 row-size=255B cardinality=3.16K
 |  in pipelines: 07(GETNEXT), 05(OPEN)
 |
 |--05:AGGREGATE [FINALIZE]
@@ -207,7 +207,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  runtime filters: RF004[bloom] <- ca_address_sk, RF005[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6,5 row-size=221B cardinality=2.27K
+|  tuple-ids=6,5 row-size=221B cardinality=2.21K
 |  in pipelines: 07(GETNEXT), 06(OPEN)
 |
 |--06:SCAN HDFS [tpcds_parquet.customer_address]
@@ -220,7 +220,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ca_state = 'GA'
 |     parquet dictionary predicates: ca_state = 'GA'
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=5 row-size=18B cardinality=980
+|     tuple-ids=5 row-size=18B cardinality=950
 |     in pipelines: 06(GETNEXT)
 |
 07:SCAN HDFS [tpcds_parquet.customer]
@@ -262,7 +262,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
 |  other join predicates: sum(wr_return_amt) > avg(ctr_total_return) * CAST(1.2 AS DECIMAL(2,1))
 |  runtime filters: RF000[bloom] <- ctr2.ctr_state, RF001[min_max] <- ctr2.ctr_state
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6,5,3 row-size=255B cardinality=3.26K
+|  tuple-ids=6,5,3 row-size=255B cardinality=3.16K
 |  in pipelines: 07(GETNEXT), 30(OPEN)
 |
 |--31:EXCHANGE [BROADCAST]
@@ -384,7 +384,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
 |  fk/pk conjuncts: none
 |  runtime filters: RF002[bloom] <- wr_returning_customer_sk, RF003[min_max] <- wr_returning_customer_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6,5,3 row-size=255B cardinality=3.26K
+|  tuple-ids=6,5,3 row-size=255B cardinality=3.16K
 |  in pipelines: 07(GETNEXT), 23(OPEN)
 |
 |--24:EXCHANGE [BROADCAST]
@@ -485,12 +485,12 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  runtime filters: RF004[bloom] <- ca_address_sk, RF005[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6,5 row-size=221B cardinality=2.27K
+|  tuple-ids=6,5 row-size=221B cardinality=2.21K
 |  in pipelines: 07(GETNEXT), 06(OPEN)
 |
 |--19:EXCHANGE [BROADCAST]
-|  |  mem-estimate=38.28KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=18B cardinality=980
+|  |  mem-estimate=37.11KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5 row-size=18B cardinality=950
 |  |  in pipelines: 06(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -505,7 +505,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
 |     parquet statistics predicates: ca_state = 'GA'
 |     parquet dictionary predicates: ca_state = 'GA'
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=5 row-size=18B cardinality=980
+|     tuple-ids=5 row-size=18B cardinality=950
 |     in pipelines: 06(GETNEXT)
 |
 07:SCAN HDFS [tpcds_parquet.customer, RANDOM]
@@ -548,7 +548,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |  hash predicates: ca_state = ctr2.ctr_state
 |  other join predicates: sum(wr_return_amt) > avg(ctr_total_return) * CAST(1.2 AS DECIMAL(2,1))
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6,5,3 row-size=255B cardinality=3.26K
+|  tuple-ids=6,5,3 row-size=255B cardinality=3.16K
 |  in pipelines: 07(GETNEXT), 30(OPEN)
 |
 |--F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -696,7 +696,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |  hash predicates: c_customer_sk = wr_returning_customer_sk
 |  fk/pk conjuncts: none
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6,5,3 row-size=255B cardinality=3.26K
+|  tuple-ids=6,5,3 row-size=255B cardinality=3.16K
 |  in pipelines: 07(GETNEXT), 23(OPEN)
 |
 |--F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -823,7 +823,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |  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=64.00KB thread-reservation=0
-|  tuple-ids=6,5 row-size=221B cardinality=2.27K
+|  tuple-ids=6,5 row-size=221B cardinality=2.21K
 |  in pipelines: 07(GETNEXT), 06(OPEN)
 |
 |--F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -835,8 +835,8 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  19:EXCHANGE [BROADCAST]
-|  |  mem-estimate=38.28KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=18B cardinality=980
+|  |  mem-estimate=37.11KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5 row-size=18B cardinality=950
 |  |  in pipelines: 06(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -851,7 +851,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |     parquet statistics predicates: ca_state = 'GA'
 |     parquet dictionary predicates: ca_state = 'GA'
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=5 row-size=18B cardinality=980
+|     tuple-ids=5 row-size=18B cardinality=950
 |     in pipelines: 06(GETNEXT)
 |
 07:SCAN HDFS [tpcds_parquet.customer, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
index 05f159f..c7ec587 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
@@ -109,7 +109,7 @@ PLAN-ROOT SINK
 |  |  hash predicates: i_manufact_id = i_manufact_id
 |  |  runtime filters: RF017[min_max] <- i_manufact_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 25(OPEN)
 |  |
 |  |--25:SCAN HDFS [tpcds_parquet.item]
@@ -130,7 +130,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  runtime filters: RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 24(OPEN)
 |  |
 |  |--24:SCAN HDFS [tpcds_parquet.item]
@@ -149,7 +149,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  runtime filters: RF020[bloom] <- ca_address_sk, RF021[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |
 |  |--23:SCAN HDFS [tpcds_parquet.customer_address]
@@ -162,7 +162,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |     in pipelines: 23(GETNEXT)
 |  |
 |  26:HASH JOIN [INNER JOIN]
@@ -208,7 +208,7 @@ PLAN-ROOT SINK
 |  |  hash predicates: i_manufact_id = i_manufact_id
 |  |  runtime filters: RF008[bloom] <- i_manufact_id, RF009[min_max] <- i_manufact_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 15(OPEN)
 |  |
 |  |--15:SCAN HDFS [tpcds_parquet.item]
@@ -229,7 +229,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  runtime filters: RF010[bloom] <- i_item_sk, RF011[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 14(OPEN)
 |  |
 |  |--14:SCAN HDFS [tpcds_parquet.item]
@@ -248,7 +248,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |
 |  |--13:SCAN HDFS [tpcds_parquet.customer_address]
@@ -261,7 +261,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |     in pipelines: 13(GETNEXT)
 |  |
 |  16:HASH JOIN [INNER JOIN]
@@ -307,7 +307,7 @@ PLAN-ROOT SINK
 |  hash predicates: i_manufact_id = i_manufact_id
 |  runtime filters: RF000[bloom] <- i_manufact_id, RF001[min_max] <- i_manufact_id
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=52B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=52B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--05:SCAN HDFS [tpcds_parquet.item]
@@ -328,7 +328,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  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=0,1,2,3 row-size=52B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=52B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--04:SCAN HDFS [tpcds_parquet.item]
@@ -347,7 +347,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  runtime filters: RF004[bloom] <- ca_address_sk, RF005[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--03:SCAN HDFS [tpcds_parquet.customer_address]
@@ -360,7 +360,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN]
@@ -458,7 +458,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F17:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=14.45MB mem-reservation=5.88MB thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.44MB mem-reservation=5.88MB thread-reservation=1
 |  30:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_sales_price)
 |  |  group by: i_manufact_id
@@ -470,7 +470,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |  hash predicates: i_manufact_id = i_manufact_id
 |  |  runtime filters: RF017[min_max] <- i_manufact_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 25(OPEN)
 |  |
 |  |--50:EXCHANGE [BROADCAST]
@@ -498,7 +498,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  runtime filters: RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 24(OPEN)
 |  |
 |  |--49:EXCHANGE [HASH(i_item_sk)]
@@ -520,23 +520,23 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |     in pipelines: 24(GETNEXT)
 |  |
 |  48:EXCHANGE [HASH(ws_item_sk)]
-|  |  mem-estimate=295.62KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  mem-estimate=289.66KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F13:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=133.97MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=133.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
 |  27:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ws_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  runtime filters: RF020[bloom] <- ca_address_sk, RF021[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |
 |  |--47:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=18 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 23(GETNEXT)
 |  |  |
 |  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -551,7 +551,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |     in pipelines: 23(GETNEXT)
 |  |
 |  26:HASH JOIN [INNER JOIN, BROADCAST]
@@ -606,7 +606,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F10:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=16.50MB mem-reservation=7.88MB thread-reservation=1 runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=16.49MB mem-reservation=7.88MB thread-reservation=1 runtime-filters-memory=2.00MB
 |  20:AGGREGATE [STREAMING]
 |  |  output: sum(cs_ext_sales_price)
 |  |  group by: i_manufact_id
@@ -618,7 +618,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |  hash predicates: i_manufact_id = i_manufact_id
 |  |  runtime filters: RF008[bloom] <- i_manufact_id, RF009[min_max] <- i_manufact_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 15(OPEN)
 |  |
 |  |--43:EXCHANGE [BROADCAST]
@@ -646,7 +646,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  runtime filters: RF010[bloom] <- i_item_sk, RF011[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 14(OPEN)
 |  |
 |  |--42:EXCHANGE [HASH(i_item_sk)]
@@ -668,23 +668,23 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  41:EXCHANGE [HASH(cs_item_sk)]
-|  |  mem-estimate=348.84KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  mem-estimate=342.84KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=198.97MB mem-reservation=14.88MB thread-reservation=2 runtime-filters-memory=3.00MB
+|  Per-Host Resources: mem-estimate=198.96MB mem-reservation=14.88MB thread-reservation=2 runtime-filters-memory=3.00MB
 |  17:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: cs_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |
 |  |--40:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=10 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=10 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 13(GETNEXT)
 |  |  |
 |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -699,7 +699,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |     in pipelines: 13(GETNEXT)
 |  |
 |  16:HASH JOIN [INNER JOIN, BROADCAST]
@@ -754,7 +754,7 @@ Per-Host Resources: mem-estimate=20.07MB mem-reservation=3.94MB thread-reservati
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=70.13MB mem-reservation=15.75MB thread-reservation=2 runtime-filters-memory=4.00MB
+Per-Host Resources: mem-estimate=70.12MB mem-reservation=15.75MB thread-reservation=2 runtime-filters-memory=4.00MB
 10:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price)
 |  group by: i_manufact_id
@@ -766,7 +766,7 @@ Per-Host Resources: mem-estimate=70.13MB mem-reservation=15.75MB thread-reservat
 |  hash predicates: i_manufact_id = i_manufact_id
 |  runtime filters: RF000[bloom] <- i_manufact_id, RF001[min_max] <- i_manufact_id
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=52B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=52B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--36:EXCHANGE [BROADCAST]
@@ -794,7 +794,7 @@ Per-Host Resources: mem-estimate=70.13MB mem-reservation=15.75MB thread-reservat
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  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=0,1,2,3 row-size=52B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=52B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--35:EXCHANGE [BROADCAST]
@@ -820,12 +820,12 @@ Per-Host Resources: mem-estimate=70.13MB mem-reservation=15.75MB thread-reservat
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  runtime filters: RF004[bloom] <- ca_address_sk, RF005[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=8B cardinality=8.07K
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -840,7 +840,7 @@ Per-Host Resources: mem-estimate=70.13MB mem-reservation=15.75MB thread-reservat
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
@@ -945,7 +945,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F17:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
-|  Per-Instance Resources: mem-estimate=10.29MB mem-reservation=2.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.28MB mem-reservation=2.00MB thread-reservation=1
 |  30:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_sales_price)
 |  |  group by: i_manufact_id
@@ -957,7 +957,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  hash-table-id=08
 |  |  hash predicates: i_manufact_id = i_manufact_id
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 25(OPEN)
 |  |
 |  |--F31:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
@@ -993,7 +993,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: ws_item_sk = i_item_sk
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=52B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 24(OPEN)
 |  |
 |  |--F32:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
@@ -1023,8 +1023,8 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |     in pipelines: 24(GETNEXT)
 |  |
 |  48:EXCHANGE [HASH(ws_item_sk)]
-|  |  mem-estimate=295.62KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  mem-estimate=289.66KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F13:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1035,7 +1035,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: ws_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |
 |  |--F33:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1047,8 +1047,8 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  47:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=18 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 23(GETNEXT)
 |  |  |
 |  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1063,7 +1063,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |     in pipelines: 23(GETNEXT)
 |  |
 |  26:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1126,7 +1126,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F10:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
-|  Per-Instance Resources: mem-estimate=10.34MB mem-reservation=2.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.33MB mem-reservation=2.00MB thread-reservation=1
 |  20:AGGREGATE [STREAMING]
 |  |  output: sum(cs_ext_sales_price)
 |  |  group by: i_manufact_id
@@ -1138,7 +1138,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  hash-table-id=04
 |  |  hash predicates: i_manufact_id = i_manufact_id
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 15(OPEN)
 |  |
 |  |--F27:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
@@ -1174,7 +1174,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: cs_item_sk = i_item_sk
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=52B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 14(OPEN)
 |  |
 |  |--F28:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
@@ -1205,8 +1205,8 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  41:EXCHANGE [HASH(cs_item_sk)]
-|  |  mem-estimate=348.84KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  mem-estimate=342.84KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1217,7 +1217,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: cs_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |
 |  |--F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1229,8 +1229,8 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  40:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=10 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=10 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 13(GETNEXT)
 |  |  |
 |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1245,7 +1245,7 @@ Per-Instance Resources: mem-estimate=20.14MB mem-reservation=3.94MB thread-reser
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |     in pipelines: 13(GETNEXT)
 |  |
 |  16:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1321,7 +1321,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser
 |  hash-table-id=00
 |  hash predicates: i_manufact_id = i_manufact_id
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=52B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=52B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1357,7 +1357,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=52B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=52B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--F24:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1392,7 +1392,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: ss_addr_sk = ca_address_sk
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1404,8 +1404,8 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=8B cardinality=8.07K
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1420,7 +1420,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=4.00MB thread-reser
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
index 2c99e4f..2bf4074 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
@@ -43,7 +43,7 @@ PLAN-ROOT SINK
 |  output: sum(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_category_id, item.i_category
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=42B cardinality=1.73K
+|  tuple-ids=3 row-size=42B cardinality=1.72K
 |  in pipelines: 05(GETNEXT), 01(OPEN)
 |
 04:HASH JOIN [INNER JOIN]
@@ -51,7 +51,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: store_sales.ss_sold_date_sk = dt.d_date_sk
 |  runtime filters: RF000[bloom] <- dt.d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0 row-size=62B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=62B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--00:SCAN HDFS [tpcds_parquet.date_dim dt]
@@ -72,7 +72,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk
 |  runtime filters: RF002[bloom] <- item.i_item_sk, RF003[min_max] <- item.i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=50B cardinality=29.12K
+|  tuple-ids=1,2 row-size=50B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.item]
@@ -85,7 +85,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: item.i_manager_id = CAST(1 AS INT)
 |     parquet dictionary predicates: item.i_manager_id = CAST(1 AS INT)
 |     mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=34B cardinality=182
+|     tuple-ids=2 row-size=34B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales]
@@ -127,12 +127,12 @@ Per-Host Resources: mem-estimate=10.16MB mem-reservation=1.94MB thread-reservati
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_category_id, item.i_category
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=42B cardinality=1.73K
+|  tuple-ids=3 row-size=42B cardinality=1.72K
 |  in pipelines: 10(GETNEXT), 01(OPEN)
 |
 09:EXCHANGE [HASH(dt.d_year,item.i_category_id,item.i_category)]
-|  mem-estimate=161.21KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=3 row-size=42B cardinality=1.73K
+|  mem-estimate=161.09KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=42B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -141,7 +141,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |  output: sum(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_category_id, item.i_category
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=42B cardinality=1.73K
+|  tuple-ids=3 row-size=42B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -149,7 +149,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |  fk/pk conjuncts: store_sales.ss_sold_date_sk = dt.d_date_sk
 |  runtime filters: RF000[bloom] <- dt.d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0 row-size=62B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=62B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--08:EXCHANGE [BROADCAST]
@@ -177,12 +177,12 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |  fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk
 |  runtime filters: RF002[bloom] <- item.i_item_sk, RF003[min_max] <- item.i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=50B cardinality=29.12K
+|  tuple-ids=1,2 row-size=50B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--07:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=34B cardinality=182
+|  |  tuple-ids=2 row-size=34B cardinality=181
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -197,7 +197,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |     parquet statistics predicates: item.i_manager_id = CAST(1 AS INT)
 |     parquet dictionary predicates: item.i_manager_id = CAST(1 AS INT)
 |     mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=34B cardinality=182
+|     tuple-ids=2 row-size=34B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
@@ -239,12 +239,12 @@ Per-Instance Resources: mem-estimate=10.29MB mem-reservation=1.94MB thread-reser
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_category_id, item.i_category
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=42B cardinality=1.73K
+|  tuple-ids=3 row-size=42B cardinality=1.72K
 |  in pipelines: 10(GETNEXT), 01(OPEN)
 |
 09:EXCHANGE [HASH(dt.d_year,item.i_category_id,item.i_category)]
-|  mem-estimate=298.90KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=3 row-size=42B cardinality=1.73K
+|  mem-estimate=298.78KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=42B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -254,7 +254,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  output: sum(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_category_id, item.i_category
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=42B cardinality=1.73K
+|  tuple-ids=3 row-size=42B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -262,7 +262,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  hash predicates: store_sales.ss_sold_date_sk = dt.d_date_sk
 |  fk/pk conjuncts: store_sales.ss_sold_date_sk = dt.d_date_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0 row-size=62B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=62B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -298,7 +298,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  hash predicates: store_sales.ss_item_sk = item.i_item_sk
 |  fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=50B cardinality=29.12K
+|  tuple-ids=1,2 row-size=50B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -311,7 +311,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  |
 |  07:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=34B cardinality=182
+|  |  tuple-ids=2 row-size=34B cardinality=181
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -326,7 +326,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |     parquet statistics predicates: item.i_manager_id = CAST(1 AS INT)
 |     parquet dictionary predicates: item.i_manager_id = CAST(1 AS INT)
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=2 row-size=34B cardinality=182
+|     tuple-ids=2 row-size=34B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
index a78e10b..53f360f 100755
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
@@ -129,7 +129,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |  |  |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |  |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |  |     tuple-ids=17 row-size=12B cardinality=53.06K
+|  |  |     tuple-ids=17 row-size=12B cardinality=51.85K
 |  |  |     in pipelines: 11(GETNEXT)
 |  |  |
 |  |  10:AGGREGATE [FINALIZE]
@@ -150,7 +150,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |  |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=13 row-size=16B cardinality=480.07K
+|  |     tuple-ids=13 row-size=16B cardinality=458.39K
 |  |     in pipelines: 09(GETNEXT)
 |  |
 |  20:HASH JOIN [INNER JOIN]
@@ -214,7 +214,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |  |  |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |  |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |  |     tuple-ids=4 row-size=12B cardinality=53.06K
+|  |  |     tuple-ids=4 row-size=12B cardinality=51.85K
 |  |  |     in pipelines: 02(GETNEXT)
 |  |  |
 |  |  01:AGGREGATE [FINALIZE]
@@ -235,7 +235,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |  |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=16B cardinality=480.07K
+|  |     tuple-ids=0 row-size=16B cardinality=458.39K
 |  |     in pipelines: 00(GETNEXT)
 |  |
 |  18:SCAN HDFS [tpcds_parquet.item i1]
@@ -408,7 +408,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |  |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=17 row-size=12B cardinality=53.06K
+|  |     tuple-ids=17 row-size=12B cardinality=51.85K
 |  |     in pipelines: 11(GETNEXT)
 |  |
 |  34:AGGREGATE [FINALIZE]
@@ -443,7 +443,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=13 row-size=16B cardinality=480.07K
+|     tuple-ids=13 row-size=16B cardinality=458.39K
 |     in pipelines: 09(GETNEXT)
 |
 20:HASH JOIN [INNER JOIN, BROADCAST]
@@ -566,7 +566,7 @@ Per-Host Resources: mem-estimate=10.17MB mem-reservation=1.94MB thread-reservati
 |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=4 row-size=12B cardinality=53.06K
+|     tuple-ids=4 row-size=12B cardinality=51.85K
 |     in pipelines: 02(GETNEXT)
 |
 25:AGGREGATE [FINALIZE]
@@ -601,7 +601,7 @@ Per-Host Resources: mem-estimate=58.00MB mem-reservation=3.00MB thread-reservati
    parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
    parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=0 row-size=16B cardinality=480.07K
+   tuple-ids=0 row-size=16B cardinality=458.39K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=63.88MB Threads=27
@@ -775,7 +775,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |  |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=17 row-size=12B cardinality=53.06K
+|  |     tuple-ids=17 row-size=12B cardinality=51.85K
 |  |     in pipelines: 11(GETNEXT)
 |  |
 |  34:AGGREGATE [FINALIZE]
@@ -810,7 +810,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=13 row-size=16B cardinality=480.07K
+|     tuple-ids=13 row-size=16B cardinality=458.39K
 |     in pipelines: 09(GETNEXT)
 |
 20:HASH JOIN [INNER JOIN, BROADCAST]
@@ -948,7 +948,7 @@ Per-Instance Resources: mem-estimate=10.21MB mem-reservation=1.94MB thread-reser
 |     parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
 |     parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=4 row-size=12B cardinality=53.06K
+|     tuple-ids=4 row-size=12B cardinality=51.85K
 |     in pipelines: 02(GETNEXT)
 |
 25:AGGREGATE [FINALIZE]
@@ -983,6 +983,6 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
    parquet statistics predicates: ss_store_sk = CAST(4 AS INT)
    parquet dictionary predicates: ss_store_sk = CAST(4 AS INT)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=0 row-size=16B cardinality=480.07K
+   tuple-ids=0 row-size=16B cardinality=458.39K
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
index 09f63a4..7a41475 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
@@ -59,7 +59,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF000[bloom] <- s_store_sk, RF001[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=2,0,3,4,1 row-size=122B cardinality=8.70K
+|  tuple-ids=2,0,3,4,1 row-size=122B cardinality=8.56K
 |  in pipelines: 02(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds_parquet.store]
@@ -78,7 +78,7 @@ PLAN-ROOT SINK
 |  other predicates: cd_marital_status = 'M' OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price >= CAST(150.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ss_sales_price >= CAS [...]
 |  runtime filters: RF002[bloom] <- ss_cdemo_sk, RF003[min_max] <- ss_cdemo_sk
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=2,0,3,4 row-size=118B cardinality=8.70K
+|  tuple-ids=2,0,3,4 row-size=118B cardinality=8.56K
 |  in pipelines: 02(GETNEXT), 00(OPEN)
 |
 |--06:HASH JOIN [INNER JOIN]
@@ -86,7 +86,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  runtime filters: RF004[bloom] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3,4 row-size=79B cardinality=10.43K
+|  |  tuple-ids=0,3,4 row-size=79B cardinality=10.26K
 |  |  in pipelines: 00(GETNEXT), 04(OPEN)
 |  |
 |  |--04:SCAN HDFS [tpcds_parquet.date_dim]
@@ -108,7 +108,7 @@ PLAN-ROOT SINK
 |  |  other predicates: ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit <= CAST(3000 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(25000 AS DECIMAL(5,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit <= CAST(3000 AS DECIMAL(5,0)) OR ss_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(25000 AS DECIMAL(5,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_prof [...]
 |  |  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3 row-size=71B cardinality=50.98K
+|  |  tuple-ids=0,3 row-size=71B cardinality=50.19K
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--03:SCAN HDFS [tpcds_parquet.customer_address]
@@ -121,7 +121,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_state IN ('CO', 'OH', 'TX', 'OR', 'MN', 'KY', 'VA', 'CA', 'MS'), ca_country = 'United States'
 |  |     parquet dictionary predicates: ca_state IN ('CO', 'OH', 'TX', 'OR', 'MN', 'KY', 'VA', 'CA', 'MS'), ca_country = 'United States'
 |  |     mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=3 row-size=43B cardinality=8.82K
+|  |     tuple-ids=3 row-size=43B cardinality=8.69K
 |  |     in pipelines: 03(GETNEXT)
 |  |
 |  00:SCAN HDFS [tpcds_parquet.store_sales]
@@ -170,7 +170,7 @@ PLAN-ROOT SINK
 |  in pipelines: 09(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=19.71MB mem-reservation=12.44MB thread-reservation=1 runtime-filters-memory=2.00MB
+Per-Host Resources: mem-estimate=19.70MB mem-reservation=12.44MB thread-reservation=1 runtime-filters-memory=2.00MB
 09:AGGREGATE
 |  output: sum(CAST(ss_quantity AS BIGINT))
 |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -182,7 +182,7 @@ Per-Host Resources: mem-estimate=19.71MB mem-reservation=12.44MB thread-reservat
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF000[bloom] <- s_store_sk, RF001[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,4,2,1 row-size=122B cardinality=8.70K
+|  tuple-ids=0,3,4,2,1 row-size=122B cardinality=8.56K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--14:EXCHANGE [BROADCAST]
@@ -208,7 +208,7 @@ Per-Host Resources: mem-estimate=19.71MB mem-reservation=12.44MB thread-reservat
 |  other predicates: cd_marital_status = 'M' OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price >= CAST(150.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ss_sales_price >= CAS [...]
 |  runtime filters: RF002[bloom] <- cd_demo_sk, RF003[min_max] <- cd_demo_sk
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,3,4,2 row-size=118B cardinality=8.70K
+|  tuple-ids=0,3,4,2 row-size=118B cardinality=8.56K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--13:EXCHANGE [HASH(cd_demo_sk)]
@@ -230,18 +230,18 @@ Per-Host Resources: mem-estimate=19.71MB mem-reservation=12.44MB thread-reservat
 |     in pipelines: 02(GETNEXT)
 |
 12:EXCHANGE [HASH(ss_cdemo_sk)]
-|  mem-estimate=541.09KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,4 row-size=79B cardinality=10.43K
+|  mem-estimate=536.95KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,4 row-size=79B cardinality=10.26K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=104.30MB mem-reservation=9.88MB thread-reservation=2 runtime-filters-memory=4.00MB
+Per-Host Resources: mem-estimate=104.29MB mem-reservation=9.88MB thread-reservation=2 runtime-filters-memory=4.00MB
 06: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: RF004[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,4 row-size=79B cardinality=10.43K
+|  tuple-ids=0,3,4 row-size=79B cardinality=10.26K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--11:EXCHANGE [BROADCAST]
@@ -270,12 +270,12 @@ Per-Host Resources: mem-estimate=104.30MB mem-reservation=9.88MB thread-reservat
 |  other predicates: ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit <= CAST(3000 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(25000 AS DECIMAL(5,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit <= CAST(3000 AS DECIMAL(5,0)) OR ss_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(25000 AS DECIMAL(5,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit  [...]
 |  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=71B cardinality=50.98K
+|  tuple-ids=0,3 row-size=71B cardinality=50.19K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--10:EXCHANGE [BROADCAST]
-|  |  mem-estimate=417.54KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=43B cardinality=8.82K
+|  |  mem-estimate=411.83KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=43B cardinality=8.69K
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -290,7 +290,7 @@ Per-Host Resources: mem-estimate=104.30MB mem-reservation=9.88MB thread-reservat
 |     parquet statistics predicates: ca_state IN ('CO', 'OH', 'TX', 'OR', 'MN', 'KY', 'VA', 'CA', 'MS'), ca_country = 'United States'
 |     parquet dictionary predicates: ca_state IN ('CO', 'OH', 'TX', 'OR', 'MN', 'KY', 'VA', 'CA', 'MS'), ca_country = 'United States'
 |     mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=3 row-size=43B cardinality=8.82K
+|     tuple-ids=3 row-size=43B cardinality=8.69K
 |     in pipelines: 03(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
@@ -327,7 +327,7 @@ PLAN-ROOT SINK
 |  in pipelines: 09(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=830.09KB mem-reservation=0B thread-reservation=1
+Per-Instance Resources: mem-estimate=825.95KB mem-reservation=0B thread-reservation=1
 09:AGGREGATE
 |  output: sum(CAST(ss_quantity AS BIGINT))
 |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -339,7 +339,7 @@ Per-Instance Resources: mem-estimate=830.09KB mem-reservation=0B thread-reservat
 |  hash predicates: ss_store_sk = s_store_sk
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,4,2,1 row-size=122B cardinality=8.70K
+|  tuple-ids=0,3,4,2,1 row-size=122B cardinality=8.56K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--F07:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=3
@@ -373,7 +373,7 @@ Per-Instance Resources: mem-estimate=830.09KB mem-reservation=0B thread-reservat
 |  fk/pk conjuncts: ss_cdemo_sk = cd_demo_sk
 |  other predicates: cd_marital_status = 'M' OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price <= CAST(100.00 AS DECIMAL(5,2)) OR ss_sales_price >= CAST(150.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ss_sales_price <= CAST(200.00 AS DECIMAL(5,2)), cd_marital_status = 'M' OR ss_sales_price >= CAST(50.00 AS DECIMAL(4,2)) OR ss_sales_price >= CAS [...]
 |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,3,4,2 row-size=118B cardinality=8.70K
+|  tuple-ids=0,3,4,2 row-size=118B cardinality=8.56K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--F08:PLAN FRAGMENT [HASH(ss_cdemo_sk)] hosts=3 instances=6
@@ -403,8 +403,8 @@ Per-Instance Resources: mem-estimate=830.09KB mem-reservation=0B thread-reservat
 |     in pipelines: 02(GETNEXT)
 |
 12:EXCHANGE [HASH(ss_cdemo_sk)]
-|  mem-estimate=814.09KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,4 row-size=79B cardinality=10.43K
+|  mem-estimate=809.95KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,3,4 row-size=79B cardinality=10.26K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -415,7 +415,7 @@ Per-Instance Resources: mem-estimate=16.00MB 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=0,3,4 row-size=79B cardinality=10.43K
+|  tuple-ids=0,3,4 row-size=79B cardinality=10.26K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -452,7 +452,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  other predicates: ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit <= CAST(3000 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(25000 AS DECIMAL(5,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit <= CAST(3000 AS DECIMAL(5,0)) OR ss_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit <= CAST(25000 AS DECIMAL(5,0)), ca_state IN ('CO', 'OH', 'TX') OR ss_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ss_net_profit  [...]
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=71B cardinality=50.98K
+|  tuple-ids=0,3 row-size=71B cardinality=50.19K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -464,8 +464,8 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  10:EXCHANGE [BROADCAST]
-|  |  mem-estimate=417.54KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=3 row-size=43B cardinality=8.82K
+|  |  mem-estimate=411.83KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=3 row-size=43B cardinality=8.69K
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -480,7 +480,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |     parquet statistics predicates: ca_state IN ('CO', 'OH', 'TX', 'OR', 'MN', 'KY', 'VA', 'CA', 'MS'), ca_country = 'United States'
 |     parquet dictionary predicates: ca_state IN ('CO', 'OH', 'TX', 'OR', 'MN', 'KY', 'VA', 'CA', 'MS'), ca_country = 'United States'
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=3 row-size=43B cardinality=8.82K
+|     tuple-ids=3 row-size=43B cardinality=8.69K
 |     in pipelines: 03(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
index 019d103..45ba9ba 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
@@ -42,7 +42,7 @@ PLAN-ROOT SINK
 |  output: sum(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_brand, item.i_brand_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=52B cardinality=1.73K
+|  tuple-ids=3 row-size=52B cardinality=1.72K
 |  in pipelines: 05(GETNEXT), 01(OPEN)
 |
 04:HASH JOIN [INNER JOIN]
@@ -50,7 +50,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: store_sales.ss_sold_date_sk = dt.d_date_sk
 |  runtime filters: RF000[bloom] <- dt.d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0 row-size=72B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=72B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--00:SCAN HDFS [tpcds_parquet.date_dim dt]
@@ -71,7 +71,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk
 |  runtime filters: RF002[bloom] <- item.i_item_sk, RF003[min_max] <- item.i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=60B cardinality=29.12K
+|  tuple-ids=1,2 row-size=60B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.item]
@@ -84,7 +84,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: item.i_manager_id = CAST(1 AS INT)
 |     parquet dictionary predicates: item.i_manager_id = CAST(1 AS INT)
 |     mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=44B cardinality=182
+|     tuple-ids=2 row-size=44B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales]
@@ -126,12 +126,12 @@ Per-Host Resources: mem-estimate=10.19MB mem-reservation=1.94MB thread-reservati
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_brand, item.i_brand_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=52B cardinality=1.73K
+|  tuple-ids=3 row-size=52B cardinality=1.72K
 |  in pipelines: 10(GETNEXT), 01(OPEN)
 |
 09:EXCHANGE [HASH(dt.d_year,item.i_brand,item.i_brand_id)]
-|  mem-estimate=197.90KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=3 row-size=52B cardinality=1.73K
+|  mem-estimate=197.75KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=52B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -140,7 +140,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |  output: sum(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_brand, item.i_brand_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=52B cardinality=1.73K
+|  tuple-ids=3 row-size=52B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -148,7 +148,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |  fk/pk conjuncts: store_sales.ss_sold_date_sk = dt.d_date_sk
 |  runtime filters: RF000[bloom] <- dt.d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0 row-size=72B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=72B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--08:EXCHANGE [BROADCAST]
@@ -176,12 +176,12 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |  fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk
 |  runtime filters: RF002[bloom] <- item.i_item_sk, RF003[min_max] <- item.i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=60B cardinality=29.12K
+|  tuple-ids=1,2 row-size=60B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--07:EXCHANGE [BROADCAST]
-|  |  mem-estimate=16.42KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=44B cardinality=182
+|  |  mem-estimate=16.33KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=44B cardinality=181
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -196,7 +196,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |     parquet statistics predicates: item.i_manager_id = CAST(1 AS INT)
 |     parquet dictionary predicates: item.i_manager_id = CAST(1 AS INT)
 |     mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=44B cardinality=182
+|     tuple-ids=2 row-size=44B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
@@ -238,12 +238,12 @@ Per-Instance Resources: mem-estimate=10.36MB mem-reservation=1.94MB thread-reser
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_brand, item.i_brand_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=52B cardinality=1.73K
+|  tuple-ids=3 row-size=52B cardinality=1.72K
 |  in pipelines: 10(GETNEXT), 01(OPEN)
 |
 09:EXCHANGE [HASH(dt.d_year,item.i_brand,item.i_brand_id)]
-|  mem-estimate=366.49KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=3 row-size=52B cardinality=1.73K
+|  mem-estimate=366.34KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=52B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -253,7 +253,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  output: sum(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_brand, item.i_brand_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=52B cardinality=1.73K
+|  tuple-ids=3 row-size=52B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -261,7 +261,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  hash predicates: store_sales.ss_sold_date_sk = dt.d_date_sk
 |  fk/pk conjuncts: store_sales.ss_sold_date_sk = dt.d_date_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2,0 row-size=72B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=72B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -297,7 +297,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  hash predicates: store_sales.ss_item_sk = item.i_item_sk
 |  fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=60B cardinality=29.12K
+|  tuple-ids=1,2 row-size=60B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -309,8 +309,8 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  07:EXCHANGE [BROADCAST]
-|  |  mem-estimate=16.42KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=44B cardinality=182
+|  |  mem-estimate=16.33KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=44B cardinality=181
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -325,7 +325,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |     parquet statistics predicates: item.i_manager_id = CAST(1 AS INT)
 |     parquet dictionary predicates: item.i_manager_id = CAST(1 AS INT)
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=2 row-size=44B cardinality=182
+|     tuple-ids=2 row-size=44B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
index cba0f05..1155263 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
@@ -39,7 +39,7 @@ PLAN-ROOT SINK
 |  output: sum(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=48B cardinality=1.73K
+|  tuple-ids=3 row-size=48B cardinality=1.72K
 |  in pipelines: 05(GETNEXT), 01(OPEN)
 |
 04:HASH JOIN [INNER JOIN]
@@ -47,7 +47,7 @@ PLAN-ROOT SINK
 |  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=1,2,0 row-size=72B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=72B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--00:SCAN HDFS [tpcds_parquet.date_dim]
@@ -68,7 +68,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  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=1,2 row-size=60B cardinality=29.12K
+|  tuple-ids=1,2 row-size=60B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.item]
@@ -81,7 +81,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: i_manager_id = CAST(36 AS INT)
 |     parquet dictionary predicates: i_manager_id = CAST(36 AS INT)
 |     mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=44B cardinality=182
+|     tuple-ids=2 row-size=44B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales]
@@ -123,12 +123,12 @@ Per-Host Resources: mem-estimate=10.18MB mem-reservation=1.94MB thread-reservati
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=48B cardinality=1.73K
+|  tuple-ids=3 row-size=48B cardinality=1.72K
 |  in pipelines: 10(GETNEXT), 01(OPEN)
 |
 09:EXCHANGE [HASH(i_brand,i_brand_id)]
-|  mem-estimate=183.66KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=3 row-size=48B cardinality=1.73K
+|  mem-estimate=183.51KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=48B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -137,7 +137,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |  output: sum(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=48B cardinality=1.73K
+|  tuple-ids=3 row-size=48B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -145,7 +145,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB 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=1,2,0 row-size=72B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=72B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--08:EXCHANGE [BROADCAST]
@@ -173,12 +173,12 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  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=1,2 row-size=60B cardinality=29.12K
+|  tuple-ids=1,2 row-size=60B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--07:EXCHANGE [BROADCAST]
-|  |  mem-estimate=16.42KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=44B cardinality=182
+|  |  mem-estimate=16.33KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=44B cardinality=181
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -193,7 +193,7 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
 |     parquet statistics predicates: i_manager_id = CAST(36 AS INT)
 |     parquet dictionary predicates: i_manager_id = CAST(36 AS INT)
 |     mem-estimate=64.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=44B cardinality=182
+|     tuple-ids=2 row-size=44B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
@@ -235,12 +235,12 @@ Per-Instance Resources: mem-estimate=10.33MB mem-reservation=1.94MB thread-reser
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=48B cardinality=1.73K
+|  tuple-ids=3 row-size=48B cardinality=1.72K
 |  in pipelines: 10(GETNEXT), 01(OPEN)
 |
 09:EXCHANGE [HASH(i_brand,i_brand_id)]
-|  mem-estimate=340.25KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=3 row-size=48B cardinality=1.73K
+|  mem-estimate=340.11KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=48B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -250,7 +250,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  output: sum(ss_ext_sales_price)
 |  group by: i_brand, i_brand_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3 row-size=48B cardinality=1.73K
+|  tuple-ids=3 row-size=48B cardinality=1.72K
 |  in pipelines: 01(GETNEXT)
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -258,7 +258,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.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=1,2,0 row-size=72B cardinality=1.73K
+|  tuple-ids=1,2,0 row-size=72B cardinality=1.72K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -294,7 +294,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,2 row-size=60B cardinality=29.12K
+|  tuple-ids=1,2 row-size=60B cardinality=28.96K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -306,8 +306,8 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  07:EXCHANGE [BROADCAST]
-|  |  mem-estimate=16.42KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=44B cardinality=182
+|  |  mem-estimate=16.33KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=44B cardinality=181
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -322,7 +322,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |     parquet statistics predicates: i_manager_id = CAST(36 AS INT)
 |     parquet dictionary predicates: i_manager_id = CAST(36 AS INT)
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=2 row-size=44B cardinality=182
+|     tuple-ids=2 row-size=44B cardinality=181
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
index 2824466..bce620e 100755
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
@@ -92,26 +92,26 @@ PLAN-ROOT SINK
 |  output: sum(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=3.52K
+|  tuple-ids=26 row-size=44B cardinality=3.41K
 |  in pipelines: 31(GETNEXT), 10(OPEN), 20(OPEN), 30(OPEN)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=24 row-size=44B cardinality=3.52K
+|  tuple-ids=24 row-size=44B cardinality=3.41K
 |  in pipelines: 10(GETNEXT), 20(GETNEXT), 30(GETNEXT)
 |
 |--30:AGGREGATE [FINALIZE]
 |  |  output: sum(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=650
+|  |  tuple-ids=22 row-size=44B cardinality=630
 |  |  in pipelines: 30(GETNEXT), 24(OPEN)
 |  |
 |  29:HASH JOIN [LEFT SEMI JOIN]
 |  |  hash predicates: i_item_id = i_item_id
 |  |  runtime filters: RF016[bloom] <- i_item_id, RF017[min_max] <- i_item_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=650
+|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=630
 |  |  in pipelines: 24(GETNEXT), 25(OPEN)
 |  |
 |  |--25:SCAN HDFS [tpcds_parquet.item]
@@ -132,7 +132,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: i_item_sk = ws_item_sk
 |  |  runtime filters: RF018[bloom] <- ws_item_sk, RF019[min_max] <- ws_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=9.81K
+|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=9.51K
 |  |  in pipelines: 24(GETNEXT), 21(OPEN)
 |  |
 |  |--27:HASH JOIN [INNER JOIN]
@@ -140,7 +140,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  |  runtime filters: RF020[bloom] <- ca_address_sk, RF021[min_max] <- ca_address_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |  |
 |  |  |--23:SCAN HDFS [tpcds_parquet.customer_address]
@@ -153,7 +153,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |  |     in pipelines: 23(GETNEXT)
 |  |  |
 |  |  26:HASH JOIN [INNER JOIN]
@@ -203,14 +203,14 @@ PLAN-ROOT SINK
 |  |  output: sum(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=982
+|  |  tuple-ids=14 row-size=44B cardinality=951
 |  |  in pipelines: 20(GETNEXT), 14(OPEN)
 |  |
 |  19:HASH JOIN [LEFT SEMI JOIN]
 |  |  hash predicates: i_item_id = i_item_id
 |  |  runtime filters: RF008[bloom] <- i_item_id, RF009[min_max] <- i_item_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=982
+|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=951
 |  |  in pipelines: 14(GETNEXT), 15(OPEN)
 |  |
 |  |--15:SCAN HDFS [tpcds_parquet.item]
@@ -231,7 +231,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: i_item_sk = cs_item_sk
 |  |  runtime filters: RF011[min_max] <- cs_item_sk
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=14.81K
+|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=14.35K
 |  |  in pipelines: 14(GETNEXT), 11(OPEN)
 |  |
 |  |--17:HASH JOIN [INNER JOIN]
@@ -239,7 +239,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |  |
 |  |  |--13:SCAN HDFS [tpcds_parquet.customer_address]
@@ -252,7 +252,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |  |     in pipelines: 13(GETNEXT)
 |  |  |
 |  |  16:HASH JOIN [INNER JOIN]
@@ -302,14 +302,14 @@ PLAN-ROOT SINK
 |  output: sum(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=1.89K
+|  tuple-ids=6 row-size=44B cardinality=1.83K
 |  in pipelines: 10(GETNEXT), 01(OPEN)
 |
 09:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: i_item_id = i_item_id
 |  runtime filters: RF000[bloom] <- i_item_id, RF001[min_max] <- i_item_id
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=1.89K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=1.83K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--05:SCAN HDFS [tpcds_parquet.item]
@@ -330,7 +330,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF003[min_max] <- i_item_sk
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--04:SCAN HDFS [tpcds_parquet.item]
@@ -349,7 +349,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  runtime filters: RF004[bloom] <- ca_address_sk, RF005[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--03:SCAN HDFS [tpcds_parquet.customer_address]
@@ -362,7 +362,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN]
@@ -425,12 +425,12 @@ Per-Host Resources: mem-estimate=10.19MB mem-reservation=1.94MB thread-reservati
 |  output: sum:merge(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=3.52K
+|  tuple-ids=26 row-size=44B cardinality=3.41K
 |  in pipelines: 55(GETNEXT), 39(OPEN), 46(OPEN), 53(OPEN)
 |
 54:EXCHANGE [HASH(i_item_id)]
-|  mem-estimate=194.45KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=3.52K
+|  mem-estimate=192.87KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=26 row-size=44B cardinality=3.41K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -439,24 +439,24 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  output: sum(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=3.52K
+|  tuple-ids=26 row-size=44B cardinality=3.41K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=24 row-size=44B cardinality=3.52K
+|  tuple-ids=24 row-size=44B cardinality=3.41K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 |--53:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=650
+|  |  tuple-ids=22 row-size=44B cardinality=630
 |  |  in pipelines: 53(GETNEXT), 21(OPEN)
 |  |
 |  52:EXCHANGE [HASH(i_item_id)]
-|  |  mem-estimate=74.90KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=650
+|  |  mem-estimate=72.60KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=22 row-size=44B cardinality=630
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F18:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
@@ -465,14 +465,14 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  |  output: sum(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=650
+|  |  tuple-ids=22 row-size=44B cardinality=630
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  29:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash predicates: i_item_id = i_item_id
 |  |  runtime filters: RF016[bloom] <- i_item_id, RF017[min_max] <- i_item_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=650
+|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=630
 |  |  in pipelines: 21(GETNEXT), 25(OPEN)
 |  |
 |  |--51:EXCHANGE [BROADCAST]
@@ -500,7 +500,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  runtime filters: RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 24(OPEN)
 |  |
 |  |--50:EXCHANGE [HASH(i_item_sk)]
@@ -522,23 +522,23 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  |     in pipelines: 24(GETNEXT)
 |  |
 |  49:EXCHANGE [HASH(ws_item_sk)]
-|  |  mem-estimate=295.62KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  mem-estimate=289.66KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=133.97MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=133.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
 |  27:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ws_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  runtime filters: RF020[bloom] <- ca_address_sk, RF021[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |
 |  |--48:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=18 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 23(GETNEXT)
 |  |  |
 |  |  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -553,7 +553,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |     in pipelines: 23(GETNEXT)
 |  |
 |  26:HASH JOIN [INNER JOIN, BROADCAST]
@@ -599,28 +599,28 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  |  output: sum:merge(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=982
+|  |  tuple-ids=14 row-size=44B cardinality=951
 |  |  in pipelines: 46(GETNEXT), 11(OPEN)
 |  |
 |  45:EXCHANGE [HASH(i_item_id)]
-|  |  mem-estimate=152.16KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=982
+|  |  mem-estimate=147.36KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=14 row-size=44B cardinality=951
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F11:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=15.93MB mem-reservation=6.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=15.92MB mem-reservation=6.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  20:AGGREGATE [STREAMING]
 |  |  output: sum(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=982
+|  |  tuple-ids=14 row-size=44B cardinality=951
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  19:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash predicates: i_item_id = i_item_id
 |  |  runtime filters: RF008[bloom] <- i_item_id, RF009[min_max] <- i_item_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=982
+|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=951
 |  |  in pipelines: 11(GETNEXT), 15(OPEN)
 |  |
 |  |--44:EXCHANGE [BROADCAST]
@@ -648,7 +648,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  runtime filters: RF011[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 14(OPEN)
 |  |
 |  |--43:EXCHANGE [HASH(i_item_sk)]
@@ -670,23 +670,23 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  42:EXCHANGE [HASH(cs_item_sk)]
-|  |  mem-estimate=348.84KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  mem-estimate=342.84KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=197.97MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=197.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
 |  17:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: cs_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |
 |  |--41:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=10 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=10 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 13(GETNEXT)
 |  |  |
 |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -701,7 +701,7 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |     in pipelines: 13(GETNEXT)
 |  |
 |  16:HASH JOIN [INNER JOIN, BROADCAST]
@@ -747,28 +747,28 @@ Per-Host Resources: mem-estimate=20.17MB mem-reservation=3.94MB thread-reservati
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=1.89K
+|  tuple-ids=6 row-size=44B cardinality=1.83K
 |  in pipelines: 39(GETNEXT), 01(OPEN)
 |
 38:EXCHANGE [HASH(i_item_id)]
-|  mem-estimate=171.07KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=1.89K
+|  mem-estimate=170.22KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=44B cardinality=1.83K
 |  in pipelines: 01(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=17.10MB mem-reservation=7.88MB thread-reservation=1 runtime-filters-memory=2.00MB
+Per-Host Resources: mem-estimate=17.09MB mem-reservation=7.88MB thread-reservation=1 runtime-filters-memory=2.00MB
 10:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=1.89K
+|  tuple-ids=6 row-size=44B cardinality=1.83K
 |  in pipelines: 01(GETNEXT)
 |
 09:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  hash predicates: i_item_id = i_item_id
 |  runtime filters: RF000[bloom] <- i_item_id, RF001[min_max] <- i_item_id
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=1.89K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=1.83K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--37:EXCHANGE [BROADCAST]
@@ -796,7 +796,7 @@ Per-Host Resources: mem-estimate=17.10MB mem-reservation=7.88MB thread-reservati
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  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=0,1,2,3 row-size=76B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--36:EXCHANGE [HASH(i_item_sk)]
@@ -818,23 +818,23 @@ Per-Host Resources: mem-estimate=17.10MB mem-reservation=7.88MB thread-reservati
 |     in pipelines: 04(GETNEXT)
 |
 35:EXCHANGE [HASH(ss_item_sk)]
-|  mem-estimate=527.13KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  mem-estimate=515.61KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=54.97MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=3.00MB
+Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=3.00MB
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_addr_sk = ca_address_sk
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  runtime filters: RF004[bloom] <- ca_address_sk, RF005[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=8B cardinality=8.07K
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -849,7 +849,7 @@ Per-Host Resources: mem-estimate=54.97MB mem-reservation=8.88MB thread-reservati
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
@@ -919,12 +919,12 @@ Per-Instance Resources: mem-estimate=10.33MB mem-reservation=1.94MB thread-reser
 |  output: sum:merge(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=3.52K
+|  tuple-ids=26 row-size=44B cardinality=3.41K
 |  in pipelines: 55(GETNEXT), 39(OPEN), 46(OPEN), 53(OPEN)
 |
 54:EXCHANGE [HASH(i_item_id)]
-|  mem-estimate=338.45KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=3.52K
+|  mem-estimate=336.87KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=26 row-size=44B cardinality=3.41K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -933,40 +933,40 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  output: sum(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=3.52K
+|  tuple-ids=26 row-size=44B cardinality=3.41K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=24 row-size=44B cardinality=3.52K
+|  tuple-ids=24 row-size=44B cardinality=3.41K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 |--53:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=650
+|  |  tuple-ids=22 row-size=44B cardinality=630
 |  |  in pipelines: 53(GETNEXT), 21(OPEN)
 |  |
 |  52:EXCHANGE [HASH(i_item_id)]
-|  |  mem-estimate=74.90KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=650
+|  |  mem-estimate=72.60KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=22 row-size=44B cardinality=630
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F18:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
-|  Per-Instance Resources: mem-estimate=10.29MB mem-reservation=2.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.28MB mem-reservation=2.00MB thread-reservation=1
 |  30:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=650
+|  |  tuple-ids=22 row-size=44B cardinality=630
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  29:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash-table-id=08
 |  |  hash predicates: i_item_id = i_item_id
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=650
+|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=630
 |  |  in pipelines: 21(GETNEXT), 25(OPEN)
 |  |
 |  |--F32:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
@@ -1002,7 +1002,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: ws_item_sk = i_item_sk
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 24(OPEN)
 |  |
 |  |--F33:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
@@ -1033,8 +1033,8 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |     in pipelines: 24(GETNEXT)
 |  |
 |  49:EXCHANGE [HASH(ws_item_sk)]
-|  |  mem-estimate=295.62KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  mem-estimate=289.66KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1045,7 +1045,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: ws_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |
 |  |--F34:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1057,8 +1057,8 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  48:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=18 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 23(GETNEXT)
 |  |  |
 |  |  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1073,7 +1073,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |     in pipelines: 23(GETNEXT)
 |  |
 |  26:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1127,28 +1127,28 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |  output: sum:merge(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=982
+|  |  tuple-ids=14 row-size=44B cardinality=951
 |  |  in pipelines: 46(GETNEXT), 11(OPEN)
 |  |
 |  45:EXCHANGE [HASH(i_item_id)]
-|  |  mem-estimate=152.16KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=982
+|  |  mem-estimate=147.36KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=14 row-size=44B cardinality=951
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F11:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
-|  Per-Instance Resources: mem-estimate=10.34MB mem-reservation=2.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.33MB mem-reservation=2.00MB thread-reservation=1
 |  20:AGGREGATE [STREAMING]
 |  |  output: sum(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=982
+|  |  tuple-ids=14 row-size=44B cardinality=951
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  19:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash-table-id=04
 |  |  hash predicates: i_item_id = i_item_id
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=982
+|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=951
 |  |  in pipelines: 11(GETNEXT), 15(OPEN)
 |  |
 |  |--F28:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
@@ -1184,7 +1184,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: cs_item_sk = i_item_sk
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 14(OPEN)
 |  |
 |  |--F29:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
@@ -1215,8 +1215,8 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  42:EXCHANGE [HASH(cs_item_sk)]
-|  |  mem-estimate=348.84KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  mem-estimate=342.84KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1227,7 +1227,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: cs_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |
 |  |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1239,8 +1239,8 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  41:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=10 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=10 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 13(GETNEXT)
 |  |  |
 |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1255,7 +1255,7 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |     in pipelines: 13(GETNEXT)
 |  |
 |  16:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1309,28 +1309,28 @@ Per-Instance Resources: mem-estimate=20.31MB mem-reservation=3.94MB thread-reser
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=1.89K
+|  tuple-ids=6 row-size=44B cardinality=1.83K
 |  in pipelines: 39(GETNEXT), 01(OPEN)
 |
 38:EXCHANGE [HASH(i_item_id)]
-|  mem-estimate=315.07KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=1.89K
+|  mem-estimate=314.22KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=44B cardinality=1.83K
 |  in pipelines: 01(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reservation=1
+Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reservation=1
 10:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=1.89K
+|  tuple-ids=6 row-size=44B cardinality=1.83K
 |  in pipelines: 01(GETNEXT)
 |
 09:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: i_item_id = i_item_id
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=1.89K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=1.83K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--F24:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3
@@ -1366,7 +1366,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--F25:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=6
@@ -1397,8 +1397,8 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser
 |     in pipelines: 04(GETNEXT)
 |
 35:EXCHANGE [HASH(ss_item_sk)]
-|  mem-estimate=683.13KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  mem-estimate=671.61KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1409,7 +1409,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_addr_sk = ca_address_sk
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1421,8 +1421,8 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=8B cardinality=8.07K
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1437,7 +1437,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
index c0123d8..080fca2 100755
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
@@ -93,26 +93,26 @@ PLAN-ROOT SINK
 |  output: sum(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=10.80K
+|  tuple-ids=26 row-size=44B cardinality=10.46K
 |  in pipelines: 31(GETNEXT), 10(OPEN), 20(OPEN), 30(OPEN)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=24 row-size=44B cardinality=10.80K
+|  tuple-ids=24 row-size=44B cardinality=10.46K
 |  in pipelines: 10(GETNEXT), 20(GETNEXT), 30(GETNEXT)
 |
 |--30:AGGREGATE [FINALIZE]
 |  |  output: sum(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=2.00K
+|  |  tuple-ids=22 row-size=44B cardinality=1.93K
 |  |  in pipelines: 30(GETNEXT), 24(OPEN)
 |  |
 |  29:HASH JOIN [LEFT SEMI JOIN]
 |  |  hash predicates: i_item_id = i_item_id
 |  |  runtime filters: RF016[bloom] <- i_item_id, RF017[min_max] <- i_item_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=2.00K
+|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=1.93K
 |  |  in pipelines: 24(GETNEXT), 25(OPEN)
 |  |
 |  |--25:SCAN HDFS [tpcds_parquet.item]
@@ -133,7 +133,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: i_item_sk = ws_item_sk
 |  |  runtime filters: RF018[bloom] <- ws_item_sk, RF019[min_max] <- ws_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=9.81K
+|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=9.51K
 |  |  in pipelines: 24(GETNEXT), 21(OPEN)
 |  |
 |  |--27:HASH JOIN [INNER JOIN]
@@ -141,7 +141,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  |  runtime filters: RF020[bloom] <- ca_address_sk, RF021[min_max] <- ca_address_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |  |
 |  |  |--23:SCAN HDFS [tpcds_parquet.customer_address]
@@ -154,7 +154,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |  |     in pipelines: 23(GETNEXT)
 |  |  |
 |  |  26:HASH JOIN [INNER JOIN]
@@ -204,14 +204,14 @@ PLAN-ROOT SINK
 |  |  output: sum(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=3.01K
+|  |  tuple-ids=14 row-size=44B cardinality=2.92K
 |  |  in pipelines: 20(GETNEXT), 14(OPEN)
 |  |
 |  19:HASH JOIN [LEFT SEMI JOIN]
 |  |  hash predicates: i_item_id = i_item_id
 |  |  runtime filters: RF008[bloom] <- i_item_id, RF009[min_max] <- i_item_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=3.01K
+|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=2.92K
 |  |  in pipelines: 14(GETNEXT), 15(OPEN)
 |  |
 |  |--15:SCAN HDFS [tpcds_parquet.item]
@@ -232,7 +232,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: i_item_sk = cs_item_sk
 |  |  runtime filters: RF011[min_max] <- cs_item_sk
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=14.81K
+|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=14.35K
 |  |  in pipelines: 14(GETNEXT), 11(OPEN)
 |  |
 |  |--17:HASH JOIN [INNER JOIN]
@@ -240,7 +240,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |  |
 |  |  |--13:SCAN HDFS [tpcds_parquet.customer_address]
@@ -253,7 +253,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |  |     in pipelines: 13(GETNEXT)
 |  |  |
 |  |  16:HASH JOIN [INNER JOIN]
@@ -303,14 +303,14 @@ PLAN-ROOT SINK
 |  output: sum(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=5.79K
+|  tuple-ids=6 row-size=44B cardinality=5.62K
 |  in pipelines: 10(GETNEXT), 01(OPEN)
 |
 09:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: i_item_id = i_item_id
 |  runtime filters: RF000[bloom] <- i_item_id, RF001[min_max] <- i_item_id
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=5.79K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=5.62K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--05:SCAN HDFS [tpcds_parquet.item]
@@ -331,7 +331,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF003[min_max] <- i_item_sk
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--04:SCAN HDFS [tpcds_parquet.item]
@@ -350,7 +350,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  runtime filters: RF004[bloom] <- ca_address_sk, RF005[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--03:SCAN HDFS [tpcds_parquet.customer_address]
@@ -363,7 +363,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN]
@@ -426,12 +426,12 @@ Per-Host Resources: mem-estimate=10.29MB mem-reservation=1.94MB thread-reservati
 |  output: sum:merge(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=10.80K
+|  tuple-ids=26 row-size=44B cardinality=10.46K
 |  in pipelines: 55(GETNEXT), 39(OPEN), 46(OPEN), 53(OPEN)
 |
 54:EXCHANGE [HASH(i_item_id)]
-|  mem-estimate=298.70KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=10.80K
+|  mem-estimate=293.89KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=26 row-size=44B cardinality=10.46K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -440,40 +440,40 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  output: sum(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=10.80K
+|  tuple-ids=26 row-size=44B cardinality=10.46K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=24 row-size=44B cardinality=10.80K
+|  tuple-ids=24 row-size=44B cardinality=10.46K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 |--53:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=2.00K
+|  |  tuple-ids=22 row-size=44B cardinality=1.93K
 |  |  in pipelines: 53(GETNEXT), 21(OPEN)
 |  |
 |  52:EXCHANGE [HASH(i_item_id)]
-|  |  mem-estimate=138.86KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=2.00K
+|  |  mem-estimate=137.53KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=22 row-size=44B cardinality=1.93K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F18:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=15.95MB mem-reservation=6.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=15.94MB mem-reservation=6.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  30:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=2.00K
+|  |  tuple-ids=22 row-size=44B cardinality=1.93K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  29:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash predicates: i_item_id = i_item_id
 |  |  runtime filters: RF016[bloom] <- i_item_id, RF017[min_max] <- i_item_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=2.00K
+|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=1.93K
 |  |  in pipelines: 21(GETNEXT), 25(OPEN)
 |  |
 |  |--51:EXCHANGE [BROADCAST]
@@ -501,7 +501,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  runtime filters: RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 24(OPEN)
 |  |
 |  |--50:EXCHANGE [HASH(i_item_sk)]
@@ -523,23 +523,23 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  |     in pipelines: 24(GETNEXT)
 |  |
 |  49:EXCHANGE [HASH(ws_item_sk)]
-|  |  mem-estimate=295.62KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  mem-estimate=289.66KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=133.97MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=133.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
 |  27:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ws_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  runtime filters: RF020[bloom] <- ca_address_sk, RF021[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |
 |  |--48:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=18 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 23(GETNEXT)
 |  |  |
 |  |  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -554,7 +554,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |     in pipelines: 23(GETNEXT)
 |  |
 |  26:HASH JOIN [INNER JOIN, BROADCAST]
@@ -600,28 +600,28 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  |  output: sum:merge(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=3.01K
+|  |  tuple-ids=14 row-size=44B cardinality=2.92K
 |  |  in pipelines: 46(GETNEXT), 11(OPEN)
 |  |
 |  45:EXCHANGE [HASH(i_item_id)]
-|  |  mem-estimate=187.13KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=3.01K
+|  |  mem-estimate=185.78KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=14 row-size=44B cardinality=2.92K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F11:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=6.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=15.99MB mem-reservation=6.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  20:AGGREGATE [STREAMING]
 |  |  output: sum(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=3.01K
+|  |  tuple-ids=14 row-size=44B cardinality=2.92K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  19:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash predicates: i_item_id = i_item_id
 |  |  runtime filters: RF008[bloom] <- i_item_id, RF009[min_max] <- i_item_id
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=3.01K
+|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=2.92K
 |  |  in pipelines: 11(GETNEXT), 15(OPEN)
 |  |
 |  |--44:EXCHANGE [BROADCAST]
@@ -649,7 +649,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  runtime filters: RF011[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 14(OPEN)
 |  |
 |  |--43:EXCHANGE [HASH(i_item_sk)]
@@ -671,23 +671,23 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  42:EXCHANGE [HASH(cs_item_sk)]
-|  |  mem-estimate=348.84KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  mem-estimate=342.84KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=197.97MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=197.96MB mem-reservation=13.88MB thread-reservation=2 runtime-filters-memory=2.00MB
 |  17:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: cs_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |
 |  |--41:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=10 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=10 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 13(GETNEXT)
 |  |  |
 |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -702,7 +702,7 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |     in pipelines: 13(GETNEXT)
 |  |
 |  16:HASH JOIN [INNER JOIN, BROADCAST]
@@ -748,28 +748,28 @@ Per-Host Resources: mem-estimate=20.22MB mem-reservation=3.94MB thread-reservati
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=5.79K
+|  tuple-ids=6 row-size=44B cardinality=5.62K
 |  in pipelines: 39(GETNEXT), 01(OPEN)
 |
 38:EXCHANGE [HASH(i_item_id)]
-|  mem-estimate=227.00KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=5.79K
+|  mem-estimate=224.42KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=44B cardinality=5.62K
 |  in pipelines: 01(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=17.17MB mem-reservation=7.88MB thread-reservation=1 runtime-filters-memory=2.00MB
+Per-Host Resources: mem-estimate=17.16MB mem-reservation=7.88MB thread-reservation=1 runtime-filters-memory=2.00MB
 10:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=5.79K
+|  tuple-ids=6 row-size=44B cardinality=5.62K
 |  in pipelines: 01(GETNEXT)
 |
 09:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  hash predicates: i_item_id = i_item_id
 |  runtime filters: RF000[bloom] <- i_item_id, RF001[min_max] <- i_item_id
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=5.79K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=5.62K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--37:EXCHANGE [BROADCAST]
@@ -797,7 +797,7 @@ Per-Host Resources: mem-estimate=17.17MB mem-reservation=7.88MB thread-reservati
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  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=0,1,2,3 row-size=76B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--36:EXCHANGE [HASH(i_item_sk)]
@@ -819,23 +819,23 @@ Per-Host Resources: mem-estimate=17.17MB mem-reservation=7.88MB thread-reservati
 |     in pipelines: 04(GETNEXT)
 |
 35:EXCHANGE [HASH(ss_item_sk)]
-|  mem-estimate=527.13KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  mem-estimate=515.61KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=54.97MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=3.00MB
+Per-Host Resources: mem-estimate=54.96MB mem-reservation=8.88MB thread-reservation=2 runtime-filters-memory=3.00MB
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_addr_sk = ca_address_sk
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  runtime filters: RF004[bloom] <- ca_address_sk, RF005[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=8B cardinality=8.07K
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -850,7 +850,7 @@ Per-Host Resources: mem-estimate=54.97MB mem-reservation=8.88MB thread-reservati
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
@@ -920,12 +920,12 @@ Per-Instance Resources: mem-estimate=10.43MB mem-reservation=1.94MB thread-reser
 |  output: sum:merge(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=10.80K
+|  tuple-ids=26 row-size=44B cardinality=10.46K
 |  in pipelines: 55(GETNEXT), 39(OPEN), 46(OPEN), 53(OPEN)
 |
 54:EXCHANGE [HASH(i_item_id)]
-|  mem-estimate=442.70KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=10.80K
+|  mem-estimate=437.89KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=26 row-size=44B cardinality=10.46K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -934,40 +934,40 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  output: sum(total_sales)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=26 row-size=44B cardinality=10.80K
+|  tuple-ids=26 row-size=44B cardinality=10.46K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=24 row-size=44B cardinality=10.80K
+|  tuple-ids=24 row-size=44B cardinality=10.46K
 |  in pipelines: 39(GETNEXT), 46(GETNEXT), 53(GETNEXT)
 |
 |--53:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=2.00K
+|  |  tuple-ids=22 row-size=44B cardinality=1.93K
 |  |  in pipelines: 53(GETNEXT), 21(OPEN)
 |  |
 |  52:EXCHANGE [HASH(i_item_id)]
-|  |  mem-estimate=138.86KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=2.00K
+|  |  mem-estimate=137.53KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=22 row-size=44B cardinality=1.93K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F18:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
-|  Per-Instance Resources: mem-estimate=10.29MB mem-reservation=2.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.28MB mem-reservation=2.00MB thread-reservation=1
 |  30:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=22 row-size=44B cardinality=2.00K
+|  |  tuple-ids=22 row-size=44B cardinality=1.93K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  29:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash-table-id=08
 |  |  hash predicates: i_item_id = i_item_id
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=2.00K
+|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=1.93K
 |  |  in pipelines: 21(GETNEXT), 25(OPEN)
 |  |
 |  |--F32:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
@@ -1003,7 +1003,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: ws_item_sk = i_item_sk
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=9.81K
+|  |  tuple-ids=16,17,18,19 row-size=76B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 24(OPEN)
 |  |
 |  |--F33:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=2 instances=2
@@ -1034,8 +1034,8 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |     in pipelines: 24(GETNEXT)
 |  |
 |  49:EXCHANGE [HASH(ws_item_sk)]
-|  |  mem-estimate=295.62KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  mem-estimate=289.66KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT)
 |  |
 |  F14:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1046,7 +1046,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: ws_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: ws_bill_addr_sk = ca_address_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.81K
+|  |  tuple-ids=16,17,18 row-size=40B cardinality=9.51K
 |  |  in pipelines: 21(GETNEXT), 23(OPEN)
 |  |
 |  |--F34:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1058,8 +1058,8 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  48:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=18 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 23(GETNEXT)
 |  |  |
 |  |  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1074,7 +1074,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=18 row-size=8B cardinality=8.33K
+|  |     tuple-ids=18 row-size=8B cardinality=8.07K
 |  |     in pipelines: 23(GETNEXT)
 |  |
 |  26:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1128,28 +1128,28 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |  output: sum:merge(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=3.01K
+|  |  tuple-ids=14 row-size=44B cardinality=2.92K
 |  |  in pipelines: 46(GETNEXT), 11(OPEN)
 |  |
 |  45:EXCHANGE [HASH(i_item_id)]
-|  |  mem-estimate=187.13KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=3.01K
+|  |  mem-estimate=185.78KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=14 row-size=44B cardinality=2.92K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F11:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
-|  Per-Instance Resources: mem-estimate=10.34MB mem-reservation=2.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.33MB mem-reservation=2.00MB thread-reservation=1
 |  20:AGGREGATE [STREAMING]
 |  |  output: sum(cs_ext_sales_price)
 |  |  group by: i_item_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=14 row-size=44B cardinality=3.01K
+|  |  tuple-ids=14 row-size=44B cardinality=2.92K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  19:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash-table-id=04
 |  |  hash predicates: i_item_id = i_item_id
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=3.01K
+|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=2.92K
 |  |  in pipelines: 11(GETNEXT), 15(OPEN)
 |  |
 |  |--F28:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
@@ -1185,7 +1185,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: cs_item_sk = i_item_sk
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=14.81K
+|  |  tuple-ids=8,9,10,11 row-size=76B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 14(OPEN)
 |  |
 |  |--F29:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=3
@@ -1216,8 +1216,8 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  42:EXCHANGE [HASH(cs_item_sk)]
-|  |  mem-estimate=348.84KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  mem-estimate=342.84KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1228,7 +1228,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |  hash predicates: cs_bill_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: cs_bill_addr_sk = ca_address_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.81K
+|  |  tuple-ids=8,9,10 row-size=40B cardinality=14.35K
 |  |  in pipelines: 11(GETNEXT), 13(OPEN)
 |  |
 |  |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1240,8 +1240,8 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  41:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=10 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=10 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 13(GETNEXT)
 |  |  |
 |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1256,7 +1256,7 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=10 row-size=8B cardinality=8.33K
+|  |     tuple-ids=10 row-size=8B cardinality=8.07K
 |  |     in pipelines: 13(GETNEXT)
 |  |
 |  16:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1310,28 +1310,28 @@ Per-Instance Resources: mem-estimate=20.36MB mem-reservation=3.94MB thread-reser
 |  output: sum:merge(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=5.79K
+|  tuple-ids=6 row-size=44B cardinality=5.62K
 |  in pipelines: 39(GETNEXT), 01(OPEN)
 |
 38:EXCHANGE [HASH(i_item_id)]
-|  mem-estimate=371.00KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=5.79K
+|  mem-estimate=368.42KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=44B cardinality=5.62K
 |  in pipelines: 01(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reservation=1
+Per-Instance Resources: mem-estimate=10.66MB mem-reservation=2.00MB thread-reservation=1
 10:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price)
 |  group by: i_item_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=6 row-size=44B cardinality=5.79K
+|  tuple-ids=6 row-size=44B cardinality=5.62K
 |  in pipelines: 01(GETNEXT)
 |
 09:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: i_item_id = i_item_id
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=5.79K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=5.62K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--F24:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3
@@ -1367,7 +1367,7 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3 row-size=76B cardinality=28.50K
+|  tuple-ids=0,1,2,3 row-size=76B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--F25:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=6
@@ -1398,8 +1398,8 @@ Per-Instance Resources: mem-estimate=10.67MB mem-reservation=2.00MB thread-reser
 |     in pipelines: 04(GETNEXT)
 |
 35:EXCHANGE [HASH(ss_item_sk)]
-|  mem-estimate=683.13KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  mem-estimate=671.61KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1410,7 +1410,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_addr_sk = ca_address_sk
 |  fk/pk conjuncts: ss_addr_sk = ca_address_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1,2 row-size=40B cardinality=28.50K
+|  tuple-ids=0,1,2 row-size=40B cardinality=27.62K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1422,8 +1422,8 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=8B cardinality=8.07K
 |  |  in pipelines: 03(GETNEXT)
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1438,7 +1438,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=2 row-size=8B cardinality=8.33K
+|     tuple-ids=2 row-size=8B cardinality=8.07K
 |     in pipelines: 03(GETNEXT)
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
index 421b285..c6a6595 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
@@ -77,7 +77,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.30K
+|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.19K
 |  |  in pipelines: 14(GETNEXT), 18(OPEN)
 |  |
 |  |--18:SCAN HDFS [tpcds_parquet.customer_address]
@@ -90,7 +90,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=13 row-size=8B cardinality=8.33K
+|  |     tuple-ids=13 row-size=8B cardinality=8.07K
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  23:HASH JOIN [INNER JOIN]
@@ -98,7 +98,7 @@ PLAN-ROOT SINK
 |  |  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=9,11,14,10,12 row-size=78B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 17(OPEN)
 |  |
 |  |--17:SCAN HDFS [tpcds_parquet.customer]
@@ -117,7 +117,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  |  runtime filters: RF017[min_max] <- s_store_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 15(OPEN)
 |  |
 |  |--15:SCAN HDFS [tpcds_parquet.store]
@@ -138,7 +138,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  runtime filters: RF018[bloom] <- i_item_sk, RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.06K
+|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 19(OPEN)
 |  |
 |  |--19:SCAN HDFS [tpcds_parquet.item]
@@ -198,7 +198,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  runtime filters: RF000[bloom] <- ca_address_sk, RF001[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.28K
+|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.17K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--05:SCAN HDFS [tpcds_parquet.customer_address]
@@ -211,7 +211,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=5 row-size=8B cardinality=8.33K
+|     tuple-ids=5 row-size=8B cardinality=8.07K
 |     in pipelines: 05(GETNEXT)
 |
 11:HASH JOIN [INNER JOIN]
@@ -219,7 +219,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF002[bloom] <- c_customer_sk, RF003[min_max] <- c_customer_sk
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--04:SCAN HDFS [tpcds_parquet.customer]
@@ -238,7 +238,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF004[bloom] <- s_store_sk, RF005[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds_parquet.store]
@@ -259,7 +259,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  runtime filters: RF006[bloom] <- p_promo_sk, RF007[min_max] <- p_promo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.94K
+|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.promotion]
@@ -278,7 +278,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF008[bloom] <- i_item_sk, RF009[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6 row-size=66B cardinality=17.06K
+|  tuple-ids=0,3,6 row-size=66B cardinality=17.02K
 |  in pipelines: 00(GETNEXT), 06(OPEN)
 |
 |--06:SCAN HDFS [tpcds_parquet.item]
@@ -377,12 +377,12 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.30K
+|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.19K
 |  |  in pipelines: 14(GETNEXT), 18(OPEN)
 |  |
 |  |--42:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=13 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=13 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 18(GETNEXT)
 |  |  |
 |  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -397,7 +397,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=13 row-size=8B cardinality=8.33K
+|  |     tuple-ids=13 row-size=8B cardinality=8.07K
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  23:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -405,7 +405,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  |  runtime filters: RF014[bloom] <- c_customer_sk, RF015[min_max] <- c_customer_sk
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 17(OPEN)
 |  |
 |  |--41:EXCHANGE [HASH(c_customer_sk)]
@@ -427,8 +427,8 @@ PLAN-ROOT SINK
 |  |     in pipelines: 17(GETNEXT)
 |  |
 |  40:EXCHANGE [HASH(ss_customer_sk)]
-|  |  mem-estimate=645.75KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  mem-estimate=644.88KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT)
 |  |
 |  F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -438,7 +438,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  |  runtime filters: RF017[min_max] <- s_store_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 15(OPEN)
 |  |
 |  |--39:EXCHANGE [BROADCAST]
@@ -466,11 +466,11 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  runtime filters: RF018[bloom] <- i_item_sk, RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.06K
+|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 19(OPEN)
 |  |
 |  |--38:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=75.41KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=75.31KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=14 row-size=26B cardinality=1.80K
 |  |  |  in pipelines: 19(GETNEXT)
 |  |  |
@@ -553,12 +553,12 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  runtime filters: RF000[bloom] <- ca_address_sk, RF001[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.28K
+|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.17K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5 row-size=8B cardinality=8.07K
 |  |  in pipelines: 05(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -573,7 +573,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=5 row-size=8B cardinality=8.33K
+|     tuple-ids=5 row-size=8B cardinality=8.07K
 |     in pipelines: 05(GETNEXT)
 |
 11:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -581,7 +581,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF002[bloom] <- c_customer_sk, RF003[min_max] <- c_customer_sk
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--33:EXCHANGE [HASH(c_customer_sk)]
@@ -604,7 +604,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio
 |
 32:EXCHANGE [HASH(ss_customer_sk)]
 |  mem-estimate=1.03MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -614,7 +614,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF004[bloom] <- s_store_sk, RF005[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--31:EXCHANGE [BROADCAST]
@@ -642,7 +642,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  runtime filters: RF006[bloom] <- p_promo_sk, RF007[min_max] <- p_promo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.94K
+|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--30:EXCHANGE [BROADCAST]
@@ -668,11 +668,11 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF008[bloom] <- i_item_sk, RF009[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6 row-size=66B cardinality=17.06K
+|  tuple-ids=0,3,6 row-size=66B cardinality=17.02K
 |  in pipelines: 00(GETNEXT), 06(OPEN)
 |
 |--29:EXCHANGE [BROADCAST]
-|  |  mem-estimate=75.41KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=75.31KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=6 row-size=26B cardinality=1.80K
 |  |  in pipelines: 06(GETNEXT)
 |  |
@@ -776,7 +776,7 @@ PLAN-ROOT SINK
 |  |  in pipelines: 25(GETNEXT)
 |  |
 |  F14:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=919.43KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=918.57KB mem-reservation=0B thread-reservation=1
 |  25:AGGREGATE
 |  |  output: sum(ss_ext_sales_price)
 |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -788,7 +788,7 @@ PLAN-ROOT SINK
 |  |  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=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.30K
+|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.19K
 |  |  in pipelines: 14(GETNEXT), 18(OPEN)
 |  |
 |  |--F18:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
@@ -800,8 +800,8 @@ PLAN-ROOT SINK
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  42:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=13 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=13 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 18(GETNEXT)
 |  |  |
 |  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -816,7 +816,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=13 row-size=8B cardinality=8.33K
+|  |     tuple-ids=13 row-size=8B cardinality=8.07K
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  23:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -824,7 +824,7 @@ PLAN-ROOT SINK
 |  |  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=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 17(OPEN)
 |  |
 |  |--F19:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
@@ -855,8 +855,8 @@ PLAN-ROOT SINK
 |  |     in pipelines: 17(GETNEXT)
 |  |
 |  40:EXCHANGE [HASH(ss_customer_sk)]
-|  |  mem-estimate=903.43KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  mem-estimate=902.57KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT)
 |  |
 |  F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -867,7 +867,7 @@ PLAN-ROOT SINK
 |  |  hash predicates: ss_store_sk = s_store_sk
 |  |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 15(OPEN)
 |  |
 |  |--F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -903,7 +903,7 @@ PLAN-ROOT SINK
 |  |  hash predicates: ss_item_sk = i_item_sk
 |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.06K
+|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 19(OPEN)
 |  |
 |  |--F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -915,7 +915,7 @@ PLAN-ROOT SINK
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  38:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=75.41KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=75.31KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=14 row-size=26B cardinality=1.80K
 |  |  |  in pipelines: 19(GETNEXT)
 |  |  |
@@ -1006,7 +1006,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |  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=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.28K
+|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.17K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--F23:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
@@ -1018,8 +1018,8 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5 row-size=8B cardinality=8.07K
 |  |  in pipelines: 05(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1034,7 +1034,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=5 row-size=8B cardinality=8.33K
+|     tuple-ids=5 row-size=8B cardinality=8.07K
 |     in pipelines: 05(GETNEXT)
 |
 11:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1042,7 +1042,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |  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=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--F24:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
@@ -1074,7 +1074,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |
 32:EXCHANGE [HASH(ss_customer_sk)]
 |  mem-estimate=1.43MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1085,7 +1085,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_store_sk = s_store_sk
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1121,7 +1121,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_promo_sk = p_promo_sk
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.94K
+|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1155,7 +1155,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6 row-size=66B cardinality=17.06K
+|  tuple-ids=0,3,6 row-size=66B cardinality=17.02K
 |  in pipelines: 00(GETNEXT), 06(OPEN)
 |
 |--F27:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1167,7 +1167,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  29:EXCHANGE [BROADCAST]
-|  |  mem-estimate=75.41KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=75.31KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=6 row-size=26B cardinality=1.80K
 |  |  in pipelines: 06(GETNEXT)
 |  |
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
index dbe28a1..176d207 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
@@ -70,7 +70,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.30K
+|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.19K
 |  |  in pipelines: 14(GETNEXT), 18(OPEN)
 |  |
 |  |--18:SCAN HDFS [tpcds_parquet.customer_address]
@@ -83,7 +83,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=13 row-size=8B cardinality=8.33K
+|  |     tuple-ids=13 row-size=8B cardinality=8.07K
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  23:HASH JOIN [INNER JOIN]
@@ -91,7 +91,7 @@ PLAN-ROOT SINK
 |  |  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=9,11,14,10,12 row-size=78B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 17(OPEN)
 |  |
 |  |--17:SCAN HDFS [tpcds_parquet.customer]
@@ -110,7 +110,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  |  runtime filters: RF017[min_max] <- s_store_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 15(OPEN)
 |  |
 |  |--15:SCAN HDFS [tpcds_parquet.store]
@@ -131,7 +131,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  runtime filters: RF018[bloom] <- i_item_sk, RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.06K
+|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 19(OPEN)
 |  |
 |  |--19:SCAN HDFS [tpcds_parquet.item]
@@ -191,7 +191,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  runtime filters: RF000[bloom] <- ca_address_sk, RF001[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.28K
+|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.17K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--05:SCAN HDFS [tpcds_parquet.customer_address]
@@ -204,7 +204,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=5 row-size=8B cardinality=8.33K
+|     tuple-ids=5 row-size=8B cardinality=8.07K
 |     in pipelines: 05(GETNEXT)
 |
 11:HASH JOIN [INNER JOIN]
@@ -212,7 +212,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF002[bloom] <- c_customer_sk, RF003[min_max] <- c_customer_sk
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--04:SCAN HDFS [tpcds_parquet.customer]
@@ -231,7 +231,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF004[bloom] <- s_store_sk, RF005[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds_parquet.store]
@@ -252,7 +252,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  runtime filters: RF006[bloom] <- p_promo_sk, RF007[min_max] <- p_promo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.94K
+|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.promotion]
@@ -271,7 +271,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF008[bloom] <- i_item_sk, RF009[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6 row-size=66B cardinality=17.06K
+|  tuple-ids=0,3,6 row-size=66B cardinality=17.02K
 |  in pipelines: 00(GETNEXT), 06(OPEN)
 |
 |--06:SCAN HDFS [tpcds_parquet.item]
@@ -370,12 +370,12 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  |  runtime filters: RF012[bloom] <- ca_address_sk, RF013[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.30K
+|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.19K
 |  |  in pipelines: 14(GETNEXT), 18(OPEN)
 |  |
 |  |--42:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=13 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=13 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 18(GETNEXT)
 |  |  |
 |  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -390,7 +390,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=13 row-size=8B cardinality=8.33K
+|  |     tuple-ids=13 row-size=8B cardinality=8.07K
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  23:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -398,7 +398,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  |  runtime filters: RF014[bloom] <- c_customer_sk, RF015[min_max] <- c_customer_sk
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 17(OPEN)
 |  |
 |  |--41:EXCHANGE [HASH(c_customer_sk)]
@@ -420,8 +420,8 @@ PLAN-ROOT SINK
 |  |     in pipelines: 17(GETNEXT)
 |  |
 |  40:EXCHANGE [HASH(ss_customer_sk)]
-|  |  mem-estimate=645.75KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  mem-estimate=644.88KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT)
 |  |
 |  F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -431,7 +431,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  |  runtime filters: RF017[min_max] <- s_store_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 15(OPEN)
 |  |
 |  |--39:EXCHANGE [BROADCAST]
@@ -459,11 +459,11 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  runtime filters: RF018[bloom] <- i_item_sk, RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.06K
+|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 19(OPEN)
 |  |
 |  |--38:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=75.41KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=75.31KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=14 row-size=26B cardinality=1.80K
 |  |  |  in pipelines: 19(GETNEXT)
 |  |  |
@@ -546,12 +546,12 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio
 |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  runtime filters: RF000[bloom] <- ca_address_sk, RF001[min_max] <- ca_address_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.28K
+|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.17K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5 row-size=8B cardinality=8.07K
 |  |  in pipelines: 05(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -566,7 +566,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=5 row-size=8B cardinality=8.33K
+|     tuple-ids=5 row-size=8B cardinality=8.07K
 |     in pipelines: 05(GETNEXT)
 |
 11:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -574,7 +574,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio
 |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
 |  runtime filters: RF002[bloom] <- c_customer_sk, RF003[min_max] <- c_customer_sk
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--33:EXCHANGE [HASH(c_customer_sk)]
@@ -597,7 +597,7 @@ Per-Host Resources: mem-estimate=8.69MB mem-reservation=6.81MB thread-reservatio
 |
 32:EXCHANGE [HASH(ss_customer_sk)]
 |  mem-estimate=1.03MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -607,7 +607,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF004[bloom] <- s_store_sk, RF005[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--31:EXCHANGE [BROADCAST]
@@ -635,7 +635,7 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  runtime filters: RF006[bloom] <- p_promo_sk, RF007[min_max] <- p_promo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.94K
+|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--30:EXCHANGE [BROADCAST]
@@ -661,11 +661,11 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  runtime filters: RF008[bloom] <- i_item_sk, RF009[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6 row-size=66B cardinality=17.06K
+|  tuple-ids=0,3,6 row-size=66B cardinality=17.02K
 |  in pipelines: 00(GETNEXT), 06(OPEN)
 |
 |--29:EXCHANGE [BROADCAST]
-|  |  mem-estimate=75.41KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=75.31KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=6 row-size=26B cardinality=1.80K
 |  |  in pipelines: 06(GETNEXT)
 |  |
@@ -769,7 +769,7 @@ PLAN-ROOT SINK
 |  |  in pipelines: 25(GETNEXT)
 |  |
 |  F14:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=919.43KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=918.57KB mem-reservation=0B thread-reservation=1
 |  25:AGGREGATE
 |  |  output: sum(ss_ext_sales_price)
 |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -781,7 +781,7 @@ PLAN-ROOT SINK
 |  |  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=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.30K
+|  |  tuple-ids=9,11,14,10,12,13 row-size=86B cardinality=3.19K
 |  |  in pipelines: 14(GETNEXT), 18(OPEN)
 |  |
 |  |--F18:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
@@ -793,8 +793,8 @@ PLAN-ROOT SINK
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  42:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=13 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=13 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 18(GETNEXT)
 |  |  |
 |  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -809,7 +809,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=13 row-size=8B cardinality=8.33K
+|  |     tuple-ids=13 row-size=8B cardinality=8.07K
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  23:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -817,7 +817,7 @@ PLAN-ROOT SINK
 |  |  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=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10,12 row-size=78B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 17(OPEN)
 |  |
 |  |--F19:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
@@ -848,8 +848,8 @@ PLAN-ROOT SINK
 |  |     in pipelines: 17(GETNEXT)
 |  |
 |  40:EXCHANGE [HASH(ss_customer_sk)]
-|  |  mem-estimate=903.43KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  mem-estimate=902.57KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT)
 |  |
 |  F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -860,7 +860,7 @@ PLAN-ROOT SINK
 |  |  hash predicates: ss_store_sk = s_store_sk
 |  |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.06K
+|  |  tuple-ids=9,11,14,10 row-size=70B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 15(OPEN)
 |  |
 |  |--F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -896,7 +896,7 @@ PLAN-ROOT SINK
 |  |  hash predicates: ss_item_sk = i_item_sk
 |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.06K
+|  |  tuple-ids=9,11,14 row-size=62B cardinality=17.02K
 |  |  in pipelines: 14(GETNEXT), 19(OPEN)
 |  |
 |  |--F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -908,7 +908,7 @@ PLAN-ROOT SINK
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  38:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=75.41KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=75.31KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=14 row-size=26B cardinality=1.80K
 |  |  |  in pipelines: 19(GETNEXT)
 |  |  |
@@ -999,7 +999,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |  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=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.28K
+|  tuple-ids=0,3,6,2,1,4,5 row-size=133B cardinality=3.17K
 |  in pipelines: 00(GETNEXT), 05(OPEN)
 |
 |--F23:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=3
@@ -1011,8 +1011,8 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  34:EXCHANGE [BROADCAST]
-|  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=8B cardinality=8.33K
+|  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5 row-size=8B cardinality=8.07K
 |  |  in pipelines: 05(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1027,7 +1027,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |     parquet statistics predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     parquet dictionary predicates: ca_gmt_offset = CAST(-5 AS DECIMAL(3,0))
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=5 row-size=8B cardinality=8.33K
+|     tuple-ids=5 row-size=8B cardinality=8.07K
 |     in pipelines: 05(GETNEXT)
 |
 11:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1035,7 +1035,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |  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=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1,4 row-size=125B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 04(OPEN)
 |
 |--F24:PLAN FRAGMENT [HASH(ss_customer_sk)] hosts=3 instances=6
@@ -1067,7 +1067,7 @@ Per-Instance Resources: mem-estimate=1.45MB mem-reservation=0B thread-reservatio
 |
 32:EXCHANGE [HASH(ss_customer_sk)]
 |  mem-estimate=1.43MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1078,7 +1078,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_store_sk = s_store_sk
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.94K
+|  tuple-ids=0,3,6,2,1 row-size=117B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1114,7 +1114,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_promo_sk = p_promo_sk
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.94K
+|  tuple-ids=0,3,6,2 row-size=109B cardinality=16.91K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1148,7 +1148,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: ss_item_sk = i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3,6 row-size=66B cardinality=17.06K
+|  tuple-ids=0,3,6 row-size=66B cardinality=17.02K
 |  in pipelines: 00(GETNEXT), 06(OPEN)
 |
 |--F27:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1160,7 +1160,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  29:EXCHANGE [BROADCAST]
-|  |  mem-estimate=75.41KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=75.31KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=6 row-size=26B cardinality=1.80K
 |  |  in pipelines: 06(GETNEXT)
 |  |
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 f7f7371..cbad5b3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
@@ -57,14 +57,14 @@ PLAN-ROOT SINK
 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=1.01K
+|  tuple-ids=11 row-size=56B cardinality=1.00K
 |  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=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=1.01K
+|  tuple-ids=10 row-size=56B cardinality=1.00K
 |  in pipelines: 14(GETNEXT), 11(OPEN)
 |
 13:HASH JOIN [INNER JOIN]
@@ -72,7 +72,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: assumed fk/pk
 |  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=9,7,0 row-size=91B cardinality=1.01K
+|  tuple-ids=9,7,0 row-size=91B cardinality=1.00K
 |  in pipelines: 11(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
 |
 |--12:HASH JOIN [INNER JOIN]
@@ -80,7 +80,7 @@ PLAN-ROOT SINK
 |  |  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=1.01K
+|  |  tuple-ids=7,0 row-size=60B cardinality=1.00K
 |  |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 00(OPEN)
 |  |
 |  |--00:SCAN HDFS [tpcds_parquet.item]
@@ -93,7 +93,7 @@ PLAN-ROOT SINK
 |  |     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
+|  |     tuple-ids=0 row-size=44B cardinality=181
 |  |     in pipelines: 00(GETNEXT)
 |  |
 |  01:UNION
@@ -222,8 +222,8 @@ PLAN-ROOT SINK
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ext_price) DESC, i_brand_id ASC
-|  mem-estimate=196.21KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=11 row-size=56B cardinality=1.01K
+|  mem-estimate=195.24KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=11 row-size=56B cardinality=1.00K
 |  in pipelines: 15(GETNEXT)
 |
 F10:PLAN FRAGMENT [HASH(i_brand,i_brand_id,t_hour,t_minute)] hosts=3 instances=3
@@ -231,19 +231,19 @@ Per-Host Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-reservat
 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=1.01K
+|  tuple-ids=11 row-size=56B cardinality=1.00K
 |  in pipelines: 15(GETNEXT), 23(OPEN)
 |
 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=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=1.01K
+|  tuple-ids=10 row-size=56B cardinality=1.00K
 |  in pipelines: 23(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
 |
 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
+|  mem-estimate=195.24KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
 F09:PLAN FRAGMENT [HASH(time_sk)] hosts=3 instances=3
@@ -252,7 +252,7 @@ Per-Host Resources: mem-estimate=15.82MB mem-reservation=5.88MB thread-reservati
 |  output: sum(ext_price)
 |  group by: i_brand, i_brand_id, t_hour, t_minute
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=1.01K
+|  tuple-ids=10 row-size=56B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
 13:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -260,7 +260,7 @@ Per-Host Resources: mem-estimate=15.82MB mem-reservation=5.88MB thread-reservati
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF000[bloom] <- t_time_sk, RF001[min_max] <- t_time_sk
 |  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
+|  tuple-ids=7,0,9 row-size=91B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 11(OPEN)
 |
 |--21:EXCHANGE [HASH(t_time_sk)]
@@ -284,8 +284,8 @@ Per-Host Resources: mem-estimate=15.82MB mem-reservation=5.88MB thread-reservati
 |     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
+|  mem-estimate=220.05KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=7,0 row-size=60B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
 F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -295,12 +295,12 @@ Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reserva
 |  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=1.01K
+|  tuple-ids=7,0 row-size=60B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 00(OPEN)
 |
 |--19:EXCHANGE [BROADCAST]
-|  |  mem-estimate=16.42KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0 row-size=44B cardinality=182
+|  |  mem-estimate=16.33KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0 row-size=44B cardinality=181
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -315,7 +315,7 @@ Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reserva
 |     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
+|     tuple-ids=0 row-size=44B cardinality=181
 |     in pipelines: 00(GETNEXT)
 |
 01:UNION
@@ -444,15 +444,15 @@ Per-Host Resources: mem-estimate=200.91MB mem-reservation=16.88MB thread-reserva
 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
+|  Per-Instance Resources: mem-estimate=4.36MB 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=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ext_price) DESC, i_brand_id ASC
-|  mem-estimate=373.98KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=11 row-size=56B cardinality=1.01K
+|  mem-estimate=372.12KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=11 row-size=56B cardinality=1.00K
 |  in pipelines: 15(GETNEXT)
 |
 F10:PLAN FRAGMENT [HASH(i_brand,i_brand_id,t_hour,t_minute)] hosts=3 instances=6
@@ -460,19 +460,19 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese
 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=1.01K
+|  tuple-ids=11 row-size=56B cardinality=1.00K
 |  in pipelines: 15(GETNEXT), 23(OPEN)
 |
 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=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=1.01K
+|  tuple-ids=10 row-size=56B cardinality=1.00K
 |  in pipelines: 23(GETNEXT), 02(OPEN), 05(OPEN), 08(OPEN)
 |
 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
+|  mem-estimate=372.12KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=10 row-size=56B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
 F09:PLAN FRAGMENT [HASH(time_sk)] hosts=3 instances=6
@@ -481,7 +481,7 @@ Per-Instance Resources: mem-estimate=10.41MB mem-reservation=2.00MB thread-reser
 |  output: sum(ext_price)
 |  group by: i_brand, i_brand_id, t_hour, t_minute
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=10 row-size=56B cardinality=1.01K
+|  tuple-ids=10 row-size=56B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
 13:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -489,7 +489,7 @@ Per-Instance Resources: mem-estimate=10.41MB mem-reservation=2.00MB thread-reser
 |  hash predicates: time_sk = t_time_sk
 |  fk/pk conjuncts: assumed fk/pk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7,0,9 row-size=91B cardinality=1.01K
+|  tuple-ids=7,0,9 row-size=91B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 11(OPEN)
 |
 |--F12:PLAN FRAGMENT [HASH(time_sk)] hosts=3 instances=6
@@ -521,8 +521,8 @@ Per-Instance Resources: mem-estimate=10.41MB mem-reservation=2.00MB thread-reser
 |     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
+|  mem-estimate=420.45KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=7,0 row-size=60B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
 F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -533,7 +533,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser
 |  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=1.01K
+|  tuple-ids=7,0 row-size=60B cardinality=1.00K
 |  in pipelines: 02(GETNEXT), 05(GETNEXT), 08(GETNEXT), 00(OPEN)
 |
 |--F13:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -545,8 +545,8 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  19:EXCHANGE [BROADCAST]
-|  |  mem-estimate=16.42KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0 row-size=44B cardinality=182
+|  |  mem-estimate=16.33KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0 row-size=44B cardinality=181
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -561,7 +561,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser
 |     parquet statistics predicates: i_manager_id = CAST(1 AS INT)
 |     parquet dictionary predicates: i_manager_id = CAST(1 AS INT)
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=0 row-size=44B cardinality=182
+|     tuple-ids=0 row-size=44B cardinality=181
 |     in pipelines: 00(GETNEXT)
 |
 01:UNION
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
index 3d5b599..faf29c5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
@@ -110,12 +110,12 @@ PLAN-ROOT SINK
 |  46:AGGREGATE [FINALIZE]
 |  |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 46(GETNEXT), 28(OPEN), 35(OPEN), 42(OPEN)
 |  |
 |  24:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 28(GETNEXT), 35(GETNEXT), 42(GETNEXT)
 |  |
 |  |--45:HASH JOIN [RIGHT OUTER JOIN]
@@ -123,7 +123,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: wr_item_sk = ws_item_sk, wr_order_number = ws_order_number
 |  |  |  runtime filters: RF050[min_max] <- ws_item_sk, RF051[min_max] <- ws_order_number
 |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  |  tuple-ids=27N,24,25,26 row-size=102B cardinality=14.80K
+|  |  |  tuple-ids=27N,24,25,26 row-size=102B cardinality=14.77K
 |  |  |  in pipelines: 42(GETNEXT), 39(OPEN)
 |  |  |
 |  |  |--44:HASH JOIN [INNER JOIN]
@@ -131,7 +131,7 @@ PLAN-ROOT SINK
 |  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  |  |  runtime filters: RF053[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=78B cardinality=14.80K
+|  |  |  |  tuple-ids=24,25,26 row-size=78B cardinality=14.77K
 |  |  |  |  in pipelines: 39(GETNEXT), 41(OPEN)
 |  |  |  |
 |  |  |  |--41:SCAN HDFS [tpcds_parquet.date_dim]
@@ -152,7 +152,7 @@ PLAN-ROOT SINK
 |  |  |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  |  |  runtime filters: RF054[bloom] <- i_item_sk, RF055[min_max] <- i_item_sk
 |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  |  tuple-ids=24,25 row-size=70B cardinality=71.94K
+|  |  |  |  tuple-ids=24,25 row-size=70B cardinality=71.78K
 |  |  |  |  in pipelines: 39(GETNEXT), 40(OPEN)
 |  |  |  |
 |  |  |  |--40:SCAN HDFS [tpcds_parquet.item]
@@ -195,15 +195,14 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: sr_item_sk = ss_item_sk, sr_ticket_number = ss_ticket_number
 |  |  |  runtime filters: RF042[min_max] <- ss_item_sk, RF043[min_max] <- ss_ticket_number
 |  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  |  tuple-ids=23N,20,21,22 row-size=102B cardinality=58.90K
+|  |  |  tuple-ids=23N,20,21,22 row-size=102B cardinality=58.77K
 |  |  |  in pipelines: 35(GETNEXT), 32(OPEN)
 |  |  |
 |  |  |--37: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: RF044[bloom] <- d_date_sk
 |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  |  tuple-ids=20,21,22 row-size=78B cardinality=58.90K
+|  |  |  |  tuple-ids=20,21,22 row-size=78B cardinality=58.77K
 |  |  |  |  in pipelines: 32(GETNEXT), 34(OPEN)
 |  |  |  |
 |  |  |  |--34:SCAN HDFS [tpcds_parquet.date_dim]
@@ -224,7 +223,7 @@ PLAN-ROOT SINK
 |  |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  |  |  runtime filters: RF046[bloom] <- i_item_sk, RF047[min_max] <- i_item_sk
 |  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  |  tuple-ids=20,21 row-size=70B cardinality=288.04K
+|  |  |  |  tuple-ids=20,21 row-size=70B cardinality=287.40K
 |  |  |  |  in pipelines: 32(GETNEXT), 33(OPEN)
 |  |  |  |
 |  |  |  |--33:SCAN HDFS [tpcds_parquet.item]
@@ -242,7 +241,7 @@ PLAN-ROOT SINK
 |  |  |  |
 |  |  |  32:SCAN HDFS [tpcds_parquet.store_sales]
 |  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|  |  |     runtime filters: RF047[min_max] -> ss_item_sk, RF046[bloom] -> ss_item_sk, RF044[bloom] -> ss_sold_date_sk
+|  |  |     runtime filters: RF047[min_max] -> ss_item_sk, RF046[bloom] -> ss_item_sk
 |  |  |     stored statistics:
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
@@ -266,17 +265,17 @@ PLAN-ROOT SINK
 |  31:HASH JOIN [RIGHT OUTER JOIN]
 |  |  hash predicates: cr_item_sk = cs_item_sk, cr_order_number = cs_order_number
 |  |  fk/pk conjuncts: cr_item_sk = cs_item_sk, cr_order_number = cs_order_number
-|  |  runtime filters: RF034[min_max] <- cs_item_sk, RF035[min_max] <- cs_order_number
+|  |  runtime filters: RF032[bloom] <- cs_item_sk, RF033[bloom] <- cs_order_number, RF034[min_max] <- cs_item_sk, RF035[min_max] <- cs_order_number
 |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  |  tuple-ids=19N,16,17,18 row-size=102B cardinality=29.46K
+|  |  tuple-ids=19N,16,17,18 row-size=102B cardinality=29.40K
 |  |  in pipelines: 28(GETNEXT), 25(OPEN)
 |  |
 |  |--30: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: RF036[bloom] <- d_date_sk, RF037[min_max] <- d_date_sk
+|  |  |  runtime filters: RF037[min_max] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=16,17,18 row-size=78B cardinality=29.46K
+|  |  |  tuple-ids=16,17,18 row-size=78B cardinality=29.40K
 |  |  |  in pipelines: 25(GETNEXT), 27(OPEN)
 |  |  |
 |  |  |--27:SCAN HDFS [tpcds_parquet.date_dim]
@@ -297,7 +296,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  |  runtime filters: RF038[bloom] <- i_item_sk, RF039[min_max] <- i_item_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=16,17 row-size=70B cardinality=144.16K
+|  |  |  tuple-ids=16,17 row-size=70B cardinality=143.84K
 |  |  |  in pipelines: 25(GETNEXT), 26(OPEN)
 |  |  |
 |  |  |--26:SCAN HDFS [tpcds_parquet.item]
@@ -315,7 +314,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  25:SCAN HDFS [tpcds_parquet.catalog_sales]
 |  |     HDFS partitions=1/1 files=3 size=96.62MB
-|  |     runtime filters: RF039[min_max] -> cs_item_sk, RF037[min_max] -> cs_sold_date_sk, RF038[bloom] -> cs_item_sk, RF036[bloom] -> cs_sold_date_sk
+|  |     runtime filters: RF039[min_max] -> cs_item_sk, RF037[min_max] -> cs_sold_date_sk, RF038[bloom] -> cs_item_sk
 |  |     stored statistics:
 |  |       table: rows=1.44M size=96.62MB
 |  |       columns: all
@@ -326,7 +325,7 @@ PLAN-ROOT SINK
 |  |
 |  28:SCAN HDFS [tpcds_parquet.catalog_returns]
 |     HDFS partitions=1/1 files=1 size=10.62MB
-|     runtime filters: RF034[min_max] -> cr_item_sk, RF035[min_max] -> cr_order_number
+|     runtime filters: RF034[min_max] -> cr_item_sk, RF035[min_max] -> cr_order_number, RF032[bloom] -> cr_item_sk, RF033[bloom] -> cr_order_number
 |     stored statistics:
 |       table: rows=144.07K size=10.62MB
 |       columns: all
@@ -346,12 +345,12 @@ PLAN-ROOT SINK
 22:AGGREGATE [FINALIZE]
 |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 22(GETNEXT), 04(OPEN), 11(OPEN), 18(OPEN)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 04(GETNEXT), 11(GETNEXT), 18(GETNEXT)
 |
 |--21:HASH JOIN [RIGHT OUTER JOIN]
@@ -359,7 +358,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: wr_item_sk = ws_item_sk, wr_order_number = ws_order_number
 |  |  runtime filters: RF026[min_max] <- ws_item_sk, RF027[min_max] <- ws_order_number
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=11N,8,9,10 row-size=102B cardinality=14.80K
+|  |  tuple-ids=11N,8,9,10 row-size=102B cardinality=14.77K
 |  |  in pipelines: 18(GETNEXT), 15(OPEN)
 |  |
 |  |--20:HASH JOIN [INNER JOIN]
@@ -367,7 +366,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  |  runtime filters: RF029[min_max] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=8,9,10 row-size=78B cardinality=14.80K
+|  |  |  tuple-ids=8,9,10 row-size=78B cardinality=14.77K
 |  |  |  in pipelines: 15(GETNEXT), 17(OPEN)
 |  |  |
 |  |  |--17:SCAN HDFS [tpcds_parquet.date_dim]
@@ -388,7 +387,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  |  runtime filters: RF030[bloom] <- i_item_sk, RF031[min_max] <- i_item_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=8,9 row-size=70B cardinality=71.94K
+|  |  |  tuple-ids=8,9 row-size=70B cardinality=71.78K
 |  |  |  in pipelines: 15(GETNEXT), 16(OPEN)
 |  |  |
 |  |  |--16:SCAN HDFS [tpcds_parquet.item]
@@ -432,15 +431,14 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: sr_item_sk = ss_item_sk, sr_ticket_number = ss_ticket_number
 |  |  runtime filters: RF018[min_max] <- ss_item_sk, RF019[min_max] <- ss_ticket_number
 |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=7N,4,5,6 row-size=102B cardinality=58.90K
+|  |  tuple-ids=7N,4,5,6 row-size=102B cardinality=58.77K
 |  |  in pipelines: 11(GETNEXT), 08(OPEN)
 |  |
 |  |--13: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: RF020[bloom] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=4,5,6 row-size=78B cardinality=58.90K
+|  |  |  tuple-ids=4,5,6 row-size=78B cardinality=58.77K
 |  |  |  in pipelines: 08(GETNEXT), 10(OPEN)
 |  |  |
 |  |  |--10:SCAN HDFS [tpcds_parquet.date_dim]
@@ -461,7 +459,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  |  runtime filters: RF022[bloom] <- i_item_sk, RF023[min_max] <- i_item_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=4,5 row-size=70B cardinality=288.04K
+|  |  |  tuple-ids=4,5 row-size=70B cardinality=287.40K
 |  |  |  in pipelines: 08(GETNEXT), 09(OPEN)
 |  |  |
 |  |  |--09:SCAN HDFS [tpcds_parquet.item]
@@ -480,7 +478,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  08:SCAN HDFS [tpcds_parquet.store_sales]
 |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
-|  |     runtime filters: RF023[min_max] -> ss_item_sk, RF022[bloom] -> ss_item_sk, RF020[bloom] -> ss_sold_date_sk
+|  |     runtime filters: RF023[min_max] -> ss_item_sk, RF022[bloom] -> ss_item_sk
 |  |     stored statistics:
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
@@ -504,17 +502,17 @@ PLAN-ROOT SINK
 07:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: cr_item_sk = cs_item_sk, cr_order_number = cs_order_number
 |  fk/pk conjuncts: cr_item_sk = cs_item_sk, cr_order_number = cs_order_number
-|  runtime filters: RF010[min_max] <- cs_item_sk, RF011[min_max] <- cs_order_number
+|  runtime filters: RF008[bloom] <- cs_item_sk, RF009[bloom] <- cs_order_number, RF010[min_max] <- cs_item_sk, RF011[min_max] <- cs_order_number
 |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=3N,0,1,2 row-size=102B cardinality=29.46K
+|  tuple-ids=3N,0,1,2 row-size=102B cardinality=29.40K
 |  in pipelines: 04(GETNEXT), 01(OPEN)
 |
 |--06: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: RF012[bloom] <- d_date_sk, RF013[min_max] <- d_date_sk
+|  |  runtime filters: RF013[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,1,2 row-size=78B cardinality=29.46K
+|  |  tuple-ids=0,1,2 row-size=78B cardinality=29.40K
 |  |  in pipelines: 01(GETNEXT), 03(OPEN)
 |  |
 |  |--03:SCAN HDFS [tpcds_parquet.date_dim]
@@ -535,7 +533,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  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=0,1 row-size=70B cardinality=144.16K
+|  |  tuple-ids=0,1 row-size=70B cardinality=143.84K
 |  |  in pipelines: 01(GETNEXT), 02(OPEN)
 |  |
 |  |--02:SCAN HDFS [tpcds_parquet.item]
@@ -554,7 +552,7 @@ PLAN-ROOT SINK
 |  |
 |  01:SCAN HDFS [tpcds_parquet.catalog_sales]
 |     HDFS partitions=1/1 files=3 size=96.62MB
-|     runtime filters: RF015[min_max] -> cs_item_sk, RF013[min_max] -> cs_sold_date_sk, RF014[bloom] -> cs_item_sk, RF012[bloom] -> cs_sold_date_sk
+|     runtime filters: RF015[min_max] -> cs_item_sk, RF013[min_max] -> cs_sold_date_sk, RF014[bloom] -> cs_item_sk
 |     stored statistics:
 |       table: rows=1.44M size=96.62MB
 |       columns: all
@@ -565,7 +563,7 @@ PLAN-ROOT SINK
 |
 04:SCAN HDFS [tpcds_parquet.catalog_returns]
    HDFS partitions=1/1 files=1 size=10.62MB
-   runtime filters: RF010[min_max] -> cr_item_sk, RF011[min_max] -> cr_order_number
+   runtime filters: RF010[min_max] -> cr_item_sk, RF011[min_max] -> cr_order_number, RF008[bloom] -> cr_item_sk, RF009[bloom] -> cr_order_number
    stored statistics:
      table: rows=144.07K size=10.62MB
      columns: all
@@ -623,7 +621,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |
 |  80:EXCHANGE [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id)]
 |  |  mem-estimate=1.58MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=30 row-size=44B cardinality=103.17K
+|  |  tuple-ids=30 row-size=44B cardinality=102.94K
 |  |  in pipelines: 79(GETNEXT)
 |  |
 |  F34:PLAN FRAGMENT [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt)] hosts=3 instances=3
@@ -632,18 +630,18 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  output: sum(sales_cnt), sum(sales_amt)
 |  |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id
 |  |  mem-estimate=10.00MB mem-reservation=3.00MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=30 row-size=44B cardinality=103.17K
+|  |  tuple-ids=30 row-size=44B cardinality=102.94K
 |  |  in pipelines: 79(GETNEXT)
 |  |
 |  79:AGGREGATE [FINALIZE]
 |  |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 79(GETNEXT), 25(OPEN), 32(OPEN), 39(OPEN)
 |  |
 |  78:EXCHANGE [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt)]
 |  |  mem-estimate=1.15MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 25(GETNEXT), 32(GETNEXT), 39(GETNEXT)
 |  |
 |  F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -651,19 +649,19 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  46:AGGREGATE [STREAMING]
 |  |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  |  mem-estimate=10.00MB mem-reservation=3.00MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 25(GETNEXT), 32(GETNEXT), 39(GETNEXT)
 |  |
 |  24:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 25(GETNEXT), 32(GETNEXT), 39(GETNEXT)
 |  |
 |  |--45:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
 |  |  |  hash predicates: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  |  fk/pk conjuncts: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  |  tuple-ids=24,25,26,27N row-size=102B cardinality=14.80K
+|  |  |  tuple-ids=24,25,26,27N row-size=102B cardinality=14.77K
 |  |  |  in pipelines: 39(GETNEXT), 42(OPEN)
 |  |  |
 |  |  |--77:EXCHANGE [HASH(wr_item_sk,wr_order_number)]
@@ -684,8 +682,8 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  |     in pipelines: 42(GETNEXT)
 |  |  |
 |  |  76:EXCHANGE [HASH(ws_item_sk,ws_order_number)]
-|  |  |  mem-estimate=742.74KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=24,25,26 row-size=78B cardinality=14.80K
+|  |  |  mem-estimate=741.48KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=24,25,26 row-size=78B cardinality=14.77K
 |  |  |  in pipelines: 39(GETNEXT)
 |  |  |
 |  |  F28:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -695,7 +693,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  |  runtime filters: RF029[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=78B cardinality=14.80K
+|  |  |  tuple-ids=24,25,26 row-size=78B cardinality=14.77K
 |  |  |  in pipelines: 39(GETNEXT), 41(OPEN)
 |  |  |
 |  |  |--75:EXCHANGE [BROADCAST]
@@ -723,11 +721,11 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  |  runtime filters: RF030[bloom] <- i_item_sk, RF031[min_max] <- i_item_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=24,25 row-size=70B cardinality=71.94K
+|  |  |  tuple-ids=24,25 row-size=70B cardinality=71.78K
 |  |  |  in pipelines: 39(GETNEXT), 40(OPEN)
 |  |  |
 |  |  |--74:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  |  tuple-ids=25 row-size=42B cardinality=1.80K
 |  |  |  |  in pipelines: 40(GETNEXT)
 |  |  |  |
@@ -761,7 +759,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  |  tuple-ids=20,21,22,23N row-size=102B cardinality=58.90K
+|  |  |  tuple-ids=20,21,22,23N row-size=102B cardinality=58.77K
 |  |  |  in pipelines: 32(GETNEXT), 35(OPEN)
 |  |  |
 |  |  |--73:EXCHANGE [HASH(sr_item_sk,sr_ticket_number)]
@@ -783,7 +781,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  |
 |  |  72:EXCHANGE [HASH(ss_item_sk,ss_ticket_number)]
 |  |  |  mem-estimate=1.72MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=20,21,22 row-size=78B cardinality=58.90K
+|  |  |  tuple-ids=20,21,22 row-size=78B cardinality=58.77K
 |  |  |  in pipelines: 32(GETNEXT)
 |  |  |
 |  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -793,7 +791,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  |  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=20,21,22 row-size=78B cardinality=58.90K
+|  |  |  tuple-ids=20,21,22 row-size=78B cardinality=58.77K
 |  |  |  in pipelines: 32(GETNEXT), 34(OPEN)
 |  |  |
 |  |  |--71:EXCHANGE [BROADCAST]
@@ -821,11 +819,11 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  |  runtime filters: RF026[bloom] <- i_item_sk, RF027[min_max] <- i_item_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=20,21 row-size=70B cardinality=288.04K
+|  |  |  tuple-ids=20,21 row-size=70B cardinality=287.40K
 |  |  |  in pipelines: 32(GETNEXT), 33(OPEN)
 |  |  |
 |  |  |--70:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  |  tuple-ids=21 row-size=42B cardinality=1.80K
 |  |  |  |  in pipelines: 33(GETNEXT)
 |  |  |  |
@@ -860,7 +858,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  hash predicates: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  |  fk/pk conjuncts: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19N row-size=102B cardinality=29.46K
+|  |  tuple-ids=16,17,18,19N row-size=102B cardinality=29.40K
 |  |  in pipelines: 25(GETNEXT), 28(OPEN)
 |  |
 |  |--69:EXCHANGE [HASH(cr_item_sk,cr_order_number)]
@@ -881,8 +879,8 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |     in pipelines: 28(GETNEXT)
 |  |
 |  68:EXCHANGE [HASH(cs_item_sk,cs_order_number)]
-|  |  mem-estimate=1016.76KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=78B cardinality=29.46K
+|  |  mem-estimate=1015.11KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16,17,18 row-size=78B cardinality=29.40K
 |  |  in pipelines: 25(GETNEXT)
 |  |
 |  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -892,7 +890,7 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  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=16,17,18 row-size=78B cardinality=29.46K
+|  |  tuple-ids=16,17,18 row-size=78B cardinality=29.40K
 |  |  in pipelines: 25(GETNEXT), 27(OPEN)
 |  |
 |  |--67:EXCHANGE [BROADCAST]
@@ -920,11 +918,11 @@ Per-Host Resources: mem-estimate=1.99MB mem-reservation=1.94MB thread-reservatio
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  runtime filters: RF022[bloom] <- i_item_sk, RF023[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17 row-size=70B cardinality=144.16K
+|  |  tuple-ids=16,17 row-size=70B cardinality=143.84K
 |  |  in pipelines: 25(GETNEXT), 26(OPEN)
 |  |
 |  |--66:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=17 row-size=42B cardinality=1.80K
 |  |  |  in pipelines: 26(GETNEXT)
 |  |  |
@@ -971,7 +969,7 @@ Per-Host Resources: mem-estimate=11.58MB mem-reservation=2.88MB thread-reservati
 |
 64:EXCHANGE [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id)]
 |  mem-estimate=1.58MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=14 row-size=44B cardinality=103.17K
+|  tuple-ids=14 row-size=44B cardinality=102.94K
 |  in pipelines: 63(GETNEXT)
 |
 F16:PLAN FRAGMENT [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt)] hosts=3 instances=3
@@ -980,18 +978,18 @@ Per-Host Resources: mem-estimate=21.15MB mem-reservation=5.88MB thread-reservati
 |  output: sum(sales_cnt), sum(sales_amt)
 |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id
 |  mem-estimate=10.00MB mem-reservation=3.00MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=14 row-size=44B cardinality=103.17K
+|  tuple-ids=14 row-size=44B cardinality=102.94K
 |  in pipelines: 63(GETNEXT)
 |
 63:AGGREGATE [FINALIZE]
 |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 63(GETNEXT), 01(OPEN), 08(OPEN), 15(OPEN)
 |
 62:EXCHANGE [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt)]
 |  mem-estimate=1.15MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 01(GETNEXT), 08(GETNEXT), 15(GETNEXT)
 |
 F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -999,19 +997,19 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 22:AGGREGATE [STREAMING]
 |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  mem-estimate=10.00MB mem-reservation=3.00MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 01(GETNEXT), 08(GETNEXT), 15(GETNEXT)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 01(GETNEXT), 08(GETNEXT), 15(GETNEXT)
 |
 |--21:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
 |  |  hash predicates: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  fk/pk conjuncts: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11N row-size=102B cardinality=14.80K
+|  |  tuple-ids=8,9,10,11N row-size=102B cardinality=14.77K
 |  |  in pipelines: 15(GETNEXT), 18(OPEN)
 |  |
 |  |--61:EXCHANGE [HASH(wr_item_sk,wr_order_number)]
@@ -1032,8 +1030,8 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  60:EXCHANGE [HASH(ws_item_sk,ws_order_number)]
-|  |  mem-estimate=742.74KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=78B cardinality=14.80K
+|  |  mem-estimate=741.48KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8,9,10 row-size=78B cardinality=14.77K
 |  |  in pipelines: 15(GETNEXT)
 |  |
 |  F10:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1043,7 +1041,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  runtime filters: RF017[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=78B cardinality=14.80K
+|  |  tuple-ids=8,9,10 row-size=78B cardinality=14.77K
 |  |  in pipelines: 15(GETNEXT), 17(OPEN)
 |  |
 |  |--59:EXCHANGE [BROADCAST]
@@ -1071,11 +1069,11 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  runtime filters: RF018[bloom] <- i_item_sk, RF019[min_max] <- i_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9 row-size=70B cardinality=71.94K
+|  |  tuple-ids=8,9 row-size=70B cardinality=71.78K
 |  |  in pipelines: 15(GETNEXT), 16(OPEN)
 |  |
 |  |--58:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=9 row-size=42B cardinality=1.80K
 |  |  |  in pipelines: 16(GETNEXT)
 |  |  |
@@ -1110,7 +1108,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  |  tuple-ids=4,5,6,7N row-size=102B cardinality=58.90K
+|  |  tuple-ids=4,5,6,7N row-size=102B cardinality=58.77K
 |  |  in pipelines: 08(GETNEXT), 11(OPEN)
 |  |
 |  |--57:EXCHANGE [HASH(sr_item_sk,sr_ticket_number)]
@@ -1132,7 +1130,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |  |
 |  56:EXCHANGE [HASH(ss_item_sk,ss_ticket_number)]
 |  |  mem-estimate=1.72MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=4,5,6 row-size=78B cardinality=58.90K
+|  |  tuple-ids=4,5,6 row-size=78B cardinality=58.77K
 |  |  in pipelines: 08(GETNEXT)
 |  |
 |  F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1142,7 +1140,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |  |  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=4,5,6 row-size=78B cardinality=58.90K
+|  |  tuple-ids=4,5,6 row-size=78B cardinality=58.77K
 |  |  in pipelines: 08(GETNEXT), 10(OPEN)
 |  |
 |  |--55:EXCHANGE [BROADCAST]
@@ -1170,11 +1168,11 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  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=4,5 row-size=70B cardinality=288.04K
+|  |  tuple-ids=4,5 row-size=70B cardinality=287.40K
 |  |  in pipelines: 08(GETNEXT), 09(OPEN)
 |  |
 |  |--54:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=5 row-size=42B cardinality=1.80K
 |  |  |  in pipelines: 09(GETNEXT)
 |  |  |
@@ -1210,7 +1208,7 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |  hash predicates: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  fk/pk conjuncts: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3N row-size=102B cardinality=29.46K
+|  tuple-ids=0,1,2,3N row-size=102B cardinality=29.40K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--53:EXCHANGE [HASH(cr_item_sk,cr_order_number)]
@@ -1231,8 +1229,8 @@ Per-Host Resources: mem-estimate=26.83MB mem-reservation=11.50MB thread-reservat
 |     in pipelines: 04(GETNEXT)
 |
 52:EXCHANGE [HASH(cs_item_sk,cs_order_number)]
-|  mem-estimate=1016.76KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,1,2 row-size=78B cardinality=29.46K
+|  mem-estimate=1015.11KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,1,2 row-size=78B cardinality=29.40K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1242,7 +1240,7 @@ Per-Host Resources: mem-estimate=246.01MB mem-reservation=13.88MB thread-reserva
 |  fk/pk conjuncts: cs_sold_date_sk = 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=0,1,2 row-size=78B cardinality=29.46K
+|  tuple-ids=0,1,2 row-size=78B cardinality=29.40K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--51:EXCHANGE [BROADCAST]
@@ -1270,11 +1268,11 @@ Per-Host Resources: mem-estimate=246.01MB mem-reservation=13.88MB thread-reserva
 |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  runtime filters: RF010[bloom] <- i_item_sk, RF011[min_max] <- i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1 row-size=70B cardinality=144.16K
+|  tuple-ids=0,1 row-size=70B cardinality=143.84K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--50:EXCHANGE [BROADCAST]
-|  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=42B cardinality=1.80K
 |  |  in pipelines: 02(GETNEXT)
 |  |
@@ -1362,7 +1360,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |
 |  80:EXCHANGE [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id)]
 |  |  mem-estimate=1.72MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=30 row-size=44B cardinality=103.17K
+|  |  tuple-ids=30 row-size=44B cardinality=102.94K
 |  |  in pipelines: 79(GETNEXT)
 |  |
 |  F34:PLAN FRAGMENT [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt)] hosts=3 instances=6
@@ -1371,31 +1369,31 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  output: sum(sales_cnt), sum(sales_amt)
 |  |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=30 row-size=44B cardinality=103.17K
+|  |  tuple-ids=30 row-size=44B cardinality=102.94K
 |  |  in pipelines: 79(GETNEXT)
 |  |
 |  79:AGGREGATE [FINALIZE]
 |  |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 79(GETNEXT), 25(OPEN), 32(OPEN), 39(OPEN)
 |  |
 |  78:EXCHANGE [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt)]
 |  |  mem-estimate=1.26MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 25(GETNEXT), 32(GETNEXT), 39(GETNEXT)
 |  |
 |  F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reservation=1
 |  46:AGGREGATE [STREAMING]
 |  |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 25(GETNEXT), 32(GETNEXT), 39(GETNEXT)
 |  |
 |  24:UNION
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=28 row-size=32B cardinality=103.17K
+|  |  tuple-ids=28 row-size=32B cardinality=102.94K
 |  |  in pipelines: 25(GETNEXT), 32(GETNEXT), 39(GETNEXT)
 |  |
 |  |--45:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
@@ -1403,7 +1401,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  hash predicates: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  |  fk/pk conjuncts: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=24,25,26,27N row-size=102B cardinality=14.80K
+|  |  |  tuple-ids=24,25,26,27N row-size=102B cardinality=14.77K
 |  |  |  in pipelines: 39(GETNEXT), 42(OPEN)
 |  |  |
 |  |  |--F41:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1431,8 +1429,8 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |     in pipelines: 42(GETNEXT)
 |  |  |
 |  |  76:EXCHANGE [HASH(ws_item_sk,ws_order_number)]
-|  |  |  mem-estimate=742.74KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=24,25,26 row-size=78B cardinality=14.80K
+|  |  |  mem-estimate=741.48KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=24,25,26 row-size=78B cardinality=14.77K
 |  |  |  in pipelines: 39(GETNEXT)
 |  |  |
 |  |  F28:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1443,7 +1441,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  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=24,25,26 row-size=78B cardinality=14.80K
+|  |  |  tuple-ids=24,25,26 row-size=78B cardinality=14.77K
 |  |  |  in pipelines: 39(GETNEXT), 41(OPEN)
 |  |  |
 |  |  |--F46:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1479,7 +1477,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  hash predicates: ws_item_sk = i_item_sk
 |  |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=24,25 row-size=70B cardinality=71.94K
+|  |  |  tuple-ids=24,25 row-size=70B cardinality=71.78K
 |  |  |  in pipelines: 39(GETNEXT), 40(OPEN)
 |  |  |
 |  |  |--F47:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1491,7 +1489,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |  |
 |  |  |  74:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  |  tuple-ids=25 row-size=42B cardinality=1.80K
 |  |  |  |  in pipelines: 40(GETNEXT)
 |  |  |  |
@@ -1526,7 +1524,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  |  |  tuple-ids=20,21,22,23N row-size=102B cardinality=58.90K
+|  |  |  tuple-ids=20,21,22,23N row-size=102B cardinality=58.77K
 |  |  |  in pipelines: 32(GETNEXT), 35(OPEN)
 |  |  |
 |  |  |--F40:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1554,8 +1552,8 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |     in pipelines: 35(GETNEXT)
 |  |  |
 |  |  72:EXCHANGE [HASH(ss_item_sk,ss_ticket_number)]
-|  |  |  mem-estimate=1.99MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=20,21,22 row-size=78B cardinality=58.90K
+|  |  |  mem-estimate=1.98MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=20,21,22 row-size=78B cardinality=58.77K
 |  |  |  in pipelines: 32(GETNEXT)
 |  |  |
 |  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1566,7 +1564,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  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=20,21,22 row-size=78B cardinality=58.90K
+|  |  |  tuple-ids=20,21,22 row-size=78B cardinality=58.77K
 |  |  |  in pipelines: 32(GETNEXT), 34(OPEN)
 |  |  |
 |  |  |--F44:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1602,7 +1600,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  hash predicates: ss_item_sk = i_item_sk
 |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=20,21 row-size=70B cardinality=288.04K
+|  |  |  tuple-ids=20,21 row-size=70B cardinality=287.40K
 |  |  |  in pipelines: 32(GETNEXT), 33(OPEN)
 |  |  |
 |  |  |--F45:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1614,7 +1612,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |  |
 |  |  |  70:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  |  tuple-ids=21 row-size=42B cardinality=1.80K
 |  |  |  |  in pipelines: 33(GETNEXT)
 |  |  |  |
@@ -1650,7 +1648,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  hash predicates: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  |  fk/pk conjuncts: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=16,17,18,19N row-size=102B cardinality=29.46K
+|  |  tuple-ids=16,17,18,19N row-size=102B cardinality=29.40K
 |  |  in pipelines: 25(GETNEXT), 28(OPEN)
 |  |
 |  |--F39:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1678,8 +1676,8 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |     in pipelines: 28(GETNEXT)
 |  |
 |  68:EXCHANGE [HASH(cs_item_sk,cs_order_number)]
-|  |  mem-estimate=1016.76KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16,17,18 row-size=78B cardinality=29.46K
+|  |  mem-estimate=1015.11KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16,17,18 row-size=78B cardinality=29.40K
 |  |  in pipelines: 25(GETNEXT)
 |  |
 |  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1690,7 +1688,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  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=16,17,18 row-size=78B cardinality=29.46K
+|  |  tuple-ids=16,17,18 row-size=78B cardinality=29.40K
 |  |  in pipelines: 25(GETNEXT), 27(OPEN)
 |  |
 |  |--F42:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1726,7 +1724,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  hash predicates: cs_item_sk = i_item_sk
 |  |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17 row-size=70B cardinality=144.16K
+|  |  tuple-ids=16,17 row-size=70B cardinality=143.84K
 |  |  in pipelines: 25(GETNEXT), 26(OPEN)
 |  |
 |  |--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1738,7 +1736,7 @@ Per-Instance Resources: mem-estimate=58.76KB mem-reservation=0B thread-reservati
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  66:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=17 row-size=42B cardinality=1.80K
 |  |  |  in pipelines: 26(GETNEXT)
 |  |  |
@@ -1785,7 +1783,7 @@ Per-Instance Resources: mem-estimate=11.72MB mem-reservation=1.94MB thread-reser
 |
 64:EXCHANGE [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id)]
 |  mem-estimate=1.72MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=14 row-size=44B cardinality=103.17K
+|  tuple-ids=14 row-size=44B cardinality=102.94K
 |  in pipelines: 63(GETNEXT)
 |
 F16:PLAN FRAGMENT [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt)] hosts=3 instances=6
@@ -1794,31 +1792,31 @@ Per-Instance Resources: mem-estimate=21.26MB mem-reservation=3.94MB thread-reser
 |  output: sum(sales_cnt), sum(sales_amt)
 |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=14 row-size=44B cardinality=103.17K
+|  tuple-ids=14 row-size=44B cardinality=102.94K
 |  in pipelines: 63(GETNEXT)
 |
 63:AGGREGATE [FINALIZE]
 |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 63(GETNEXT), 01(OPEN), 08(OPEN), 15(OPEN)
 |
 62:EXCHANGE [HASH(d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt)]
 |  mem-estimate=1.26MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 01(GETNEXT), 08(GETNEXT), 15(GETNEXT)
 |
 F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reservation=1
+Per-Instance Resources: mem-estimate=11.98MB mem-reservation=2.00MB thread-reservation=1
 22:AGGREGATE [STREAMING]
 |  group by: d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_amt
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 01(GETNEXT), 08(GETNEXT), 15(GETNEXT)
 |
 00:UNION
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=12 row-size=32B cardinality=103.17K
+|  tuple-ids=12 row-size=32B cardinality=102.94K
 |  in pipelines: 01(GETNEXT), 08(GETNEXT), 15(GETNEXT)
 |
 |--21:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
@@ -1826,7 +1824,7 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |  hash predicates: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  fk/pk conjuncts: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9,10,11N row-size=102B cardinality=14.80K
+|  |  tuple-ids=8,9,10,11N row-size=102B cardinality=14.77K
 |  |  in pipelines: 15(GETNEXT), 18(OPEN)
 |  |
 |  |--F50:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1854,8 +1852,8 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  60:EXCHANGE [HASH(ws_item_sk,ws_order_number)]
-|  |  mem-estimate=742.74KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8,9,10 row-size=78B cardinality=14.80K
+|  |  mem-estimate=741.48KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8,9,10 row-size=78B cardinality=14.77K
 |  |  in pipelines: 15(GETNEXT)
 |  |
 |  F10:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1866,7 +1864,7 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |  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=8,9,10 row-size=78B cardinality=14.80K
+|  |  tuple-ids=8,9,10 row-size=78B cardinality=14.77K
 |  |  in pipelines: 15(GETNEXT), 17(OPEN)
 |  |
 |  |--F55:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1902,7 +1900,7 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |  hash predicates: ws_item_sk = i_item_sk
 |  |  fk/pk conjuncts: ws_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9 row-size=70B cardinality=71.94K
+|  |  tuple-ids=8,9 row-size=70B cardinality=71.78K
 |  |  in pipelines: 15(GETNEXT), 16(OPEN)
 |  |
 |  |--F56:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1914,7 +1912,7 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  58:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=9 row-size=42B cardinality=1.80K
 |  |  |  in pipelines: 16(GETNEXT)
 |  |  |
@@ -1950,7 +1948,7 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  |  tuple-ids=4,5,6,7N row-size=102B cardinality=58.90K
+|  |  tuple-ids=4,5,6,7N row-size=102B cardinality=58.77K
 |  |  in pipelines: 08(GETNEXT), 11(OPEN)
 |  |
 |  |--F49:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1978,8 +1976,8 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |     in pipelines: 11(GETNEXT)
 |  |
 |  56:EXCHANGE [HASH(ss_item_sk,ss_ticket_number)]
-|  |  mem-estimate=1.99MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=4,5,6 row-size=78B cardinality=58.90K
+|  |  mem-estimate=1.98MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=4,5,6 row-size=78B cardinality=58.77K
 |  |  in pipelines: 08(GETNEXT)
 |  |
 |  F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -1990,7 +1988,7 @@ Per-Instance Resources: mem-estimate=11.99MB 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=4,5,6 row-size=78B cardinality=58.90K
+|  |  tuple-ids=4,5,6 row-size=78B cardinality=58.77K
 |  |  in pipelines: 08(GETNEXT), 10(OPEN)
 |  |
 |  |--F53:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -2026,7 +2024,7 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |  hash predicates: ss_item_sk = i_item_sk
 |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=4,5 row-size=70B cardinality=288.04K
+|  |  tuple-ids=4,5 row-size=70B cardinality=287.40K
 |  |  in pipelines: 08(GETNEXT), 09(OPEN)
 |  |
 |  |--F54:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -2038,7 +2036,7 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  54:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=5 row-size=42B cardinality=1.80K
 |  |  |  in pipelines: 09(GETNEXT)
 |  |  |
@@ -2075,7 +2073,7 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |  hash predicates: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  fk/pk conjuncts: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,1,2,3N row-size=102B cardinality=29.46K
+|  tuple-ids=0,1,2,3N row-size=102B cardinality=29.40K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--F48:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -2103,8 +2101,8 @@ Per-Instance Resources: mem-estimate=11.99MB mem-reservation=2.00MB thread-reser
 |     in pipelines: 04(GETNEXT)
 |
 52:EXCHANGE [HASH(cs_item_sk,cs_order_number)]
-|  mem-estimate=1016.76KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,1,2 row-size=78B cardinality=29.46K
+|  mem-estimate=1015.11KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,1,2 row-size=78B cardinality=29.40K
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -2115,7 +2113,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser
 |  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=0,1,2 row-size=78B cardinality=29.46K
+|  tuple-ids=0,1,2 row-size=78B cardinality=29.40K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--F51:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -2151,7 +2149,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser
 |  hash predicates: cs_item_sk = i_item_sk
 |  fk/pk conjuncts: cs_item_sk = i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1 row-size=70B cardinality=144.16K
+|  tuple-ids=0,1 row-size=70B cardinality=143.84K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F52:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -2163,7 +2161,7 @@ Per-Instance Resources: mem-estimate=48.00MB mem-reservation=8.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  50:EXCHANGE [BROADCAST]
-|  |  mem-estimate=119.54KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=119.38KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=42B cardinality=1.80K
 |  |  in pipelines: 02(GETNEXT)
 |  |
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
index eea17b5..724d136 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
@@ -145,7 +145,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ws_web_site_sk = web_site_sk
 |  |  runtime filters: RF017[min_max] <- web_site_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N,20,21,18,19 row-size=147B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21,18,19 row-size=147B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 28(OPEN)
 |  |
 |  |--28:SCAN HDFS [tpcds_parquet.web_site]
@@ -163,7 +163,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  runtime filters: RF019[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N,20,21,18 row-size=115B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21,18 row-size=115B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 27(OPEN)
 |  |
 |  |--27:SCAN HDFS [tpcds_parquet.date_dim]
@@ -184,7 +184,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ws_promo_sk = p_promo_sk
 |  |  runtime filters: RF020[bloom] <- p_promo_sk, RF021[min_max] <- p_promo_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N,20,21 row-size=89B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21 row-size=89B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 30(OPEN)
 |  |
 |  |--30:SCAN HDFS [tpcds_parquet.promotion]
@@ -197,7 +197,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: p_channel_tv = 'N'
 |  |     parquet dictionary predicates: p_channel_tv = 'N'
 |  |     mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=1
-|  |     tuple-ids=21 row-size=17B cardinality=300
+|  |     tuple-ids=21 row-size=17B cardinality=294
 |  |     in pipelines: 30(GETNEXT)
 |  |
 |  32:HASH JOIN [INNER JOIN]
@@ -262,7 +262,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs_catalog_page_sk = cp_catalog_page_sk
 |  |  runtime filters: RF008[bloom] <- cp_catalog_page_sk, RF009[min_max] <- cp_catalog_page_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N,12,13,10,11 row-size=147B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13,10,11 row-size=147B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 16(OPEN)
 |  |
 |  |--16:SCAN HDFS [tpcds_parquet.catalog_page]
@@ -280,7 +280,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs_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=8,9N,12,13,10 row-size=115B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13,10 row-size=115B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 15(OPEN)
 |  |
 |  |--15:SCAN HDFS [tpcds_parquet.date_dim]
@@ -301,7 +301,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs_promo_sk = p_promo_sk
 |  |  runtime filters: RF012[bloom] <- p_promo_sk, RF013[min_max] <- p_promo_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N,12,13 row-size=89B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13 row-size=89B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 18(OPEN)
 |  |
 |  |--18:SCAN HDFS [tpcds_parquet.promotion]
@@ -314,7 +314,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: p_channel_tv = 'N'
 |  |     parquet dictionary predicates: p_channel_tv = 'N'
 |  |     mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=1
-|  |     tuple-ids=13 row-size=17B cardinality=300
+|  |     tuple-ids=13 row-size=17B cardinality=294
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  20:HASH JOIN [INNER JOIN]
@@ -379,7 +379,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF000[bloom] <- s_store_sk, RF001[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N,4,5,2,3 row-size=147B cardinality=286.13K
+|  tuple-ids=0,1N,4,5,2,3 row-size=147B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--04:SCAN HDFS [tpcds_parquet.store]
@@ -397,7 +397,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF002[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N,4,5,2 row-size=115B cardinality=286.13K
+|  tuple-ids=0,1N,4,5,2 row-size=115B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--03:SCAN HDFS [tpcds_parquet.date_dim]
@@ -418,7 +418,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  runtime filters: RF004[bloom] <- p_promo_sk, RF005[min_max] <- p_promo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N,4,5 row-size=89B cardinality=286.13K
+|  tuple-ids=0,1N,4,5 row-size=89B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 06(OPEN)
 |
 |--06:SCAN HDFS [tpcds_parquet.promotion]
@@ -431,7 +431,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: p_channel_tv = 'N'
 |     parquet dictionary predicates: p_channel_tv = 'N'
 |     mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=1
-|     tuple-ids=5 row-size=17B cardinality=300
+|     tuple-ids=5 row-size=17B cardinality=294
 |     in pipelines: 06(GETNEXT)
 |
 08:HASH JOIN [INNER JOIN]
@@ -581,7 +581,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati
 |  |  fk/pk conjuncts: ws_web_site_sk = web_site_sk
 |  |  runtime filters: RF017[min_max] <- web_site_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N,20,21,18,19 row-size=147B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21,18,19 row-size=147B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 28(OPEN)
 |  |
 |  |--58:EXCHANGE [BROADCAST]
@@ -606,7 +606,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati
 |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  runtime filters: RF019[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N,20,21,18 row-size=115B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21,18 row-size=115B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 27(OPEN)
 |  |
 |  |--57:EXCHANGE [BROADCAST]
@@ -634,12 +634,12 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati
 |  |  fk/pk conjuncts: ws_promo_sk = p_promo_sk
 |  |  runtime filters: RF020[bloom] <- p_promo_sk, RF021[min_max] <- p_promo_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N,20,21 row-size=89B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21 row-size=89B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 30(OPEN)
 |  |
 |  |--56:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=21 row-size=17B cardinality=300
+|  |  |  tuple-ids=21 row-size=17B cardinality=294
 |  |  |  in pipelines: 30(GETNEXT)
 |  |  |
 |  |  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -654,7 +654,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati
 |  |     parquet statistics predicates: p_channel_tv = 'N'
 |  |     parquet dictionary predicates: p_channel_tv = 'N'
 |  |     mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=1
-|  |     tuple-ids=21 row-size=17B cardinality=300
+|  |     tuple-ids=21 row-size=17B cardinality=294
 |  |     in pipelines: 30(GETNEXT)
 |  |
 |  32:HASH JOIN [INNER JOIN, BROADCAST]
@@ -747,7 +747,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_catalog_page_sk = cp_catalog_page_sk
 |  |  runtime filters: RF008[bloom] <- cp_catalog_page_sk, RF009[min_max] <- cp_catalog_page_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N,12,13,10,11 row-size=147B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13,10,11 row-size=147B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 16(OPEN)
 |  |
 |  |--51:EXCHANGE [BROADCAST]
@@ -772,7 +772,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_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=8,9N,12,13,10 row-size=115B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13,10 row-size=115B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 15(OPEN)
 |  |
 |  |--50:EXCHANGE [BROADCAST]
@@ -800,12 +800,12 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_promo_sk = p_promo_sk
 |  |  runtime filters: RF012[bloom] <- p_promo_sk, RF013[min_max] <- p_promo_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N,12,13 row-size=89B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13 row-size=89B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 18(OPEN)
 |  |
 |  |--49:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=13 row-size=17B cardinality=300
+|  |  |  tuple-ids=13 row-size=17B cardinality=294
 |  |  |  in pipelines: 18(GETNEXT)
 |  |  |
 |  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -820,7 +820,7 @@ Per-Host Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reservati
 |  |     parquet statistics predicates: p_channel_tv = 'N'
 |  |     parquet dictionary predicates: p_channel_tv = 'N'
 |  |     mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=1
-|  |     tuple-ids=13 row-size=17B cardinality=300
+|  |     tuple-ids=13 row-size=17B cardinality=294
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  20:HASH JOIN [INNER JOIN, BROADCAST]
@@ -913,7 +913,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  runtime filters: RF000[bloom] <- s_store_sk, RF001[min_max] <- s_store_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N,4,5,2,3 row-size=147B cardinality=286.13K
+|  tuple-ids=0,1N,4,5,2,3 row-size=147B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--44:EXCHANGE [BROADCAST]
@@ -938,7 +938,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF002[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N,4,5,2 row-size=115B cardinality=286.13K
+|  tuple-ids=0,1N,4,5,2 row-size=115B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--43:EXCHANGE [BROADCAST]
@@ -966,12 +966,12 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  runtime filters: RF004[bloom] <- p_promo_sk, RF005[min_max] <- p_promo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N,4,5 row-size=89B cardinality=286.13K
+|  tuple-ids=0,1N,4,5 row-size=89B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 06(OPEN)
 |
 |--42:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=17B cardinality=300
+|  |  tuple-ids=5 row-size=17B cardinality=294
 |  |  in pipelines: 06(GETNEXT)
 |  |
 |  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -986,7 +986,7 @@ Per-Host Resources: mem-estimate=141.64MB mem-reservation=34.75MB thread-reserva
 |     parquet statistics predicates: p_channel_tv = 'N'
 |     parquet dictionary predicates: p_channel_tv = 'N'
 |     mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=1
-|     tuple-ids=5 row-size=17B cardinality=300
+|     tuple-ids=5 row-size=17B cardinality=294
 |     in pipelines: 06(GETNEXT)
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1151,7 +1151,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |  hash predicates: ws_web_site_sk = web_site_sk
 |  |  fk/pk conjuncts: ws_web_site_sk = web_site_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N,20,21,18,19 row-size=147B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21,18,19 row-size=147B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 28(OPEN)
 |  |
 |  |--F34:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1184,7 +1184,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |  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=16,17N,20,21,18 row-size=115B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21,18 row-size=115B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 27(OPEN)
 |  |
 |  |--F35:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1220,7 +1220,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |  hash predicates: ws_promo_sk = p_promo_sk
 |  |  fk/pk conjuncts: ws_promo_sk = p_promo_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N,20,21 row-size=89B cardinality=71.46K
+|  |  tuple-ids=16,17N,20,21 row-size=89B cardinality=70.03K
 |  |  in pipelines: 25(GETNEXT), 30(OPEN)
 |  |
 |  |--F36:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1233,7 +1233,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |  |
 |  |  56:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=21 row-size=17B cardinality=300
+|  |  |  tuple-ids=21 row-size=17B cardinality=294
 |  |  |  in pipelines: 30(GETNEXT)
 |  |  |
 |  |  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1248,7 +1248,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |     parquet statistics predicates: p_channel_tv = 'N'
 |  |     parquet dictionary predicates: p_channel_tv = 'N'
 |  |     mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=0
-|  |     tuple-ids=21 row-size=17B cardinality=300
+|  |     tuple-ids=21 row-size=17B cardinality=294
 |  |     in pipelines: 30(GETNEXT)
 |  |
 |  32:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1359,7 +1359,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |  hash predicates: cs_catalog_page_sk = cp_catalog_page_sk
 |  |  fk/pk conjuncts: cs_catalog_page_sk = cp_catalog_page_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N,12,13,10,11 row-size=147B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13,10,11 row-size=147B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 16(OPEN)
 |  |
 |  |--F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1392,7 +1392,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |  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=8,9N,12,13,10 row-size=115B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13,10 row-size=115B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 15(OPEN)
 |  |
 |  |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1428,7 +1428,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |  hash predicates: cs_promo_sk = p_promo_sk
 |  |  fk/pk conjuncts: cs_promo_sk = p_promo_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N,12,13 row-size=89B cardinality=143.20K
+|  |  tuple-ids=8,9N,12,13 row-size=89B cardinality=140.34K
 |  |  in pipelines: 13(GETNEXT), 18(OPEN)
 |  |
 |  |--F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1441,7 +1441,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |  |
 |  |  49:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=13 row-size=17B cardinality=300
+|  |  |  tuple-ids=13 row-size=17B cardinality=294
 |  |  |  in pipelines: 18(GETNEXT)
 |  |  |
 |  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1456,7 +1456,7 @@ Per-Instance Resources: mem-estimate=40.51MB mem-reservation=7.94MB thread-reser
 |  |     parquet statistics predicates: p_channel_tv = 'N'
 |  |     parquet dictionary predicates: p_channel_tv = 'N'
 |  |     mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=0
-|  |     tuple-ids=13 row-size=17B cardinality=300
+|  |     tuple-ids=13 row-size=17B cardinality=294
 |  |     in pipelines: 18(GETNEXT)
 |  |
 |  20:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1567,7 +1567,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser
 |  hash predicates: ss_store_sk = s_store_sk
 |  fk/pk conjuncts: ss_store_sk = s_store_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N,4,5,2,3 row-size=147B cardinality=286.13K
+|  tuple-ids=0,1N,4,5,2,3 row-size=147B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--F24:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1600,7 +1600,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.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=0,1N,4,5,2 row-size=115B cardinality=286.13K
+|  tuple-ids=0,1N,4,5,2 row-size=115B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1636,7 +1636,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser
 |  hash predicates: ss_promo_sk = p_promo_sk
 |  fk/pk conjuncts: ss_promo_sk = p_promo_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N,4,5 row-size=89B cardinality=286.13K
+|  tuple-ids=0,1N,4,5 row-size=89B cardinality=280.41K
 |  in pipelines: 01(GETNEXT), 06(OPEN)
 |
 |--F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1649,7 +1649,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser
 |  |
 |  42:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=17B cardinality=300
+|  |  tuple-ids=5 row-size=17B cardinality=294
 |  |  in pipelines: 06(GETNEXT)
 |  |
 |  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -1664,7 +1664,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=6.00MB thread-reser
 |     parquet statistics predicates: p_channel_tv = 'N'
 |     parquet dictionary predicates: p_channel_tv = 'N'
 |     mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=0
-|     tuple-ids=5 row-size=17B cardinality=300
+|     tuple-ids=5 row-size=17B cardinality=294
 |     in pipelines: 06(GETNEXT)
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
index 5643bd5..8a487b9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
@@ -75,7 +75,7 @@ PLAN-ROOT SINK
 |  other join predicates: sum(cr_return_amt_inc_tax) > avg(ctr_total_return) * CAST(1.2 AS DECIMAL(2,1))
 |  runtime filters: RF000[bloom] <- ctr2.ctr_state, RF001[min_max] <- ctr2.ctr_state
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3,6,5 row-size=325B cardinality=4.35K
+|  tuple-ids=3,6,5 row-size=325B cardinality=4.21K
 |  in pipelines: 05(GETNEXT), 14(OPEN)
 |
 |--14:AGGREGATE [FINALIZE]
@@ -148,7 +148,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: none
 |  runtime filters: RF002[bloom] <- c_customer_sk, RF003[min_max] <- c_customer_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3,6,5 row-size=325B cardinality=4.35K
+|  tuple-ids=3,6,5 row-size=325B cardinality=4.21K
 |  in pipelines: 05(GETNEXT), 07(OPEN)
 |
 |--15:HASH JOIN [INNER JOIN]
@@ -156,7 +156,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  |  runtime filters: RF008[bloom] <- ca_address_sk, RF009[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=6,5 row-size=291B cardinality=2.27K
+|  |  tuple-ids=6,5 row-size=291B cardinality=2.21K
 |  |  in pipelines: 07(GETNEXT), 06(OPEN)
 |  |
 |  |--06:SCAN HDFS [tpcds_parquet.customer_address]
@@ -169,7 +169,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_state = 'GA'
 |  |     parquet dictionary predicates: ca_state = 'GA'
 |  |     mem-estimate=128.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=5 row-size=204B cardinality=980
+|  |     tuple-ids=5 row-size=204B cardinality=950
 |  |     in pipelines: 06(GETNEXT)
 |  |
 |  07:SCAN HDFS [tpcds_parquet.customer]
@@ -257,7 +257,7 @@ PLAN-ROOT SINK
 |  in pipelines: 18(GETNEXT)
 |
 F03:PLAN FRAGMENT [HASH(cr_returning_customer_sk,ca_state)] hosts=1 instances=1
-Per-Host Resources: mem-estimate=17.70MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=2.00MB
+Per-Host Resources: mem-estimate=17.68MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=2.00MB
 18:TOP-N [LIMIT=100]
 |  order by: c_customer_id ASC, c_salutation ASC, c_first_name ASC, c_last_name ASC, ca_street_number ASC, ca_street_name ASC, ca_street_type ASC, ca_suite_number ASC, ca_city ASC, ca_county ASC, ca_state ASC, ca_zip ASC, ca_country ASC, ca_gmt_offset ASC, ca_location_type ASC, ctr_total_return ASC
 |  mem-estimate=28.79KB mem-reservation=0B thread-reservation=0
@@ -269,7 +269,7 @@ Per-Host Resources: mem-estimate=17.70MB mem-reservation=8.75MB thread-reservati
 |  other join predicates: sum(cr_return_amt_inc_tax) > avg(ctr_total_return) * CAST(1.2 AS DECIMAL(2,1))
 |  runtime filters: RF000[bloom] <- ctr2.ctr_state, RF001[min_max] <- ctr2.ctr_state
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3,6,5 row-size=325B cardinality=4.35K
+|  tuple-ids=3,6,5 row-size=325B cardinality=4.21K
 |  in pipelines: 22(GETNEXT), 30(OPEN)
 |
 |--31:EXCHANGE [BROADCAST]
@@ -391,27 +391,27 @@ Per-Host Resources: mem-estimate=17.70MB mem-reservation=8.75MB thread-reservati
 |  fk/pk conjuncts: none
 |  runtime filters: RF002[bloom] <- c_customer_sk, RF003[min_max] <- c_customer_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3,6,5 row-size=325B cardinality=4.35K
+|  tuple-ids=3,6,5 row-size=325B cardinality=4.21K
 |  in pipelines: 22(GETNEXT), 07(OPEN)
 |
 |--24:EXCHANGE [BROADCAST]
-|  |  mem-estimate=944.87KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=6,5 row-size=291B cardinality=2.27K
+|  |  mem-estimate=924.99KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=6,5 row-size=291B cardinality=2.21K
 |  |  in pipelines: 07(GETNEXT)
 |  |
 |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=99.32MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=99.31MB mem-reservation=6.94MB thread-reservation=2 runtime-filters-memory=1.00MB
 |  15:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: c_current_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: c_current_addr_sk = ca_address_sk
 |  |  runtime filters: RF008[bloom] <- ca_address_sk, RF009[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=6,5 row-size=291B cardinality=2.27K
+|  |  tuple-ids=6,5 row-size=291B cardinality=2.21K
 |  |  in pipelines: 07(GETNEXT), 06(OPEN)
 |  |
 |  |--23:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=393.52KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=5 row-size=204B cardinality=980
+|  |  |  mem-estimate=381.47KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=204B cardinality=950
 |  |  |  in pipelines: 06(GETNEXT)
 |  |  |
 |  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -426,7 +426,7 @@ Per-Host Resources: mem-estimate=17.70MB mem-reservation=8.75MB thread-reservati
 |  |     parquet statistics predicates: ca_state = 'GA'
 |  |     parquet dictionary predicates: ca_state = 'GA'
 |  |     mem-estimate=128.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=5 row-size=204B cardinality=980
+|  |     tuple-ids=5 row-size=204B cardinality=950
 |  |     in pipelines: 06(GETNEXT)
 |  |
 |  07:SCAN HDFS [tpcds_parquet.customer, RANDOM]
@@ -554,7 +554,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser
 |  hash predicates: ca_state = ctr2.ctr_state
 |  other join predicates: sum(cr_return_amt_inc_tax) > avg(ctr_total_return) * CAST(1.2 AS DECIMAL(2,1))
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3,6,5 row-size=325B cardinality=4.35K
+|  tuple-ids=3,6,5 row-size=325B cardinality=4.21K
 |  in pipelines: 22(GETNEXT), 30(OPEN)
 |
 |--F12:PLAN FRAGMENT [HASH(cr_returning_customer_sk,ca_state)] hosts=1 instances=1
@@ -702,11 +702,11 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser
 |  hash predicates: cr_returning_customer_sk = c_customer_sk
 |  fk/pk conjuncts: none
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=3,6,5 row-size=325B cardinality=4.35K
+|  tuple-ids=3,6,5 row-size=325B cardinality=4.21K
 |  in pipelines: 22(GETNEXT), 07(OPEN)
 |
 |--F15:PLAN FRAGMENT [HASH(cr_returning_customer_sk,ca_state)] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=5.80MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=5.78MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: c_customer_sk
@@ -714,8 +714,8 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  24:EXCHANGE [BROADCAST]
-|  |  mem-estimate=944.87KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=6,5 row-size=291B cardinality=2.27K
+|  |  mem-estimate=924.99KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=6,5 row-size=291B cardinality=2.21K
 |  |  in pipelines: 07(GETNEXT)
 |  |
 |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -726,11 +726,11 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser
 |  |  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=64.00KB thread-reservation=0
-|  |  tuple-ids=6,5 row-size=291B cardinality=2.27K
+|  |  tuple-ids=6,5 row-size=291B cardinality=2.21K
 |  |  in pipelines: 07(GETNEXT), 06(OPEN)
 |  |
 |  |--F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  |  Per-Instance Resources: mem-estimate=5.26MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=5.25MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=04
 |  |  |  build expressions: ca_address_sk
@@ -738,8 +738,8 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  23:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=393.52KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=5 row-size=204B cardinality=980
+|  |  |  mem-estimate=381.47KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=204B cardinality=950
 |  |  |  in pipelines: 06(GETNEXT)
 |  |  |
 |  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -754,7 +754,7 @@ Per-Instance Resources: mem-estimate=10.89MB mem-reservation=2.88MB thread-reser
 |  |     parquet statistics predicates: ca_state = 'GA'
 |  |     parquet dictionary predicates: ca_state = 'GA'
 |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=5 row-size=204B cardinality=980
+|  |     tuple-ids=5 row-size=204B cardinality=950
 |  |     in pipelines: 06(GETNEXT)
 |  |
 |  07:SCAN HDFS [tpcds_parquet.customer, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
index 252a306..7b7ddda 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
@@ -26,8 +26,8 @@ PLAN-ROOT SINK
 |
 09:TOP-N [LIMIT=100]
 |  order by: c_customer_id ASC
-|  mem-estimate=1.25KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=5 row-size=64B cardinality=20
+|  mem-estimate=1.19KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=5 row-size=64B cardinality=19
 |  in pipelines: 09(GETNEXT), 04(OPEN)
 |
 08:HASH JOIN [INNER JOIN]
@@ -35,7 +35,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: none
 |  runtime filters: RF000[bloom] <- c_current_cdemo_sk, RF001[min_max] <- c_current_cdemo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,2,0,1,3 row-size=125B cardinality=20
+|  tuple-ids=4,2,0,1,3 row-size=125B cardinality=19
 |  in pipelines: 04(GETNEXT), 02(OPEN)
 |
 |--07:HASH JOIN [INNER JOIN]
@@ -43,7 +43,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: hd_income_band_sk = ib_income_band_sk
 |  |  runtime filters: RF002[bloom] <- ib_income_band_sk, RF003[min_max] <- ib_income_band_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=18
+|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=17
 |  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |
 |  |--03:SCAN HDFS [tpcds_parquet.income_band]
@@ -64,7 +64,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: hd_demo_sk = c_current_hdemo_sk
 |  |  runtime filters: RF004[bloom] <- c_current_hdemo_sk, RF005[min_max] <- c_current_hdemo_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,0,1 row-size=109B cardinality=173
+|  |  tuple-ids=2,0,1 row-size=109B cardinality=166
 |  |  in pipelines: 02(GETNEXT), 00(OPEN)
 |  |
 |  |--05:HASH JOIN [INNER JOIN]
@@ -72,7 +72,7 @@ PLAN-ROOT SINK
 |  |  |  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=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,1 row-size=101B cardinality=177
+|  |  |  tuple-ids=0,1 row-size=101B cardinality=170
 |  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |  |
 |  |  |--01:SCAN HDFS [tpcds_parquet.customer_address]
@@ -85,7 +85,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ca_city = 'Edgewood'
 |  |  |     parquet dictionary predicates: ca_city = 'Edgewood'
 |  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=1 row-size=25B cardinality=76
+|  |  |     tuple-ids=1 row-size=25B cardinality=73
 |  |  |     in pipelines: 01(GETNEXT)
 |  |  |
 |  |  00:SCAN HDFS [tpcds_parquet.customer]
@@ -133,15 +133,15 @@ PLAN-ROOT SINK
 |  order by: c_customer_id ASC
 |  limit: 100
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=5 row-size=64B cardinality=20
+|  tuple-ids=5 row-size=64B cardinality=19
 |  in pipelines: 09(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservation=2 runtime-filters-memory=1.00MB
 09:TOP-N [LIMIT=100]
 |  order by: c_customer_id ASC
-|  mem-estimate=1.25KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=5 row-size=64B cardinality=20
+|  mem-estimate=1.19KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=5 row-size=64B cardinality=19
 |  in pipelines: 09(GETNEXT), 04(OPEN)
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
@@ -149,22 +149,22 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati
 |  fk/pk conjuncts: none
 |  runtime filters: RF000[bloom] <- c_current_cdemo_sk, RF001[min_max] <- c_current_cdemo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,2,0,1,3 row-size=125B cardinality=20
+|  tuple-ids=4,2,0,1,3 row-size=125B cardinality=19
 |  in pipelines: 04(GETNEXT), 02(OPEN)
 |
 |--13:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=18
+|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=17
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=37.93MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=37.92MB mem-reservation=5.94MB thread-reservation=2 runtime-filters-memory=2.00MB
 |  07:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: hd_income_band_sk = ib_income_band_sk
 |  |  fk/pk conjuncts: hd_income_band_sk = ib_income_band_sk
 |  |  runtime filters: RF002[bloom] <- ib_income_band_sk, RF003[min_max] <- ib_income_band_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=18
+|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=17
 |  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |
 |  |--12:EXCHANGE [BROADCAST]
@@ -192,12 +192,12 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati
 |  |  fk/pk conjuncts: hd_demo_sk = c_current_hdemo_sk
 |  |  runtime filters: RF004[bloom] <- c_current_hdemo_sk, RF005[min_max] <- c_current_hdemo_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,0,1 row-size=109B cardinality=173
+|  |  tuple-ids=2,0,1 row-size=109B cardinality=166
 |  |  in pipelines: 02(GETNEXT), 00(OPEN)
 |  |
 |  |--11:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=36.27KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0,1 row-size=101B cardinality=177
+|  |  |  mem-estimate=34.83KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=0,1 row-size=101B cardinality=170
 |  |  |  in pipelines: 00(GETNEXT)
 |  |  |
 |  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -207,12 +207,12 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati
 |  |  |  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=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,1 row-size=101B cardinality=177
+|  |  |  tuple-ids=0,1 row-size=101B cardinality=170
 |  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |  |
 |  |  |--10:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  |  tuple-ids=1 row-size=25B cardinality=76
+|  |  |  |  tuple-ids=1 row-size=25B cardinality=73
 |  |  |  |  in pipelines: 01(GETNEXT)
 |  |  |  |
 |  |  |  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -227,7 +227,7 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati
 |  |  |     parquet statistics predicates: ca_city = 'Edgewood'
 |  |  |     parquet dictionary predicates: ca_city = 'Edgewood'
 |  |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=1 row-size=25B cardinality=76
+|  |  |     tuple-ids=1 row-size=25B cardinality=73
 |  |  |     in pipelines: 01(GETNEXT)
 |  |  |
 |  |  00:SCAN HDFS [tpcds_parquet.customer, RANDOM]
@@ -275,7 +275,7 @@ PLAN-ROOT SINK
 |  order by: c_customer_id ASC
 |  limit: 100
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=5 row-size=64B cardinality=20
+|  tuple-ids=5 row-size=64B cardinality=19
 |  in pipelines: 09(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -283,8 +283,8 @@ Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-res
 Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1
 09:TOP-N [LIMIT=100]
 |  order by: c_customer_id ASC
-|  mem-estimate=1.25KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=5 row-size=64B cardinality=20
+|  mem-estimate=1.19KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=5 row-size=64B cardinality=19
 |  in pipelines: 09(GETNEXT), 04(OPEN)
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
@@ -292,7 +292,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser
 |  hash predicates: sr_cdemo_sk = c_current_cdemo_sk
 |  fk/pk conjuncts: none
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,2,0,1,3 row-size=125B cardinality=20
+|  tuple-ids=4,2,0,1,3 row-size=125B cardinality=19
 |  in pipelines: 04(GETNEXT), 02(OPEN)
 |
 |--F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -305,7 +305,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser
 |  |
 |  13:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=18
+|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=17
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -316,7 +316,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser
 |  |  hash predicates: hd_income_band_sk = ib_income_band_sk
 |  |  fk/pk conjuncts: hd_income_band_sk = ib_income_band_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=18
+|  |  tuple-ids=2,0,1,3 row-size=121B cardinality=17
 |  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |
 |  |--F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -352,7 +352,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser
 |  |  hash predicates: hd_demo_sk = c_current_hdemo_sk
 |  |  fk/pk conjuncts: hd_demo_sk = c_current_hdemo_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=2,0,1 row-size=109B cardinality=173
+|  |  tuple-ids=2,0,1 row-size=109B cardinality=166
 |  |  in pipelines: 02(GETNEXT), 00(OPEN)
 |  |
 |  |--F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -364,8 +364,8 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  11:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=36.27KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0,1 row-size=101B cardinality=177
+|  |  |  mem-estimate=34.83KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=0,1 row-size=101B cardinality=170
 |  |  |  in pipelines: 00(GETNEXT)
 |  |  |
 |  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -376,7 +376,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser
 |  |  |  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=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,1 row-size=101B cardinality=177
+|  |  |  tuple-ids=0,1 row-size=101B cardinality=170
 |  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |  |
 |  |  |--F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -389,7 +389,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser
 |  |  |  |
 |  |  |  10:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
-|  |  |  |  tuple-ids=1 row-size=25B cardinality=76
+|  |  |  |  tuple-ids=1 row-size=25B cardinality=73
 |  |  |  |  in pipelines: 01(GETNEXT)
 |  |  |  |
 |  |  |  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -404,7 +404,7 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reser
 |  |  |     parquet statistics predicates: ca_city = 'Edgewood'
 |  |  |     parquet dictionary predicates: ca_city = 'Edgewood'
 |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |  |     tuple-ids=1 row-size=25B cardinality=76
+|  |  |     tuple-ids=1 row-size=25B cardinality=73
 |  |  |     in pipelines: 01(GETNEXT)
 |  |  |
 |  |  00:SCAN HDFS [tpcds_parquet.customer, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
index 81cb1ab..2115ca5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
@@ -85,7 +85,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: wr_reason_sk = r_reason_sk
 |  runtime filters: RF000[bloom] <- r_reason_sk, RF001[min_max] <- r_reason_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,5,3,1,0,6,2,7 row-size=241B cardinality=365
+|  tuple-ids=4,5,3,1,0,6,2,7 row-size=241B cardinality=359
 |  in pipelines: 04(GETNEXT), 07(OPEN)
 |
 |--07:SCAN HDFS [tpcds_parquet.reason]
@@ -103,7 +103,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: cd2.cd_demo_sk = wr_returning_cdemo_sk
 |  runtime filters: RF002[bloom] <- wr_returning_cdemo_sk, RF003[bloom] <- cd1.cd_marital_status, RF004[bloom] <- cd1.cd_education_status, RF005[min_max] <- wr_returning_cdemo_sk, RF006[min_max] <- cd1.cd_marital_status, RF007[min_max] <- cd1.cd_education_status
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,5,3,1,0,6,2 row-size=208B cardinality=365
+|  tuple-ids=4,5,3,1,0,6,2 row-size=208B cardinality=359
 |  in pipelines: 04(GETNEXT), 05(OPEN)
 |
 |--12:HASH JOIN [INNER JOIN]
@@ -111,7 +111,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ws_web_page_sk = wp_web_page_sk
 |  |  runtime filters: RF008[bloom] <- wp_web_page_sk, RF009[min_max] <- wp_web_page_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=5,3,1,0,6,2 row-size=170B cardinality=365
+|  |  tuple-ids=5,3,1,0,6,2 row-size=170B cardinality=359
 |  |  in pipelines: 05(GETNEXT), 02(OPEN)
 |  |
 |  |--02:SCAN HDFS [tpcds_parquet.web_page]
@@ -130,7 +130,7 @@ PLAN-ROOT SINK
 |  |  other predicates: ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit >=  [...]
 |  |  runtime filters: RF010[bloom] <- wr_refunded_addr_sk, RF011[min_max] <- wr_refunded_addr_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=5,3,1,0,6 row-size=166B cardinality=365
+|  |  tuple-ids=5,3,1,0,6 row-size=166B cardinality=359
 |  |  in pipelines: 05(GETNEXT), 03(OPEN)
 |  |
 |  |--10:HASH JOIN [INNER JOIN]
@@ -218,7 +218,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: ca_state IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR'), ca_country = 'United States'
 |     parquet dictionary predicates: ca_state IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR'), ca_country = 'United States'
 |     mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=5 row-size=43B cardinality=8.82K
+|     tuple-ids=5 row-size=43B cardinality=8.69K
 |     in pipelines: 05(GETNEXT)
 |
 04:SCAN HDFS [tpcds_parquet.customer_demographics cd2]
@@ -283,7 +283,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat
 |  fk/pk conjuncts: wr_reason_sk = r_reason_sk
 |  runtime filters: RF000[bloom] <- r_reason_sk, RF001[min_max] <- r_reason_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,0,3,1,6,5,2,7 row-size=241B cardinality=365
+|  tuple-ids=4,0,3,1,6,5,2,7 row-size=241B cardinality=359
 |  in pipelines: 04(GETNEXT), 07(OPEN)
 |
 |--25:EXCHANGE [BROADCAST]
@@ -308,12 +308,12 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat
 |  fk/pk conjuncts: cd2.cd_demo_sk = wr_returning_cdemo_sk
 |  runtime filters: RF002[bloom] <- wr_returning_cdemo_sk, RF003[bloom] <- cd1.cd_marital_status, RF004[bloom] <- cd1.cd_education_status, RF005[min_max] <- wr_returning_cdemo_sk, RF006[min_max] <- cd1.cd_marital_status, RF007[min_max] <- cd1.cd_education_status
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,0,3,1,6,5,2 row-size=208B cardinality=365
+|  tuple-ids=4,0,3,1,6,5,2 row-size=208B cardinality=359
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 |--24:EXCHANGE [BROADCAST]
-|  |  mem-estimate=198.44KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,3,1,6,5,2 row-size=170B cardinality=365
+|  |  mem-estimate=195.18KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,3,1,6,5,2 row-size=170B cardinality=359
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [HASH(wr_refunded_addr_sk)] hosts=2 instances=2
@@ -323,7 +323,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat
 |  |  fk/pk conjuncts: ws_web_page_sk = wp_web_page_sk
 |  |  runtime filters: RF008[bloom] <- wp_web_page_sk, RF009[min_max] <- wp_web_page_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3,1,6,5,2 row-size=170B cardinality=365
+|  |  tuple-ids=0,3,1,6,5,2 row-size=170B cardinality=359
 |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |
 |  |--23:EXCHANGE [BROADCAST]
@@ -349,12 +349,12 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat
 |  |  other predicates: ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit >=  [...]
 |  |  runtime filters: RF010[bloom] <- ca_address_sk, RF011[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3,1,6,5 row-size=166B cardinality=365
+|  |  tuple-ids=0,3,1,6,5 row-size=166B cardinality=359
 |  |  in pipelines: 00(GETNEXT), 05(OPEN)
 |  |
 |  |--22:EXCHANGE [HASH(ca_address_sk)]
-|  |  |  mem-estimate=417.54KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=5 row-size=43B cardinality=8.82K
+|  |  |  mem-estimate=411.83KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=43B cardinality=8.69K
 |  |  |  in pipelines: 05(GETNEXT)
 |  |  |
 |  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -369,7 +369,7 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat
 |  |     parquet statistics predicates: ca_state IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR'), ca_country = 'United States'
 |  |     parquet dictionary predicates: ca_state IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR'), ca_country = 'United States'
 |  |     mem-estimate=48.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=5 row-size=43B cardinality=8.82K
+|  |     tuple-ids=5 row-size=43B cardinality=8.69K
 |  |     in pipelines: 05(GETNEXT)
 |  |
 |  21:EXCHANGE [HASH(wr_refunded_addr_sk)]
@@ -544,7 +544,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese
 |  hash predicates: wr_reason_sk = r_reason_sk
 |  fk/pk conjuncts: wr_reason_sk = r_reason_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,0,3,1,6,5,2,7 row-size=241B cardinality=365
+|  tuple-ids=4,0,3,1,6,5,2,7 row-size=241B cardinality=359
 |  in pipelines: 04(GETNEXT), 07(OPEN)
 |
 |--F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -577,7 +577,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese
 |  hash predicates: cd2.cd_demo_sk = wr_returning_cdemo_sk, cd2.cd_marital_status = cd1.cd_marital_status, cd2.cd_education_status = cd1.cd_education_status
 |  fk/pk conjuncts: cd2.cd_demo_sk = wr_returning_cdemo_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=4,0,3,1,6,5,2 row-size=208B cardinality=365
+|  tuple-ids=4,0,3,1,6,5,2 row-size=208B cardinality=359
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 |--F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -589,8 +589,8 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  24:EXCHANGE [BROADCAST]
-|  |  mem-estimate=198.44KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,3,1,6,5,2 row-size=170B cardinality=365
+|  |  mem-estimate=195.18KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,3,1,6,5,2 row-size=170B cardinality=359
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [HASH(wr_refunded_addr_sk)] hosts=2 instances=2
@@ -600,7 +600,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese
 |  |  hash predicates: ws_web_page_sk = wp_web_page_sk
 |  |  fk/pk conjuncts: ws_web_page_sk = wp_web_page_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3,1,6,5,2 row-size=170B cardinality=365
+|  |  tuple-ids=0,3,1,6,5,2 row-size=170B cardinality=359
 |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |
 |  |--F14:PLAN FRAGMENT [HASH(wr_refunded_addr_sk)] hosts=2 instances=2
@@ -634,11 +634,11 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese
 |  |  fk/pk conjuncts: wr_refunded_addr_sk = ca_address_sk
 |  |  other predicates: ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit <= CAST(300 AS DECIMAL(5,0)) OR ws_net_profit >= CAST(50 AS DECIMAL(3,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit <= CAST(250 AS DECIMAL(5,0)), ca_state IN ('IN', 'OH', 'NJ') OR ws_net_profit >= CAST(150 AS DECIMAL(5,0)) OR ws_net_profit >=  [...]
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3,1,6,5 row-size=166B cardinality=365
+|  |  tuple-ids=0,3,1,6,5 row-size=166B cardinality=359
 |  |  in pipelines: 00(GETNEXT), 05(OPEN)
 |  |
 |  |--F15:PLAN FRAGMENT [HASH(wr_refunded_addr_sk)] hosts=2 instances=2
-|  |  |  Per-Instance Resources: mem-estimate=3.35MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=3.34MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=02
 |  |  |  build expressions: ca_address_sk
@@ -646,8 +646,8 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  22:EXCHANGE [HASH(ca_address_sk)]
-|  |  |  mem-estimate=417.54KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=5 row-size=43B cardinality=8.82K
+|  |  |  mem-estimate=411.83KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=43B cardinality=8.69K
 |  |  |  in pipelines: 05(GETNEXT)
 |  |  |
 |  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -662,7 +662,7 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.00MB thread-rese
 |  |     parquet statistics predicates: ca_state IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR'), ca_country = 'United States'
 |  |     parquet dictionary predicates: ca_state IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR'), ca_country = 'United States'
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=5 row-size=43B cardinality=8.82K
+|  |     tuple-ids=5 row-size=43B cardinality=8.69K
 |  |     in pipelines: 05(GETNEXT)
 |  |
 |  21:EXCHANGE [HASH(wr_refunded_addr_sk)]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
index 7ed806e..3e6c285 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
@@ -42,14 +42,14 @@ PLAN-ROOT SINK
 14:SORT
 |  order by: sum(cr_net_loss) DESC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=8 row-size=92B cardinality=191
+|  tuple-ids=8 row-size=92B cardinality=185
 |  in pipelines: 14(GETNEXT), 13(OPEN)
 |
 13:AGGREGATE [FINALIZE]
 |  output: sum(cr_net_loss)
 |  group by: cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7 row-size=127B cardinality=191
+|  tuple-ids=7 row-size=127B cardinality=185
 |  in pipelines: 13(GETNEXT), 01(OPEN)
 |
 12:HASH JOIN [INNER JOIN]
@@ -57,7 +57,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: cr_call_center_sk = cc_call_center_sk
 |  runtime filters: RF000[bloom] <- cc_call_center_sk, RF001[min_max] <- cc_call_center_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4,2,0 row-size=194B cardinality=191
+|  tuple-ids=1,5,3,6,4,2,0 row-size=194B cardinality=185
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--00:SCAN HDFS [tpcds_parquet.call_center]
@@ -75,7 +75,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: cr_returned_date_sk = d_date_sk
 |  runtime filters: RF002[bloom] <- d_date_sk, RF003[min_max] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4,2 row-size=114B cardinality=191
+|  tuple-ids=1,5,3,6,4,2 row-size=114B cardinality=185
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.date_dim]
@@ -96,7 +96,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: none
 |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4 row-size=102B cardinality=3.61K
+|  tuple-ids=1,5,3,6,4 row-size=102B cardinality=3.50K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--09:HASH JOIN [INNER JOIN]
@@ -104,7 +104,7 @@ PLAN-ROOT SINK
 |  |  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=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.89K
+|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.83K
 |  |  in pipelines: 05(GETNEXT), 04(OPEN)
 |  |
 |  |--04:SCAN HDFS [tpcds_parquet.customer_address]
@@ -117,7 +117,7 @@ PLAN-ROOT SINK
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-7 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-7 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=4 row-size=8B cardinality=8.33K
+|  |     tuple-ids=4 row-size=8B cardinality=8.07K
 |  |     in pipelines: 04(GETNEXT)
 |  |
 |  08:HASH JOIN [INNER JOIN]
@@ -192,8 +192,8 @@ PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(cr_net_loss) DESC
-|  mem-estimate=35.19KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=8 row-size=92B cardinality=191
+|  mem-estimate=34.09KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=8 row-size=92B cardinality=185
 |  in pipelines: 14(GETNEXT)
 |
 F07:PLAN FRAGMENT [HASH(cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status)] hosts=1 instances=1
@@ -201,28 +201,28 @@ Per-Host Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-reservat
 14:SORT
 |  order by: sum(cr_net_loss) DESC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=8 row-size=92B cardinality=191
+|  tuple-ids=8 row-size=92B cardinality=185
 |  in pipelines: 14(GETNEXT), 22(OPEN)
 |
 22:AGGREGATE [FINALIZE]
 |  output: sum:merge(cr_net_loss)
 |  group by: cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7 row-size=127B cardinality=191
+|  tuple-ids=7 row-size=127B cardinality=185
 |  in pipelines: 22(GETNEXT), 01(OPEN)
 |
 21:EXCHANGE [HASH(cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status)]
-|  mem-estimate=48.09KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=7 row-size=127B cardinality=191
+|  mem-estimate=46.58KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=7 row-size=127B cardinality=185
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-Per-Host Resources: mem-estimate=115.10MB mem-reservation=12.81MB thread-reservation=2 runtime-filters-memory=3.00MB
+Per-Host Resources: mem-estimate=115.09MB mem-reservation=12.81MB thread-reservation=2 runtime-filters-memory=3.00MB
 13:AGGREGATE [STREAMING]
 |  output: sum(cr_net_loss)
 |  group by: cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7 row-size=127B cardinality=191
+|  tuple-ids=7 row-size=127B cardinality=185
 |  in pipelines: 01(GETNEXT)
 |
 12:HASH JOIN [INNER JOIN, BROADCAST]
@@ -230,7 +230,7 @@ Per-Host Resources: mem-estimate=115.10MB mem-reservation=12.81MB thread-reserva
 |  fk/pk conjuncts: cr_call_center_sk = cc_call_center_sk
 |  runtime filters: RF000[bloom] <- cc_call_center_sk, RF001[min_max] <- cc_call_center_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4,2,0 row-size=194B cardinality=191
+|  tuple-ids=1,5,3,6,4,2,0 row-size=194B cardinality=185
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--20:EXCHANGE [BROADCAST]
@@ -255,7 +255,7 @@ Per-Host Resources: mem-estimate=115.10MB mem-reservation=12.81MB thread-reserva
 |  fk/pk conjuncts: cr_returned_date_sk = d_date_sk
 |  runtime filters: RF002[bloom] <- d_date_sk, RF003[min_max] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4,2 row-size=114B cardinality=191
+|  tuple-ids=1,5,3,6,4,2 row-size=114B cardinality=185
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--19:EXCHANGE [BROADCAST]
@@ -283,12 +283,12 @@ Per-Host Resources: mem-estimate=115.10MB mem-reservation=12.81MB thread-reserva
 |  fk/pk conjuncts: none
 |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4 row-size=102B cardinality=3.61K
+|  tuple-ids=1,5,3,6,4 row-size=102B cardinality=3.50K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--18:EXCHANGE [BROADCAST]
-|  |  mem-estimate=260.77KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.89K
+|  |  mem-estimate=255.89KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.83K
 |  |  in pipelines: 05(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -298,12 +298,12 @@ Per-Host Resources: mem-estimate=115.10MB mem-reservation=12.81MB thread-reserva
 |  |  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=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.89K
+|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.83K
 |  |  in pipelines: 05(GETNEXT), 04(OPEN)
 |  |
 |  |--17:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=4 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=4 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 04(GETNEXT)
 |  |  |
 |  |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -318,7 +318,7 @@ Per-Host Resources: mem-estimate=115.10MB mem-reservation=12.81MB thread-reserva
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-7 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-7 AS DECIMAL(3,0))
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=4 row-size=8B cardinality=8.33K
+|  |     tuple-ids=4 row-size=8B cardinality=8.07K
 |  |     in pipelines: 04(GETNEXT)
 |  |
 |  08:HASH JOIN [INNER JOIN, BROADCAST]
@@ -407,8 +407,8 @@ PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(cr_net_loss) DESC
-|  mem-estimate=35.19KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=8 row-size=92B cardinality=191
+|  mem-estimate=34.09KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=8 row-size=92B cardinality=185
 |  in pipelines: 14(GETNEXT)
 |
 F07:PLAN FRAGMENT [HASH(cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status)] hosts=1 instances=1
@@ -416,19 +416,19 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese
 14:SORT
 |  order by: sum(cr_net_loss) DESC
 |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=8 row-size=92B cardinality=191
+|  tuple-ids=8 row-size=92B cardinality=185
 |  in pipelines: 14(GETNEXT), 22(OPEN)
 |
 22:AGGREGATE [FINALIZE]
 |  output: sum:merge(cr_net_loss)
 |  group by: cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7 row-size=127B cardinality=191
+|  tuple-ids=7 row-size=127B cardinality=185
 |  in pipelines: 22(GETNEXT), 01(OPEN)
 |
 21:EXCHANGE [HASH(cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status)]
-|  mem-estimate=48.09KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=7 row-size=127B cardinality=191
+|  mem-estimate=46.58KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=7 row-size=127B cardinality=185
 |  in pipelines: 01(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -438,7 +438,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser
 |  output: sum(cr_net_loss)
 |  group by: cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status
 |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=7 row-size=127B cardinality=191
+|  tuple-ids=7 row-size=127B cardinality=185
 |  in pipelines: 01(GETNEXT)
 |
 12:HASH JOIN [INNER JOIN, BROADCAST]
@@ -446,7 +446,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: cr_call_center_sk = cc_call_center_sk
 |  fk/pk conjuncts: cr_call_center_sk = cc_call_center_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4,2,0 row-size=194B cardinality=191
+|  tuple-ids=1,5,3,6,4,2,0 row-size=194B cardinality=185
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -479,7 +479,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: cr_returned_date_sk = d_date_sk
 |  fk/pk conjuncts: cr_returned_date_sk = d_date_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4,2 row-size=114B cardinality=191
+|  tuple-ids=1,5,3,6,4,2 row-size=114B cardinality=185
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -515,11 +515,11 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser
 |  hash predicates: cr_returning_customer_sk = c_customer_sk
 |  fk/pk conjuncts: none
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,5,3,6,4 row-size=102B cardinality=3.61K
+|  tuple-ids=1,5,3,6,4 row-size=102B cardinality=3.50K
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=5.13MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=5.12MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: c_customer_sk
@@ -527,8 +527,8 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  18:EXCHANGE [BROADCAST]
-|  |  mem-estimate=260.77KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.89K
+|  |  mem-estimate=255.89KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.83K
 |  |  in pipelines: 05(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -539,7 +539,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser
 |  |  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=64.00KB thread-reservation=0
-|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.89K
+|  |  tuple-ids=5,3,6,4 row-size=86B cardinality=1.83K
 |  |  in pipelines: 05(GETNEXT), 04(OPEN)
 |  |
 |  |--F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -551,8 +551,8 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  17:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=77.10KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=4 row-size=8B cardinality=8.33K
+|  |  |  mem-estimate=75.08KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=4 row-size=8B cardinality=8.07K
 |  |  |  in pipelines: 04(GETNEXT)
 |  |  |
 |  |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -567,7 +567,7 @@ Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reser
 |  |     parquet statistics predicates: ca_gmt_offset = CAST(-7 AS DECIMAL(3,0))
 |  |     parquet dictionary predicates: ca_gmt_offset = CAST(-7 AS DECIMAL(3,0))
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=4 row-size=8B cardinality=8.33K
+|  |     tuple-ids=4 row-size=8B cardinality=8.07K
 |  |     in pipelines: 04(GETNEXT)
 |  |
 |  08:HASH JOIN [INNER JOIN, BROADCAST]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
index 334b15f..130f80a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
@@ -50,13 +50,13 @@ PLAN-ROOT SINK
 |  output: sum(ws_ext_ship_cost), sum(ws_net_profit)
 |  group by: ws_order_number
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=40B cardinality=3.25K
+|  tuple-ids=8 row-size=40B cardinality=3.15K
 |  in pipelines: 11(GETNEXT), 05(OPEN)
 |
 10:HASH JOIN [RIGHT ANTI JOIN]
 |  hash predicates: wr1.wr_order_number = ws1.ws_order_number
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  in pipelines: 05(GETNEXT), 04(OPEN)
 |
 |--09:HASH JOIN [RIGHT SEMI JOIN]
@@ -64,7 +64,7 @@ PLAN-ROOT SINK
 |  |  other join predicates: ws1.ws_warehouse_sk != ws2.ws_warehouse_sk
 |  |  runtime filters: RF000[bloom] <- ws1.ws_order_number, RF001[min_max] <- ws1.ws_order_number
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  |  in pipelines: 04(GETNEXT), 00(OPEN)
 |  |
 |  |--08:HASH JOIN [INNER JOIN]
@@ -72,7 +72,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws1.ws_ship_date_sk = d_date_sk
 |  |  |  runtime filters: RF002[bloom] <- d_date_sk, RF003[min_max] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  |  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |  |
 |  |  |--01:SCAN HDFS [tpcds_parquet.date_dim]
@@ -93,7 +93,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws1.ws_web_site_sk = web_site_sk
 |  |  |  runtime filters: RF004[bloom] <- web_site_sk, RF005[min_max] <- web_site_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2,3 row-size=70B cardinality=3.25K
+|  |  |  tuple-ids=0,2,3 row-size=70B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |  |
 |  |  |--03:SCAN HDFS [tpcds_parquet.web_site]
@@ -114,7 +114,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws1.ws_ship_addr_sk = ca_address_sk
 |  |  |  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2 row-size=50B cardinality=19.52K
+|  |  |  tuple-ids=0,2 row-size=50B cardinality=18.92K
 |  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |  |
 |  |  |--02:SCAN HDFS [tpcds_parquet.customer_address]
@@ -127,7 +127,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ca_state = 'IL'
 |  |  |     parquet dictionary predicates: ca_state = 'IL'
 |  |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |     tuple-ids=2 row-size=18B cardinality=980
+|  |  |     tuple-ids=2 row-size=18B cardinality=950
 |  |  |     in pipelines: 02(GETNEXT)
 |  |  |
 |  |  00:SCAN HDFS [tpcds_parquet.web_sales ws1]
@@ -199,13 +199,13 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati
 |  output: sum(ws_ext_ship_cost), sum(ws_net_profit)
 |  group by: ws_order_number
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=40B cardinality=3.25K
+|  tuple-ids=8 row-size=40B cardinality=3.15K
 |  in pipelines: 11(GETNEXT), 04(OPEN)
 |
 10:HASH JOIN [LEFT ANTI JOIN, PARTITIONED]
 |  hash predicates: ws1.ws_order_number = wr1.wr_order_number
 |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  in pipelines: 04(GETNEXT), 05(OPEN)
 |
 |--20:EXCHANGE [HASH(wr1.wr_order_number)]
@@ -230,22 +230,22 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati
 |  other join predicates: ws1.ws_warehouse_sk != ws2.ws_warehouse_sk
 |  runtime filters: RF000[bloom] <- ws1.ws_order_number, RF001[min_max] <- ws1.ws_order_number
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 |--19:EXCHANGE [HASH(ws1.ws_order_number)]
-|  |  mem-estimate=376.37KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  |  mem-estimate=371.73KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F05:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=3.42MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=3.41MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  08:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash predicates: ws1.ws_ship_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ws1.ws_ship_date_sk = d_date_sk
 |  |  runtime filters: RF002[bloom] <- d_date_sk, RF003[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |
 |  |--17:EXCHANGE [HASH(d_date_sk)]
@@ -269,8 +269,8 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  16:EXCHANGE [HASH(ws1.ws_ship_date_sk)]
-|  |  mem-estimate=275.07KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,2,3 row-size=70B cardinality=3.25K
+|  |  mem-estimate=271.68KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,3 row-size=70B cardinality=3.15K
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -280,7 +280,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati
 |  |  fk/pk conjuncts: ws1.ws_web_site_sk = web_site_sk
 |  |  runtime filters: RF004[bloom] <- web_site_sk, RF005[min_max] <- web_site_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,3 row-size=70B cardinality=3.25K
+|  |  tuple-ids=0,2,3 row-size=70B cardinality=3.15K
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--15:EXCHANGE [BROADCAST]
@@ -308,12 +308,12 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati
 |  |  fk/pk conjuncts: ws1.ws_ship_addr_sk = ca_address_sk
 |  |  runtime filters: RF006[bloom] <- ca_address_sk, RF007[min_max] <- ca_address_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2 row-size=50B cardinality=19.52K
+|  |  tuple-ids=0,2 row-size=50B cardinality=18.92K
 |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |
 |  |--14:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=38.28KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=18B cardinality=980
+|  |  |  mem-estimate=37.11KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=2 row-size=18B cardinality=950
 |  |  |  in pipelines: 02(GETNEXT)
 |  |  |
 |  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -328,7 +328,7 @@ Per-Host Resources: mem-estimate=19.96MB mem-reservation=7.75MB thread-reservati
 |  |     parquet statistics predicates: ca_state = 'IL'
 |  |     parquet dictionary predicates: ca_state = 'IL'
 |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=18B cardinality=980
+|  |     tuple-ids=2 row-size=18B cardinality=950
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  00:SCAN HDFS [tpcds_parquet.web_sales ws1, RANDOM]
@@ -397,14 +397,14 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser
 |  output: sum(ws_ext_ship_cost), sum(ws_net_profit)
 |  group by: ws_order_number
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=8 row-size=40B cardinality=3.25K
+|  tuple-ids=8 row-size=40B cardinality=3.15K
 |  in pipelines: 11(GETNEXT), 04(OPEN)
 |
 10:HASH JOIN [LEFT ANTI JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: ws1.ws_order_number = wr1.wr_order_number
 |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  in pipelines: 04(GETNEXT), 05(OPEN)
 |
 |--F09:PLAN FRAGMENT [HASH(ws1.ws_order_number)] hosts=2 instances=2
@@ -436,11 +436,11 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser
 |  hash predicates: ws2.ws_order_number = ws1.ws_order_number
 |  other join predicates: ws1.ws_warehouse_sk != ws2.ws_warehouse_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 |--F10:PLAN FRAGMENT [HASH(ws1.ws_order_number)] hosts=2 instances=2
-|  |  Per-Instance Resources: mem-estimate=3.31MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=3.30MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: ws1.ws_order_number
@@ -448,18 +448,18 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  19:EXCHANGE [HASH(ws1.ws_order_number)]
-|  |  mem-estimate=376.37KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  |  mem-estimate=371.73KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F05:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2
-|  Per-Instance Resources: mem-estimate=275.07KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=271.68KB mem-reservation=0B thread-reservation=1
 |  08:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=02
 |  |  hash predicates: ws1.ws_ship_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ws1.ws_ship_date_sk = d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.25K
+|  |  tuple-ids=0,2,3,1 row-size=96B cardinality=3.15K
 |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |
 |  |--F11:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2
@@ -491,8 +491,8 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  16:EXCHANGE [HASH(ws1.ws_ship_date_sk)]
-|  |  mem-estimate=275.07KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,2,3 row-size=70B cardinality=3.25K
+|  |  mem-estimate=271.68KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,3 row-size=70B cardinality=3.15K
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -503,7 +503,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser
 |  |  hash predicates: ws1.ws_web_site_sk = web_site_sk
 |  |  fk/pk conjuncts: ws1.ws_web_site_sk = web_site_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,3 row-size=70B cardinality=3.25K
+|  |  tuple-ids=0,2,3 row-size=70B cardinality=3.15K
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -539,7 +539,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser
 |  |  hash predicates: ws1.ws_ship_addr_sk = ca_address_sk
 |  |  fk/pk conjuncts: ws1.ws_ship_addr_sk = ca_address_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2 row-size=50B cardinality=19.52K
+|  |  tuple-ids=0,2 row-size=50B cardinality=18.92K
 |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |
 |  |--F13:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -551,8 +551,8 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser
 |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  14:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=38.28KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=2 row-size=18B cardinality=980
+|  |  |  mem-estimate=37.11KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=2 row-size=18B cardinality=950
 |  |  |  in pipelines: 02(GETNEXT)
 |  |  |
 |  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -567,7 +567,7 @@ Per-Instance Resources: mem-estimate=14.15MB mem-reservation=1.94MB thread-reser
 |  |     parquet statistics predicates: ca_state = 'IL'
 |  |     parquet dictionary predicates: ca_state = 'IL'
 |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |     tuple-ids=2 row-size=18B cardinality=980
+|  |     tuple-ids=2 row-size=18B cardinality=950
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  00:SCAN HDFS [tpcds_parquet.web_sales ws1, RANDOM]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
index 0e71b50..6bec622 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
@@ -53,21 +53,21 @@ PLAN-ROOT SINK
 |  output: sum(ws_ext_ship_cost), sum(ws_net_profit)
 |  group by: ws_order_number
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=13 row-size=40B cardinality=3.25K
+|  tuple-ids=13 row-size=40B cardinality=3.15K
 |  in pipelines: 19(GETNEXT), 17(OPEN)
 |
 18:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: tpcds_parquet.web_returns.wr_order_number = ws1.ws_order_number
 |  runtime filters: RF000[bloom] <- ws1.ws_order_number, RF001[min_max] <- ws1.ws_order_number
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  in pipelines: 17(GETNEXT), 15(OPEN)
 |
 |--16:HASH JOIN [RIGHT SEMI JOIN]
 |  |  hash predicates: ws1.ws_order_number = ws1.ws_order_number
 |  |  runtime filters: RF006[bloom] <- ws1.ws_order_number, RF007[min_max] <- ws1.ws_order_number
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  |  in pipelines: 15(GETNEXT), 00(OPEN)
 |  |
 |  |--14:HASH JOIN [INNER JOIN]
@@ -75,7 +75,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws1.ws_ship_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=0,2,3,1 row-size=92B cardinality=3.25K
+|  |  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |  |
 |  |  |--01:SCAN HDFS [tpcds_parquet.date_dim]
@@ -95,7 +95,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws1.ws_web_site_sk = web_site_sk
 |  |  |  runtime filters: RF012[bloom] <- web_site_sk, RF013[min_max] <- web_site_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.25K
+|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |  |
 |  |  |--03:SCAN HDFS [tpcds_parquet.web_site]
@@ -116,7 +116,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: ws1.ws_ship_addr_sk = ca_address_sk
 |  |  |  runtime filters: RF014[bloom] <- ca_address_sk, RF015[min_max] <- ca_address_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2 row-size=46B cardinality=19.52K
+|  |  |  tuple-ids=0,2 row-size=46B cardinality=18.92K
 |  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |  |
 |  |  |--02:SCAN HDFS [tpcds_parquet.customer_address]
@@ -129,7 +129,7 @@ PLAN-ROOT SINK
 |  |  |     parquet statistics predicates: ca_state = 'IL'
 |  |  |     parquet dictionary predicates: ca_state = 'IL'
 |  |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |     tuple-ids=2 row-size=18B cardinality=980
+|  |  |     tuple-ids=2 row-size=18B cardinality=950
 |  |  |     in pipelines: 02(GETNEXT)
 |  |  |
 |  |  00:SCAN HDFS [tpcds_parquet.web_sales ws1]
@@ -273,26 +273,26 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva
 |  output: sum(ws_ext_ship_cost), sum(ws_net_profit)
 |  group by: ws_order_number
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=13 row-size=40B cardinality=3.25K
+|  tuple-ids=13 row-size=40B cardinality=3.15K
 |  in pipelines: 19(GETNEXT), 17(OPEN)
 |
 18:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  hash predicates: tpcds_parquet.web_returns.wr_order_number = ws1.ws_order_number
 |  runtime filters: RF000[bloom] <- ws1.ws_order_number, RF001[min_max] <- ws1.ws_order_number
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  in pipelines: 17(GETNEXT), 15(OPEN)
 |
 |--16:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  |  hash predicates: ws1.ws_order_number = ws1.ws_order_number
 |  |  runtime filters: RF006[bloom] <- ws1.ws_order_number, RF007[min_max] <- ws1.ws_order_number
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  |  in pipelines: 15(GETNEXT), 00(OPEN)
 |  |
 |  |--31:EXCHANGE [HASH(ws1.ws_order_number)]
-|  |  |  mem-estimate=362.01KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  |  |  mem-estimate=357.57KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT)
 |  |  |
 |  |  F11:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2
@@ -302,7 +302,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva
 |  |  |  fk/pk conjuncts: ws1.ws_ship_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=0,2,3,1 row-size=92B cardinality=3.25K
+|  |  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |  |
 |  |  |--30:EXCHANGE [HASH(d_date_sk)]
@@ -325,8 +325,8 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva
 |  |  |     in pipelines: 01(GETNEXT)
 |  |  |
 |  |  29:EXCHANGE [HASH(ws1.ws_ship_date_sk)]
-|  |  |  mem-estimate=260.71KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.25K
+|  |  |  mem-estimate=257.52KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT)
 |  |  |
 |  |  F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -336,7 +336,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva
 |  |  |  fk/pk conjuncts: ws1.ws_web_site_sk = web_site_sk
 |  |  |  runtime filters: RF012[bloom] <- web_site_sk, RF013[min_max] <- web_site_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.25K
+|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |  |
 |  |  |--28:EXCHANGE [BROADCAST]
@@ -364,12 +364,12 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva
 |  |  |  fk/pk conjuncts: ws1.ws_ship_addr_sk = ca_address_sk
 |  |  |  runtime filters: RF014[bloom] <- ca_address_sk, RF015[min_max] <- ca_address_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2 row-size=46B cardinality=19.52K
+|  |  |  tuple-ids=0,2 row-size=46B cardinality=18.92K
 |  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |  |
 |  |  |--27:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=38.28KB mem-reservation=0B thread-reservation=0
-|  |  |  |  tuple-ids=2 row-size=18B cardinality=980
+|  |  |  |  mem-estimate=37.11KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=2 row-size=18B cardinality=950
 |  |  |  |  in pipelines: 02(GETNEXT)
 |  |  |  |
 |  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -384,7 +384,7 @@ Per-Host Resources: mem-estimate=106.46MB mem-reservation=82.62MB thread-reserva
 |  |  |     parquet statistics predicates: ca_state = 'IL'
 |  |  |     parquet dictionary predicates: ca_state = 'IL'
 |  |  |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|  |  |     tuple-ids=2 row-size=18B cardinality=980
+|  |  |     tuple-ids=2 row-size=18B cardinality=950
 |  |  |     in pipelines: 02(GETNEXT)
 |  |  |
 |  |  00:SCAN HDFS [tpcds_parquet.web_sales ws1, RANDOM]
@@ -563,14 +563,14 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser
 |  output: sum(ws_ext_ship_cost), sum(ws_net_profit)
 |  group by: ws_order_number
 |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=13 row-size=40B cardinality=3.25K
+|  tuple-ids=13 row-size=40B cardinality=3.15K
 |  in pipelines: 19(GETNEXT), 17(OPEN)
 |
 18:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: tpcds_parquet.web_returns.wr_order_number = ws1.ws_order_number
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  in pipelines: 17(GETNEXT), 15(OPEN)
 |
 |--F13:PLAN FRAGMENT [HASH(ws1.ws_order_number)] hosts=2 instances=2
@@ -585,7 +585,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser
 |  |  hash-table-id=01
 |  |  hash predicates: ws1.ws_order_number = ws1.ws_order_number
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  |  in pipelines: 15(GETNEXT), 00(OPEN)
 |  |
 |  |--F14:PLAN FRAGMENT [HASH(ws1.ws_order_number)] hosts=2 instances=2
@@ -597,18 +597,18 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  |
 |  |  31:EXCHANGE [HASH(ws1.ws_order_number)]
-|  |  |  mem-estimate=362.01KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  |  |  mem-estimate=357.57KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT)
 |  |  |
 |  |  F11:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2
-|  |  Per-Instance Resources: mem-estimate=260.71KB mem-reservation=0B thread-reservation=1
+|  |  Per-Instance Resources: mem-estimate=257.52KB mem-reservation=0B thread-reservation=1
 |  |  14:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  |  hash-table-id=02
 |  |  |  hash predicates: ws1.ws_ship_date_sk = d_date_sk
 |  |  |  fk/pk conjuncts: ws1.ws_ship_date_sk = d_date_sk
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.25K
+|  |  |  tuple-ids=0,2,3,1 row-size=92B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |  |
 |  |  |--F15:PLAN FRAGMENT [HASH(ws1.ws_ship_date_sk)] hosts=2 instances=2
@@ -639,8 +639,8 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser
 |  |  |     in pipelines: 01(GETNEXT)
 |  |  |
 |  |  29:EXCHANGE [HASH(ws1.ws_ship_date_sk)]
-|  |  |  mem-estimate=260.71KB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.25K
+|  |  |  mem-estimate=257.52KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT)
 |  |  |
 |  |  F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -651,7 +651,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser
 |  |  |  hash predicates: ws1.ws_web_site_sk = web_site_sk
 |  |  |  fk/pk conjuncts: ws1.ws_web_site_sk = web_site_sk
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.25K
+|  |  |  tuple-ids=0,2,3 row-size=66B cardinality=3.15K
 |  |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |  |
 |  |  |--F16:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -687,7 +687,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser
 |  |  |  hash predicates: ws1.ws_ship_addr_sk = ca_address_sk
 |  |  |  fk/pk conjuncts: ws1.ws_ship_addr_sk = ca_address_sk
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=0,2 row-size=46B cardinality=19.52K
+|  |  |  tuple-ids=0,2 row-size=46B cardinality=18.92K
 |  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |  |
 |  |  |--F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -699,8 +699,8 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser
 |  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |  |  |
 |  |  |  27:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=38.28KB mem-reservation=0B thread-reservation=0
-|  |  |  |  tuple-ids=2 row-size=18B cardinality=980
+|  |  |  |  mem-estimate=37.11KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=2 row-size=18B cardinality=950
 |  |  |  |  in pipelines: 02(GETNEXT)
 |  |  |  |
 |  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
@@ -715,7 +715,7 @@ Per-Instance Resources: mem-estimate=20.00MB mem-reservation=3.88MB thread-reser
 |  |  |     parquet statistics predicates: ca_state = 'IL'
 |  |  |     parquet dictionary predicates: ca_state = 'IL'
 |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|  |  |     tuple-ids=2 row-size=18B cardinality=980
+|  |  |     tuple-ids=2 row-size=18B cardinality=950
 |  |  |     in pipelines: 02(GETNEXT)
 |  |  |
 |  |  00:SCAN HDFS [tpcds_parquet.web_sales ws1, RANDOM]