You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by cs...@apache.org on 2023/12/19 14:07:28 UTC

(impala) 03/03: IMPALA-12018: Consider runtime filter for cardinality reduction

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

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

commit b37a35aa139ff61a1f93a54a9902ea76a86cbe1d
Author: Riza Suminto <ri...@cloudera.com>
AuthorDate: Wed Aug 30 14:34:53 2023 -0700

    IMPALA-12018: Consider runtime filter for cardinality reduction
    
    Currently, Impala creates a plan first and looks for runtime filters
    based on the complete plan. This means the cardinality estimate in the
    query plan does not incorporate runtime filter selectivity. Actual scan
    cardinality from runtime execution is often much lower than the
    cardinality estimate due to the existence of runtime filters.
    
    This patch applies runtime filter selectivity to lower cardinality
    estimates of scan nodes and certain join nodes above them after runtime
    filter generation and before resource requirement computation. The
    algorithm selects a contiguous probe pipeline consisting of a scan node,
    exchanges, and reducing join nodes. Depending on whether the join node
    produces a runtime filter and the type of that runtime filter, it then
    applies the runtime filter selectivity to the scan node to reduce its
    cardinality and input cardinality estimate. The runtime filter
    selectivity is calculated with the simplest join cardinality
    formula (JoinNode.computeGenericJoinCardinality()).
    
    The reduced cardinality is stored in new fields 'filteredCardinality_'
    and 'filteredInputCardinality_', separate from existing fields
    'cardinality_' and 'inputCardinality_'. Future work should merge the new
    cardinality fields with the old cardinality fields after we can validate
    that the cardinality reduction does not regress memory estimation.
    
    While this cardinality reduction is present in all execution
    modes (MT_DOP=0, MT_DOP>0, and COMPUTE_PROCESSING_COST=1), cost-based
    planning mode will be the primary beneficiary of this patch. It can lead
    toward ProcessingCost reduction, lower scan fragment parallelism, lower
    CpuAsk, and increase the chance of query assignment to the smaller
    executor group set. Other execution modes will see no change in their
    execution parallelism or memory estimates.
    
    This patch also adds development query option named
    RUNTIME_FILTER_CARDINALITY_REDUCTION_SCALE, a range of [0.0..1.0] that
    controls the cardinality reduction scale from runtime filter analysis to
    help with benchmarking and disabling cardinality reduction if needed (by
    setting to 0.0). Default to 1.0.
    
    Testing:
    - Add fe test testRuntimeFilterCardinalityReduction and
      testRuntimeFilterCardinalityReductionOnKudu
    - Pass test_executor_groups.py.
    - Pass PlannerTest#testProcessingCost.
    - Add be test QueryOptions.SetFractionalOptions
    - Ran full TPC-DS 3TB benchmark and see no regression due to
      query plan change.
    - Pass core tests.
    
    Change-Id: I033789c9b63a8188484e3afde8e646563918b3e1
    Reviewed-on: http://gerrit.cloudera.org:8080/20498
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/query-options-test.cc               |  59 +-
 be/src/service/query-options.cc                    |   7 +
 be/src/service/query-options.h                     |   4 +-
 common/thrift/ImpalaService.thrift                 |  12 +-
 common/thrift/Query.thrift                         |   3 +
 .../org/apache/impala/planner/AggregationNode.java |   2 +
 .../org/apache/impala/planner/DataStreamSink.java  |   5 +-
 .../org/apache/impala/planner/ExchangeNode.java    |   9 +
 .../org/apache/impala/planner/HashJoinNode.java    |   4 +-
 .../org/apache/impala/planner/HdfsScanNode.java    |  23 +-
 .../apache/impala/planner/IcebergDeleteNode.java   |   2 +-
 .../java/org/apache/impala/planner/JoinNode.java   |  48 +-
 .../apache/impala/planner/NestedLoopJoinNode.java  |  21 +-
 .../java/org/apache/impala/planner/PlanNode.java   |  47 +-
 .../java/org/apache/impala/planner/Planner.java    |  18 +
 .../impala/planner/RuntimeFilterGenerator.java     | 104 ++-
 .../java/org/apache/impala/planner/ScanNode.java   | 197 ++++-
 .../java/org/apache/impala/service/Frontend.java   |   1 +
 .../org/apache/impala/planner/PlannerTest.java     |  19 +
 .../queries/PlannerTest/aggregation.test           |   2 +-
 .../PlannerTest/bloom-filter-assignment.test       |  16 +-
 .../queries/PlannerTest/convert-to-cnf.test        |  20 +-
 .../PlannerTest/explain-verbose-mt_dop.test        |   8 +-
 .../queries/PlannerTest/fk-pk-join-detection.test  |   6 +-
 .../queries/PlannerTest/kudu.test                  |   2 +-
 .../queries/PlannerTest/max-row-size.test          |   4 +-
 .../PlannerTest/min-max-runtime-filters.test       |   8 +-
 .../queries/PlannerTest/nested-collections.test    |   4 +-
 .../queries/PlannerTest/outer-to-inner-joins.test  |   2 +-
 .../queries/PlannerTest/predicate-propagation.test |  16 +-
 .../queries/PlannerTest/resource-requirements.test |   6 +-
 ...ntime-filter-cardinality-reduction-on-kudu.test | 113 +++
 .../runtime-filter-cardinality-reduction.test      | 792 +++++++++++++++++++++
 .../PlannerTest/spillable-buffer-sizing.test       |   8 +-
 .../queries/PlannerTest/tpcds-dist-method.test     |  12 +-
 .../queries/PlannerTest/tpcds-processing-cost.test | 606 ++++++++--------
 .../queries/PlannerTest/tpcds/tpcds-q01.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q03.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q04.test       |  48 +-
 .../queries/PlannerTest/tpcds/tpcds-q07.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q08.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q10a.test      |  30 +-
 .../queries/PlannerTest/tpcds/tpcds-q11.test       |  48 +-
 .../queries/PlannerTest/tpcds/tpcds-q12.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q13.test       |   2 +-
 .../queries/PlannerTest/tpcds/tpcds-q14a.test      |  24 +-
 .../queries/PlannerTest/tpcds/tpcds-q15.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q16.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q17.test       |  32 +-
 .../queries/PlannerTest/tpcds/tpcds-q18.test       |  10 +-
 .../queries/PlannerTest/tpcds/tpcds-q19.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q20.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q21.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q23a.test      |  60 +-
 .../queries/PlannerTest/tpcds/tpcds-q23b.test      |  62 +-
 .../queries/PlannerTest/tpcds/tpcds-q24a.test      |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q24b.test      |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q25.test       |  32 +-
 .../queries/PlannerTest/tpcds/tpcds-q26.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q27.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q29.test       |  32 +-
 .../queries/PlannerTest/tpcds/tpcds-q30.test       |  30 +-
 .../queries/PlannerTest/tpcds/tpcds-q31.test       |  54 +-
 .../queries/PlannerTest/tpcds/tpcds-q32.test       |   8 +-
 .../queries/PlannerTest/tpcds/tpcds-q33.test       |  36 +-
 .../queries/PlannerTest/tpcds/tpcds-q34.test       |  20 +-
 .../queries/PlannerTest/tpcds/tpcds-q35a.test      |  30 +-
 .../queries/PlannerTest/tpcds/tpcds-q36.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q37.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q39a.test      |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q39b.test      |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q40.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q42.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q43.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q44.test       |   4 +-
 .../queries/PlannerTest/tpcds/tpcds-q45.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q46.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q47.test       |  36 +-
 .../queries/PlannerTest/tpcds/tpcds-q48.test       |   2 +-
 .../queries/PlannerTest/tpcds/tpcds-q49.test       |  42 +-
 .../queries/PlannerTest/tpcds/tpcds-q52.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q53.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q54.test       |  10 +-
 .../queries/PlannerTest/tpcds/tpcds-q55.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q56.test       |  44 +-
 .../queries/PlannerTest/tpcds/tpcds-q57.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q58.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q60.test       |  44 +-
 .../queries/PlannerTest/tpcds/tpcds-q61.test       |  36 +-
 .../queries/PlannerTest/tpcds/tpcds-q63.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q64.test       |  56 +-
 .../queries/PlannerTest/tpcds/tpcds-q66.test       |  36 +-
 .../queries/PlannerTest/tpcds/tpcds-q68.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q69.test       |  30 +-
 .../queries/PlannerTest/tpcds/tpcds-q71.test       |  26 +-
 .../queries/PlannerTest/tpcds/tpcds-q73.test       |   8 +-
 .../queries/PlannerTest/tpcds/tpcds-q74.test       |  36 +-
 .../queries/PlannerTest/tpcds/tpcds-q75.test       |  44 +-
 .../queries/PlannerTest/tpcds/tpcds-q76.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q78.test       |  42 +-
 .../queries/PlannerTest/tpcds/tpcds-q79.test       |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q80.test       |  36 +-
 .../queries/PlannerTest/tpcds/tpcds-q81.test       |  30 +-
 .../queries/PlannerTest/tpcds/tpcds-q82.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q83.test       |  36 +-
 .../queries/PlannerTest/tpcds/tpcds-q84.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q85.test       |  10 +-
 .../queries/PlannerTest/tpcds/tpcds-q88.test       |  48 +-
 .../queries/PlannerTest/tpcds/tpcds-q89.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q90.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q91.test       |  12 +-
 .../queries/PlannerTest/tpcds/tpcds-q92.test       |   8 +-
 .../queries/PlannerTest/tpcds/tpcds-q94.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q95.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q96.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q98.test       |   6 +-
 .../queries/PlannerTest/tpch-kudu.test             |  50 +-
 117 files changed, 2648 insertions(+), 1294 deletions(-)

diff --git a/be/src/service/query-options-test.cc b/be/src/service/query-options-test.cc
index 0e57c33e0..909c2f10f 100644
--- a/be/src/service/query-options-test.cc
+++ b/be/src/service/query-options-test.cc
@@ -310,6 +310,45 @@ TEST(QueryOptions, SetBigIntOptions) {
   }
 }
 
+// Test double options with expected value between 0 and 1 (inclusive or exclusive).
+TEST(QueryOptions, SetFractionalOptions) {
+  TQueryOptions options;
+  // List of pairs of Key and boolean flag on whether the option is inclusive of 0 and 1.
+  pair<OptionDef<double>, bool> case_set[]{
+      {MAKE_OPTIONDEF(resource_trace_ratio), true},
+      {MAKE_OPTIONDEF(runtime_filter_error_rate), false},
+      {MAKE_OPTIONDEF(minmax_filter_threshold), true},
+      {MAKE_OPTIONDEF(join_selectivity_correlation_factor), true},
+      {MAKE_OPTIONDEF(agg_mem_correlation_factor), true},
+      {MAKE_OPTIONDEF(runtime_filter_cardinality_reduction_scale), true},
+  };
+  for (const auto& test_case : case_set) {
+    const OptionDef<double>& option_def = test_case.first;
+    const bool& is_inclusive = test_case.second;
+    auto TestOk = MakeTestOkFn(options, option_def);
+    auto TestError = MakeTestErrFn(options, option_def);
+    TestOk("0.5", 0.5);
+    TestOk("0.01", 0.01);
+    TestOk("0.001", 0.001);
+    TestOk("0.0001", 0.0001);
+    TestOk("0.0000000001", 0.0000000001);
+    TestOk("0.999999999", 0.999999999);
+    TestOk(" 0.9", 0.9);
+    if (is_inclusive) {
+      TestOk("1", 1.0);
+      TestOk("0", 0.0);
+    } else {
+      TestError("1");
+      TestError("0");
+    }
+    // Out of range values
+    TestError("-1");
+    TestError("-0.1");
+    TestError("1.1");
+    TestError("Not a number!");
+  }
+}
+
 // Test options with non regular validation rule
 TEST(QueryOptions, SetSpecialOptions) {
   // REPLICA_PREFERENCE has unsettable enum values: cache_rack(1) & disk_rack(3)
@@ -387,26 +426,6 @@ TEST(QueryOptions, SetSpecialOptions) {
     TestError("8191"); // default value of FLAGS_min_buffer_size is 8KB
     TestOk("64KB", 64 * 1024);
   }
-  {
-    // RUNTIME_FILTER_ERROR_RATE is a double in range (0.0, 1.0)
-    OptionDef<double> key_def = MAKE_OPTIONDEF(runtime_filter_error_rate);
-    auto TestOk = MakeTestOkFn(options, key_def);
-    auto TestError = MakeTestErrFn(options, key_def);
-    TestOk("0.5", 0.5);
-    TestOk("0.01", 0.01);
-    TestOk("0.001", 0.001);
-    TestOk("0.0001", 0.0001);
-    TestOk("0.0000000001", 0.0000000001);
-    TestOk("0.999999999", 0.999999999);
-    TestOk(" 0.9", 0.9);
-    // Out of range values
-    TestError("1");
-    TestError("0");
-    TestError("-1");
-    TestError("-0.1");
-    TestError("1.1");
-    TestError("Not a number!");
-  }
 }
 
 TEST(QueryOptions, ParseQueryOptions) {
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index 9d743ba9e..ee52b08a5 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -1181,6 +1181,13 @@ Status impala::SetQueryOption(const string& key, const string& value,
         query_options->__set_kudu_table_reserve_seconds(int32_t_val);
         break;
       }
+      case TImpalaQueryOptions::RUNTIME_FILTER_CARDINALITY_REDUCTION_SCALE: {
+        double double_val = 0.0;
+        RETURN_IF_ERROR(QueryOptionParser::ParseAndCheckInclusiveRange<double>(
+            option, value, 0.0, 1.0, &double_val));
+        query_options->__set_runtime_filter_cardinality_reduction_scale(double_val);
+        break;
+      }
       default:
         if (IsRemovedQueryOption(key)) {
           LOG(WARNING) << "Ignoring attempt to set removed query option '" << key << "'";
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index 3406d4fc9..8c2bff23f 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -50,7 +50,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
 // time we add or remove a query option to/from the enum TImpalaQueryOptions.
 #define QUERY_OPTS_TABLE                                                                 \
   DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),                                 \
-      TImpalaQueryOptions::DISABLE_KUDU_LOCAL_TIMESTAMP_BLOOM_FILTER + 1);               \
+      TImpalaQueryOptions::RUNTIME_FILTER_CARDINALITY_REDUCTION_SCALE + 1);              \
   REMOVED_QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED) \
   QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR, TQueryOptionLevel::REGULAR)               \
   REMOVED_QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)             \
@@ -317,6 +317,8 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
       CONVERT_KUDU_UTC_TIMESTAMPS, TQueryOptionLevel::ADVANCED)                          \
   QUERY_OPT_FN(disable_kudu_local_timestamp_bloom_filter,                                \
       DISABLE_KUDU_LOCAL_TIMESTAMP_BLOOM_FILTER, TQueryOptionLevel::ADVANCED)            \
+  QUERY_OPT_FN(runtime_filter_cardinality_reduction_scale,                               \
+      RUNTIME_FILTER_CARDINALITY_REDUCTION_SCALE, TQueryOptionLevel::DEVELOPMENT)        \
   ;
 
 /// Enforce practical limits on some query options to avoid undesired query state.
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index ff14d2a09..ed8e07bf9 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -834,7 +834,7 @@ enum TImpalaQueryOptions {
   LARGE_AGG_MEM_THRESHOLD = 162
 
   // Correlation factor that will be used to calculate a lower memory estimation of
-  // aggregation node when the default memory estimation exceed
+  // aggregation node when the default memory estimation exceeds
   // LARGE_AGG_MEM_THRESHOLD. The reduction is achieved by calculating a memScale
   // multiplier (a fraction between 0.0 and 1.0). Given N as number of non-literal
   // grouping expressions:
@@ -846,7 +846,7 @@ enum TImpalaQueryOptions {
   // low value means there is low correlation between them. High correlation means
   // aggregation node can be scheduled with lower memory estimation (lower memScale).
   // Setting value 1.0 will result in an equal memory estimate as the default estimation
-  // (no change). Default to 0.5.
+  // (no change). Defaults to 0.5.
   AGG_MEM_CORRELATION_FACTOR = 163
 
   // A per coordinator approximate limit on the memory consumption
@@ -892,6 +892,14 @@ enum TImpalaQueryOptions {
   // For those regions that do not observe DST, could set this flag to false
   // to re-enable kudu local timestamp bloom filter.
   DISABLE_KUDU_LOCAL_TIMESTAMP_BLOOM_FILTER = 170
+
+  // A range of [0.0..1.0] that controls the cardinality reduction scale from runtime
+  // filter analysis. This is a linear scale with 0.0 meaning no cardinality estimate
+  // reduction should be applied and 1.0 meaning maximum cardinality estimate reduction
+  // should be applied. For example, if a table has 1M rows and runtime filters are
+  // estimated to reduce cardinality to 500K, setting value 0.25 will result in an 875K
+  // cardinality estimate. Default to 1.0.
+  RUNTIME_FILTER_CARDINALITY_REDUCTION_SCALE = 171
 }
 
 // The summary of a DML statement.
diff --git a/common/thrift/Query.thrift b/common/thrift/Query.thrift
index 2ed9134db..bbfbbe7bd 100644
--- a/common/thrift/Query.thrift
+++ b/common/thrift/Query.thrift
@@ -687,6 +687,9 @@ struct TQueryOptions {
 
   // See comment in ImpalaService.thrift
   171: optional bool disable_kudu_local_timestamp_bloom_filter = true;
+
+  // See comment in ImpalaService.thrift
+  172: optional double runtime_filter_cardinality_reduction_scale = 1.0
 }
 
 // Impala currently has three types of sessions: Beeswax, HiveServer2 and external
diff --git a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
index a32845a17..ef382ddc6 100644
--- a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
@@ -570,6 +570,8 @@ public class AggregationNode extends PlanNode {
     processingCost_ = ProcessingCost.zero();
     AggregationNode prevAgg = getPrevAggInputNode();
     for (AggregateInfo aggInfo : aggInfos_) {
+      // TODO: Cost should be much lower for NonGroupingAggregator
+      // (aggInfo.getGroupingExprs() is empty).
       ProcessingCost aggCost = aggInfo.computeProcessingCost(
           getDisplayLabel(), getAggClassNumGroup(prevAgg, aggInfo));
       processingCost_ = ProcessingCost.sumCost(processingCost_, aggCost);
diff --git a/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java b/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
index fcf1f266e..6052226b6 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
@@ -73,7 +73,7 @@ public class DataStreamSink extends DataSink {
         outputPartition_.isPartitioned() ? exchNode_.getFragment().getNumInstances() : 1;
     long rowBatchSize = PlanNode.getRowBatchSize(queryOptions);
     long avgOutboundRowBatchSize = Math.min(
-        (long) Math.ceil(rowBatchSize * exchNode_.getAvgSerializedRowSize(exchNode_)),
+        (long) Math.ceil(rowBatchSize * ExchangeNode.getAvgSerializedRowSize(exchNode_)),
         PlanNode.ROWBATCH_MAX_MEM_USAGE);
     // Each channel has 2 OutboundRowBatch (KrpcDataStreamSender::NUM_OUTBOUND_BATCHES).
     int outboundBatchesPerChannel = 2;
@@ -89,7 +89,8 @@ public class DataStreamSink extends DataSink {
     // The sending part of the processing cost for the exchange node.
     processingCost_ =
         ProcessingCost.basicCost(getLabel() + "(" + exchNode_.getDisplayLabel() + ")",
-            exchNode_.getCardinality(), 0, exchNode_.estimateSerializationCostPerRow());
+            exchNode_.getFilteredCardinality(), 0,
+            exchNode_.estimateSerializationCostPerRow());
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
index 6a7600906..d2882cc77 100644
--- a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
@@ -17,6 +17,8 @@
 
 package org.apache.impala.planner;
 
+import java.util.Stack;
+
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.SortInfo;
@@ -338,4 +340,11 @@ public class ExchangeNode extends PlanNode {
       msg.exchange_node.setOffset(offset_);
     }
   }
+
+  @Override
+  protected void reduceCardinalityByRuntimeFilter(
+      Stack<PlanNode> nodeStack, double reductionScale) {
+    if (!nodeStack.isEmpty()) nodeStack.add(this);
+    getChild(0).reduceCardinalityByRuntimeFilter(nodeStack, reductionScale);
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
index 83581ad34..65424c4c0 100644
--- a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
@@ -334,11 +334,11 @@ public class HashJoinNode extends JoinNode {
     // Compute the processing cost for lhs.
     ProcessingCost probeProcessingCost =
         ProcessingCost.basicCost(getDisplayLabel() + " Probe side (eqJoinConjuncts_)",
-            getChild(0).getCardinality(), eqJoinPredicateEvalCost);
+            getProbeCardinalityForCosting(), eqJoinPredicateEvalCost);
     if (otherJoinPredicateEvalCost > 0) {
       probeProcessingCost = ProcessingCost.sumCost(probeProcessingCost,
           ProcessingCost.basicCost(getDisplayLabel() + " Probe side(otherJoinConjuncts_)",
-              getCardinality(), otherJoinPredicateEvalCost));
+              getFilteredCardinality(), otherJoinPredicateEvalCost));
     }
 
     // Compute the processing cost for rhs.
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
index 08a6b718d..edb410962 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -238,6 +238,9 @@ public class HdfsScanNode extends ScanNode {
   // Whether all formats scanned are Parquet. Set in computeScanRangeLocations().
   private boolean allParquet_ = false;
 
+  // Whether all formats scanned are columnar format. Set in computeScanRangeLocations().
+  private boolean allColumnarFormat_ = false;
+
   // Number of bytes in the largest scan range (i.e. hdfs split). Set in
   // computeScanRangeLocations().
   private long largestScanRangeBytes_ = 0;
@@ -1135,6 +1138,7 @@ public class HdfsScanNode extends ScanNode {
     largestScanRangeBytes_ = 0;
     maxScanRangeNumRows_ = -1;
     boolean allParquet = (partitions_.size() > 0) ? true : false;
+    boolean allColumnarFormat = (partitions_.size() > 0) ? true : false;
     long simpleLimitNumRows = 0; // only used for the simple limit case
     boolean isSimpleLimit = sampleParams_ == null &&
         (analyzer.getQueryCtx().client_request.getQuery_options()
@@ -1200,6 +1204,8 @@ public class HdfsScanNode extends ScanNode {
       if (!partition.getFileFormat().isParquetBased()) {
         allParquet = false;
       }
+      allColumnarFormat =
+          allColumnarFormat && VALID_COLUMNAR_FORMATS.contains(partition.getFileFormat());
       Preconditions.checkState(partition.getId() >= 0);
 
       if (!fsHasBlocks) {
@@ -1265,6 +1271,7 @@ public class HdfsScanNode extends ScanNode {
       }
     }
     allParquet_ = allParquet;
+    allColumnarFormat_ = allColumnarFormat;
     if (totalFilesPerFs_.isEmpty() || sumValues(totalFilesPerFs_) == 0) {
       maxScanRangeNumRows_ = 0;
     } else {
@@ -1987,11 +1994,15 @@ public class HdfsScanNode extends ScanNode {
         extrapRows = "disabled";
       }
       output.append(detailPrefix)
-            .append("extrapolated-rows=")
-            .append(extrapRows)
-            .append(" max-scan-range-rows=")
-            .append(PrintUtils.printEstCardinality(maxScanRangeNumRows_))
-            .append("\n");
+          .append("extrapolated-rows=")
+          .append(extrapRows)
+          .append(" max-scan-range-rows=")
+          .append(PrintUtils.printEstCardinality(maxScanRangeNumRows_));
+      if (filteredCardinality_ > -1 && scanRangeSelectivity_ < 1.0) {
+        output.append(String.format(" est-scan-range=%d(filtered from %d)",
+            estScanRangeAfterRuntimeFilter(), getEffectiveNumScanRanges()));
+      }
+      output.append("\n");
       if (numScanRangesNoDiskIds_ > 0) {
         output.append(detailPrefix)
             .append(String.format("missing disk ids: "
@@ -2530,4 +2541,6 @@ public class HdfsScanNode extends ScanNode {
       });
     }
   }
+
+  protected boolean isAllColumnarScanner() { return allColumnarFormat_; }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/IcebergDeleteNode.java b/fe/src/main/java/org/apache/impala/planner/IcebergDeleteNode.java
index e4c131bdf..2da88274b 100644
--- a/fe/src/main/java/org/apache/impala/planner/IcebergDeleteNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/IcebergDeleteNode.java
@@ -225,7 +225,7 @@ public class IcebergDeleteNode extends JoinNode {
     // Compute the processing cost for lhs.
     ProcessingCost probeProcessingCost =
         ProcessingCost.basicCost(getDisplayLabel() + " Probe side (eqJoinConjuncts_)",
-            getChild(0).getCardinality(), eqJoinPredicateEvalCost);
+            getProbeCardinalityForCosting(), eqJoinPredicateEvalCost);
 
     // Compute the processing cost for rhs.
     ProcessingCost buildProcessingCost =
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinNode.java b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
index ffdd788b6..16f588b6e 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Stack;
 
 import org.apache.impala.analysis.AnalyticExpr;
 import org.apache.impala.analysis.Analyzer;
@@ -429,7 +430,7 @@ public abstract class JoinNode extends PlanNode {
 
     // first collect all the join cardinalities
     for (EqJoinConjunctScanSlots slots: eqJoinConjunctSlots) {
-      joinCardList.add(getGenericJoinCardinalityInternal(slots.lhsNdv(), slots.rhsNdv(),
+      joinCardList.add(computeGenericJoinCardinality(slots.lhsNdv(), slots.rhsNdv(),
           slots.lhsNumRows(), slots.rhsNumRows(), lhsCard, rhsCard));
     }
 
@@ -461,11 +462,11 @@ public abstract class JoinNode extends PlanNode {
   }
 
   /**
-   * An internal utility method to compute generic join cardinality as described
+   * A utility method to compute generic join cardinality as described
    * in the comments for {@link JoinNode#getJoinCardinality}. The input
    * cardinalities must be >= 0.
    */
-  private long getGenericJoinCardinalityInternal(double lhsNdv, double rhsNdv,
+  public static long computeGenericJoinCardinality(double lhsNdv, double rhsNdv,
       double lhsNumRows, double rhsNumRows, long lhsCard, long rhsCard) {
     Preconditions.checkState(lhsCard >= 0 && rhsCard >= 0);
     // Adjust the NDVs on both sides to account for predicates. Intuitively, the NDVs
@@ -482,10 +483,10 @@ public abstract class JoinNode extends PlanNode {
   }
 
   /**
-   * This function mirrors the logic for {@link JoinNode#getGenericJoinCardinality} except
-   * that instead of the EqJoinConjunctScanSlots, it uses the {@link NdvAndRowCountStats}
-   * to directly access stats that were pre-populated. Currently, this function is
-   * restricted to inner and outer joins.
+   * This function mirrors the logic for {@link JoinNode#computeGenericJoinCardinality}
+   * except that instead of the EqJoinConjunctScanSlots, it uses the {@link
+   * NdvAndRowCountStats} to directly access stats that were pre-populated. Currently,
+   * this function is restricted to inner and outer joins.
    * TODO: check if applicable for anti and semi joins
    */
   private long getGenericJoinCardinality2(List<NdvAndRowCountStats> statsList,
@@ -498,7 +499,7 @@ public abstract class JoinNode extends PlanNode {
       analyzer.getQueryOptions().getJoin_selectivity_correlation_factor();
     double cumulativeSel = 1.0;
     for (NdvAndRowCountStats stats: statsList) {
-      long joinCard = getGenericJoinCardinalityInternal(stats.lhsNdv(), stats.rhsNdv(),
+      long joinCard = computeGenericJoinCardinality(stats.lhsNdv(), stats.rhsNdv(),
           stats.lhsNumRows(), stats.rhsNumRows(), lhsCard, rhsCard);
       if (result == -1) {
         result = joinCard;
@@ -1002,4 +1003,35 @@ public abstract class JoinNode extends PlanNode {
    * Does not modify the state of this node.
    */
   public abstract Pair<ProcessingCost, ProcessingCost> computeJoinProcessingCost();
+
+  protected long getProbeCardinalityForCosting() {
+    return getChild(0).getFilteredCardinality();
+  }
+
+  @Override
+  protected void reduceCardinalityByRuntimeFilter(
+      Stack<PlanNode> nodeStack, double reductionScale) {
+    if (isSelectiveAndReducingJoin()) {
+      nodeStack.add(this);
+    } else {
+      nodeStack.clear();
+    }
+    int i = 0;
+    for (PlanNode child : getChildren()) {
+      if (i > 0) nodeStack.clear(); // not probe child
+      child.reduceCardinalityByRuntimeFilter(nodeStack, reductionScale);
+      ++i;
+    }
+  }
+
+  private boolean isSelectiveAndReducingJoin() {
+    if (eqJoinConjuncts_.isEmpty() || getChild(0).getCardinality() < 0
+        || getChild(1).getCardinality() < 0
+        || (!joinOp_.isInnerJoin() && !joinOp_.isLeftOuterJoin()
+            && !joinOp_.isLeftSemiJoin())) {
+      return false;
+    }
+    double selectivity = RuntimeFilterGenerator.getJoinNodeSelectivity(this);
+    return 0 <= selectivity && selectivity <= 1;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
index 56cbc340a..d27fd12c1 100644
--- a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
@@ -101,19 +101,20 @@ public class NestedLoopJoinNode extends JoinNode {
     // TODO: The cost should consider conjuncts_ as well.
     ProcessingCost probeProcessingCost = ProcessingCost.zero();
     ProcessingCost buildProcessingCost = ProcessingCost.zero();
+    long probeCardinality = getProbeCardinalityForCosting();
+    long buildCardinality = getChild(1).getCardinality();
     if (getChild(1) instanceof SingularRowSrcNode) {
       // Compute the processing cost for lhs.
-      probeProcessingCost =
-          ProcessingCost.basicCost(getDisplayLabel() + "(c0, singularRowSrc) Probe side",
-              getChild(0).getCardinality(), 0);
+      probeProcessingCost = ProcessingCost.basicCost(
+          getDisplayLabel() + "(c0, singularRowSrc) Probe side", probeCardinality, 0);
 
       // Compute the processing cost for rhs.
       buildProcessingCost = ProcessingCost.basicCost(
           getDisplayLabel() + "(c0, singularRowSrc) Build side per probe",
-          getChild(1).getCardinality(), 0);
+          buildCardinality, 0);
       // Multiply by the number of probes
-      buildProcessingCost = ProcessingCost.scaleCost(
-          buildProcessingCost, Math.max(0, getChild(0).getCardinality()));
+      buildProcessingCost =
+          ProcessingCost.scaleCost(buildProcessingCost, Math.max(0, probeCardinality));
     } else {
       // Assume 'eqJoinConjuncts_' will be applied to all rows from lhs side,
       // and 'otherJoinConjuncts_' to the resultant rows.
@@ -124,18 +125,18 @@ public class NestedLoopJoinNode extends JoinNode {
       // Compute the processing cost for lhs.
       probeProcessingCost = ProcessingCost.basicCost(
           getDisplayLabel() + "(c0, non-singularRowSrc, eqJoinConjuncts_) Probe side",
-          getChild(0).getCardinality(), eqJoinPredicateEvalCost);
+          probeCardinality, eqJoinPredicateEvalCost);
 
       probeProcessingCost = ProcessingCost.sumCost(probeProcessingCost,
           ProcessingCost.basicCost(getDisplayLabel()
                   + "(c0, non-singularRowSrc, otherJoinConjuncts_) Probe side",
-              getCardinality(), otherJoinPredicateEvalCost));
+              getFilteredCardinality(), otherJoinPredicateEvalCost));
 
       // Compute the processing cost for rhs, assuming 'eqJoinConjuncts_' will be applied
       // to all rows from rhs side.
       buildProcessingCost = ProcessingCost.basicCost(
-          getDisplayLabel() + "(c0, non-singularRowSrc) Build side",
-          getChild(1).getCardinality(), eqJoinPredicateEvalCost);
+          getDisplayLabel() + "(c0, non-singularRowSrc) Build side", buildCardinality,
+          eqJoinPredicateEvalCost);
     }
     return Pair.create(probeProcessingCost, buildProcessingCost);
   }
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanNode.java b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
index 0f662fb81..bdd2fbcb0 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.Stack;
 
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.BinaryPredicate;
@@ -123,10 +124,16 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   // TODO for 2.3: Save this state in the PlannerContext instead.
   protected Set<ExprId> assignedConjuncts_;
 
-  // estimate of the output cardinality of this node; set in computeStats();
+  // estimate of the output cardinality of this node; set in computeStats() and can be
+  // replaced through filterCardinality();
   // invalid: -1
   protected long cardinality_;
 
+  // Estimate of the output cardinality of this node after reduction by runtime filter.
+  // Set in setFilteredCardinality(). Valid value is > -1.
+  // TODO: merge this with cardinality_.
+  protected long filteredCardinality_ = -1;
+
   // Estimated number of nodes on which the plan tree rooted at this node would be
   // scheduled;
   // set in computeStats(); invalid: -1
@@ -376,8 +383,15 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
       if (detailLevel == TExplainLevel.STANDARD) expBuilder.append(detailPrefix);
       expBuilder.append("row-size=")
           .append(PrintUtils.printBytes(Math.round(avgRowSize_)))
-          .append(" cardinality=")
-          .append(PrintUtils.printEstCardinality(cardinality_));
+          .append(" cardinality=");
+      if (filteredCardinality_ > -1) {
+        expBuilder.append(PrintUtils.printEstCardinality(filteredCardinality_))
+            .append("(filtered from ")
+            .append(PrintUtils.printEstCardinality(cardinality_))
+            .append(")");
+      } else {
+        expBuilder.append(PrintUtils.printEstCardinality(cardinality_));
+      }
       if (queryOptions.isCompute_processing_cost()) {
         // Show processing cost total.
         expBuilder.append(" cost=");
@@ -1157,4 +1171,31 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     }
     return numCols;
   }
+
+  protected void setFilteredCardinality(long newCardinality) {
+    Preconditions.checkState(newCardinality > -1);
+    Preconditions.checkState(newCardinality < cardinality_);
+    filteredCardinality_ = newCardinality;
+  }
+
+  // TODO: merge this with getCardinality().
+  protected long getFilteredCardinality() {
+    return filteredCardinality_ > -1 ? filteredCardinality_ : getCardinality();
+  }
+
+  /**
+   * Find contiguous probe pipeline consisting of ScanNode, ExchangeNodes, and JoinNodes
+   * and reduce their cardinality with assumption that runtime filter targeting
+   * the ScanNode can filter out rows effectively (as selective as the originating
+   * join node) before the join operation. 'reductionScale' is a [0.0..1.0] range to
+   * control the scale of reduction. Higher value means more reduction (lower cardinality
+   * estimate).
+   */
+  protected void reduceCardinalityByRuntimeFilter(
+      Stack<PlanNode> nodeStack, double reductionScale) {
+    nodeStack.clear();
+    for (PlanNode child : getChildren()) {
+      child.reduceCardinalityByRuntimeFilter(nodeStack, reductionScale);
+    }
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/Planner.java b/fe/src/main/java/org/apache/impala/planner/Planner.java
index 8eb0ca914..48bf339e8 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Stack;
 import java.util.stream.Collectors;
 
 import org.apache.impala.analysis.AnalysisContext;
@@ -498,6 +499,23 @@ public class Planner {
     return root.maxCore(rootCores.first, CoreCount.sum(rootCores.second));
   }
 
+  /**
+   * Reduce plan node cardinalities based on runtime filter information.
+   * Valid to call after runtime filter generation and before processing cost
+   * computation.
+   */
+  public static void reduceCardinalityByRuntimeFilter(
+      List<PlanFragment> planRoots, PlannerContext planCtx) {
+    double reductionScale = planCtx.getRootAnalyzer()
+                                .getQueryOptions()
+                                .getRuntime_filter_cardinality_reduction_scale();
+    if (reductionScale <= 0) return;
+    PlanFragment rootFragment = planRoots.get(0);
+    Stack<PlanNode> nodeStack = new Stack<>();
+    rootFragment.getPlanRoot().reduceCardinalityByRuntimeFilter(
+        nodeStack, reductionScale);
+  }
+
   /**
    * Compute processing cost of each plan fragment in the query plan and adjust each
    * fragment parallelism according to producer-consumer rate between them.
diff --git a/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java b/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
index 904904d15..d42d9d3f5 100644
--- a/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
+++ b/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
@@ -643,6 +643,15 @@ public final class RuntimeFilterGenerator {
       return null;
     }
 
+    public boolean isPartitionFilterAt(PlanNodeId targetPlanNodeId) {
+      for (RuntimeFilterTarget target : targets_) {
+        if (target.node.getId() == targetPlanNodeId) {
+          return target.isBoundByPartitionColumns;
+        }
+      }
+      return false;
+    }
+
     public List<RuntimeFilterTarget> getTargets() { return targets_; }
     public boolean hasTargets() { return !targets_.isEmpty(); }
     public Expr getSrcExpr() { return srcExpr_; }
@@ -653,6 +662,25 @@ public final class RuntimeFilterGenerator {
     public Operator getExprCompOp() { return exprCmpOp_; }
     public long getFilterSize() { return filterSizeBytes_; }
     public boolean isTimestampTruncation() { return isTimestampTruncation_; }
+    public PlanNode getSrc() { return src_; }
+
+    private long getBuildKeyNumRowStats() {
+      long minNumRows = src_.getChild(1).getCardinality();
+      SlotRef buildSlotRef = srcExpr_.unwrapSlotRef(true);
+      if (buildSlotRef == null || !buildSlotRef.hasDesc()
+          || buildSlotRef.getDesc().getParent() == null
+          || buildSlotRef.getDesc().getParent().getTable() == null
+          || buildSlotRef.getDesc().getParent().getTable().getNumRows() <= -1) {
+        return minNumRows;
+      }
+      return buildSlotRef.getDesc().getParent().getTable().getNumRows();
+    }
+
+    /**
+     * Return the estimated false-positive probability of this filter.
+     * Bloom filter type will return value in [0.0 - 1.0] range. Otherwise, return 0.
+     */
+    public double getEstFpp() { return est_fpp_; }
 
     /**
      * Return TIMESTAMP if the isTimestampTruncation_ is set as true so that
@@ -672,14 +700,7 @@ public final class RuntimeFilterGenerator {
      * associated source join node over the cardinality of that join node's left
      * child.
      */
-    public double getSelectivity() {
-      if (src_.getCardinality() == -1
-          || src_.getChild(0).getCardinality() == -1
-          || src_.getChild(0).getCardinality() == 0) {
-        return -1;
-      }
-      return src_.getCardinality() / (double) src_.getChild(0).getCardinality();
-    }
+    public double getSelectivity() { return getJoinNodeSelectivity(src_); }
 
     public void addTarget(RuntimeFilterTarget target) { targets_.add(target); }
 
@@ -752,6 +773,62 @@ public final class RuntimeFilterGenerator {
       for (RuntimeFilterTarget target: targets_) target.node.addRuntimeFilter(this);
     }
 
+    /**
+     * Return true if this filter is deemed highly selective, will arrive on-time, and
+     * applied for all rows without ever being disabled by the scanner node.
+     * Mainly used by {@link ScanNode#reduceCardinalityByRuntimeFilter(java.util.Stack)}
+     * to lower the scan cardinality estimation. These properties are hard to predict
+     * during planning, so this method is set very conservative to avoid severe
+     * underestimation.
+     */
+    public boolean isHighlySelective() {
+      return level_ <= 3 && (type_ != TRuntimeFilterType.BLOOM || est_fpp_ < 0.33);
+    }
+
+    /**
+     * Return a reduced cardinality estimate for given scanNode.
+     * @param scanNode the ScanNode to inspect.
+     * @param scanCardinality the cardinality estimate before reducing through this
+     *                        filter. This can be lower than scanNode.inputCardinality_.
+     * @param partitionSelectivities a map of column name to selectivity on that column.
+     *                               Will be populated if this filter target a partition
+     *                               column.
+     * @return a reduced cardinality estimate or -1 if unknown.
+     */
+    public long reducedCardinalityForScanNode(ScanNode scanNode, long scanCardinality,
+        Map<String, Double> partitionSelectivities) {
+      PlanNodeId scanNodeId = scanNode.getId();
+      long scanColumnNdv = getTargetExpr(scanNodeId).getNumDistinctValues();
+      if (scanColumnNdv < 0) {
+        // For cardinality reduction, it is OK to skip this filter if 'scanColumnNdv'
+        // is unknown rather than trying to estimate it.
+        return -1;
+      }
+      long buildKeyNdv = ndvEstimate_;
+      long buildKeyCard = src_.getChild(1).getCardinality();
+      long buildKeyNumRows = getBuildKeyNumRowStats();
+      // The raw input cardinality without applying scan conjuct and limits.
+      long scanNumRows = scanNode.inputCardinality_;
+
+      long estCardAfterFilter = JoinNode.computeGenericJoinCardinality(scanColumnNdv,
+          buildKeyNdv, scanNumRows, buildKeyNumRows, scanCardinality, buildKeyCard);
+
+      if (isPartitionFilterAt(scanNodeId)) {
+        // Partition filter can apply at file-level filtering.
+        // Keep the most selective partition filter per column to reduce
+        // inputCardinality_. Ignore a column if its name is unknown.
+        double thisPartSel = (double) buildKeyNdv / scanColumnNdv;
+        SlotRef targetSlot = getTargetExpr(scanNodeId).unwrapSlotRef(true);
+        if (targetSlot != null && targetSlot.hasDesc()
+            && targetSlot.getDesc().getColumn() != null) {
+          String colName = targetSlot.getDesc().getColumn().getName();
+          double currentSel = partitionSelectivities.computeIfAbsent(colName, c -> 1.0);
+          if (thisPartSel < currentSel) partitionSelectivities.put(colName, thisPartSel);
+        }
+      }
+      return estCardAfterFilter;
+    }
+
     public String debugString() {
       StringBuilder output = new StringBuilder();
       output.append("FilterID: " + id_)
@@ -1283,6 +1360,17 @@ public final class RuntimeFilterGenerator {
     return false;
   }
 
+  /**
+   * Get selectivity of joinNode.
+   */
+  public static double getJoinNodeSelectivity(JoinNode joinNode) {
+    if (joinNode.getCardinality() == -1 || joinNode.getChild(0).getCardinality() == -1
+        || joinNode.getChild(0).getCardinality() == 0) {
+      return -1;
+    }
+    return joinNode.getCardinality() / (double) joinNode.getChild(0).getCardinality();
+  }
+
   /**
    * Computes the target expr for a specified runtime filter 'filter' to be applied at
    * the scan node with target tuple descriptor 'targetTid'.
diff --git a/fe/src/main/java/org/apache/impala/planner/ScanNode.java b/fe/src/main/java/org/apache/impala/planner/ScanNode.java
index 36a694296..2ce3b2821 100644
--- a/fe/src/main/java/org/apache/impala/planner/ScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/ScanNode.java
@@ -18,7 +18,10 @@
 package org.apache.impala.planner;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Stack;
 
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.Expr;
@@ -33,6 +36,7 @@ import org.apache.impala.analysis.TupleDescriptor;
 import org.apache.impala.catalog.HdfsFileFormat;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.NotImplementedException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.common.PrintUtils;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.service.BackendConfig;
@@ -41,6 +45,8 @@ import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TScanRangeSpec;
 import org.apache.impala.thrift.TTableStats;
 import org.apache.impala.util.ExprUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.MoreObjects;
@@ -52,6 +58,7 @@ import com.google.common.math.LongMath;
  * Representation of the common elements of all scan nodes.
  */
 abstract public class ScanNode extends PlanNode {
+  private final static Logger LOG = LoggerFactory.getLogger(ScanNode.class);
   // Factor capturing the worst-case deviation from a uniform distribution of scan ranges
   // among nodes. The factor of 1.2 means that a particular node may have 20% more
   // scan ranges than would have been estimated assuming a uniform distribution.
@@ -64,6 +71,11 @@ abstract public class ScanNode extends PlanNode {
   // Total number of rows this node is expected to process
   protected long inputCardinality_ = -1;
 
+  // Total number of rows this node is expected to process after reduction by runtime
+  // filter.
+  // TODO: merge this with inputCardinality_.
+  protected long filteredInputCardinality_ = -1;
+
   // Scan-range specs. Populated in init().
   protected TScanRangeSpec scanRangeSpecs_;
 
@@ -90,6 +102,11 @@ abstract public class ScanNode extends PlanNode {
   // Default to MIN_NUM_SCAN_THREADS.
   protected int maxScannerThreads_ = MIN_NUM_SCAN_THREADS;
 
+  // Selectivity estimates of scan ranges to open if this scan node consumes a partition
+  // filter. This selectivity is based on assumption that scan ranges are uniformly
+  // distributed across all partitions. Set in reduceCardinalityByRuntimeFilter().
+  protected double scanRangeSelectivity_ = 1.0;
+
   public ScanNode(PlanNodeId id, TupleDescriptor desc, String displayName) {
     super(id, desc.getId().asList(), displayName);
     desc_ = desc;
@@ -315,6 +332,14 @@ abstract public class ScanNode extends PlanNode {
     return inputCardinality_;
   }
 
+  // TODO: merge this with getInputCardinality().
+  public long getFilteredInputCardinality() {
+    if (!hasScanConjuncts() && !hasStorageLayerConjuncts() && hasLimit()) {
+      return getLimit();
+    }
+    return filteredCardinality_ > -1 ? filteredCardinality_ : inputCardinality_;
+  }
+
   @Override
   protected String getDisplayLabelDetail() {
     Preconditions.checkNotNull(desc_.getPath());
@@ -372,8 +397,8 @@ abstract public class ScanNode extends PlanNode {
         queryOptions.getMax_fragment_instances_per_node());
     int maxThreadsGlobal = IntMath.saturatedMultiply(getNumNodes(), maxThreadsPerNode);
     maxScannerThreads_ = Math.max(MIN_NUM_SCAN_THREADS,
-        (int) Math.min(getEffectiveNumScanRanges(), maxThreadsGlobal));
-    long inputCardinality = getInputCardinality();
+        (int) Math.min(estScanRangeAfterRuntimeFilter(), maxThreadsGlobal));
+    long inputCardinality = getFilteredInputCardinality();
 
     if (inputCardinality >= 0) {
       ProcessingCost cardinalityBasedCost =
@@ -407,14 +432,19 @@ abstract public class ScanNode extends PlanNode {
    */
   private float rowMaterializationCost() {
     float perRowCost = getAvgRowSize() / 1024;
-    if (getInputCardinality() <= 0) return perRowCost;
+    if (getFilteredInputCardinality() <= 0) return perRowCost;
 
-    float scanRangeCostPerRow = BackendConfig.INSTANCE.getMinProcessingPerThread()
-        * BackendConfig.INSTANCE.getScanRangeCostFactor() / getInputCardinality()
-        * getEffectiveNumScanRanges();
+    float perScanRangeCost = BackendConfig.INSTANCE.getMinProcessingPerThread()
+        * BackendConfig.INSTANCE.getScanRangeCostFactor();
+    float scanRangeCostPerRow = perScanRangeCost / getFilteredInputCardinality()
+        * estScanRangeAfterRuntimeFilter();
     return perRowCost + scanRangeCostPerRow;
   }
 
+  protected int estScanRangeAfterRuntimeFilter() {
+    return (int) Math.ceil(getEffectiveNumScanRanges() * scanRangeSelectivity_);
+  }
+
   /**
    * Returns true if this node has conjuncts to be evaluated by Impala against the scan
    * tuple.
@@ -433,4 +463,159 @@ abstract public class ScanNode extends PlanNode {
     Preconditions.checkNotNull(scanRangeSpecs_);
     return scanRangeSpecs_.getConcrete_rangesSize();
   }
+
+  /**
+   * Return runtime filters targetting this scan node that are likely to reduce
+   * cardinality estimation. Returned filters are organized as a map of originating
+   * join node id to list of runtime filters from it.
+   */
+  private Map<PlanNodeId, List<RuntimeFilterGenerator.RuntimeFilter>>
+      groupFiltersForCardinalityReduction() {
+    // Row-level filtering is only available in Kudu scanner (through runtime filter
+    // pushdown) and HDFS columnar scanner (see EvalRuntimeFilter call in
+    // HdfsColumnarScanner::ProcessScratchBatch).
+    boolean evalAtRowLevel = (this instanceof KuduScanNode)
+        || ((this instanceof HdfsScanNode)
+            && ((HdfsScanNode) this).isAllColumnarScanner());
+
+    Map<PlanNodeId, List<RuntimeFilterGenerator.RuntimeFilter>> filtersByJoinId =
+        new HashMap<>();
+    for (RuntimeFilterGenerator.RuntimeFilter filter : getRuntimeFilters()) {
+      PlanNodeId filterSourceId = filter.getSrc().getId();
+      boolean isPartitionFilter = filter.isPartitionFilterAt(id_);
+      if (filter.isHighlySelective() && (isPartitionFilter || evalAtRowLevel)) {
+        // Partition level filtering always applies regardless of file format.
+        // Row-level runtime filtering, however, only applies at HDFS columnar file
+        // format and Kudu.
+        filtersByJoinId.computeIfAbsent(filterSourceId, id -> new ArrayList<>())
+            .add(filter);
+      }
+    }
+    return filtersByJoinId;
+  }
+
+  /**
+   * Given a contiguous probe pipeline 'nodeStack' that begins from this scan node,
+   * calculate a reduced output cardinality estimate of this scan node. The probe
+   * pipeline 'nodeStack' must have original cardinality estimate that is not increasing
+   * from scan node towards the bottom join node in stack. The join node at the bottom of
+   * stack is assumed to have the least output cardinality and all nodes below it in node
+   * tree must not have cardinality less than it.
+   * Return a pair of estimated output cardinality and partition selectivity from
+   * evaluating runtime filters.
+   */
+  private Pair<Long, Double> getReducedCardinalityByFilter(
+      Stack<PlanNode> nodeStack, double reductionScale) {
+    Map<PlanNodeId, List<RuntimeFilterGenerator.RuntimeFilter>> filtersByJoinId =
+        groupFiltersForCardinalityReduction();
+    long reducedCardinality = cardinality_;
+    Map<String, Double> partitionSelectivities = new HashMap<>();
+
+    // Compute scan cardinality reduction by applying runtime filters from the bottom
+    // of probe pipelines upward.
+    for (int i = nodeStack.size() - 1; i >= 0; i--) {
+      PlanNode node = nodeStack.get(i);
+      if (node instanceof ExchangeNode) continue;
+
+      Preconditions.checkState(node instanceof JoinNode);
+      JoinNode join = (JoinNode) node;
+      PlanNodeId joinId = join.getId();
+      if (!filtersByJoinId.containsKey(joinId)) continue;
+
+      long cardOnThisJoin = reducedCardinality;
+      for (RuntimeFilterGenerator.RuntimeFilter filter : filtersByJoinId.get(joinId)) {
+        long estCardAfterFilter = filter.reducedCardinalityForScanNode(
+            this, reducedCardinality, partitionSelectivities);
+        if (estCardAfterFilter > -1) {
+          cardOnThisJoin = Math.min(cardOnThisJoin, estCardAfterFilter);
+        }
+      }
+      reducedCardinality = cardOnThisJoin;
+    }
+
+    double partSel =
+        partitionSelectivities.values().stream().reduce(1.0, (a, b) -> a * b);
+    double scaledPartSel = 1.0 - ((1.0 - partSel) * reductionScale);
+
+    // The lowest join node in the stack should have the least output cardinality.
+    // Cap the minumum scan cardinality at highest join node's cardinality.
+    long lowestJoinCard = nodeStack.get(0).getCardinality();
+    long scaledReduction = (long) ((cardinality_ - reducedCardinality) * reductionScale);
+    long scaledCardinality = Math.max(lowestJoinCard, cardinality_ - scaledReduction);
+    return Pair.create(scaledCardinality, scaledPartSel);
+  }
+
+  /**
+   * Applies cardinality reduction over a contiguous probe pipeline 'nodeStack' that
+   * begins from this scan node. Depending on whether join nodes in 'nodeStack'
+   * produces a runtime filter and the type of that runtime filter, this method then
+   * applies the runtime filter selectivity to this scan node, reducing its cardinality
+   * and input cardinality estimate. The runtime filter selectivity is calculated with
+   * the simplest join cardinality formula from JoinNode.computeGenericJoinCardinality().
+   * 'reductionScale' is a [0.0..1.0] range to control the scale of reduction.
+   * Higher value means more reduction (lower cardinality estimate).
+   */
+  @Override
+  protected void reduceCardinalityByRuntimeFilter(
+      Stack<PlanNode> nodeStack, double reductionScale) {
+    if (nodeStack.isEmpty() || inputCardinality_ <= 0) {
+      nodeStack.clear();
+      return;
+    }
+    // Sanity check that original cardinality is non-increasing from top to bottom of
+    // stack.
+    long prevCardinality = cardinality_;
+    for (int i = nodeStack.size() - 1; i >= 0; i--) {
+      long currentCardinality = nodeStack.get(i).getCardinality();
+      Preconditions.checkState(currentCardinality <= prevCardinality,
+          "Original cardinality of " + nodeStack.get(i).getDisplayLabel()
+              + " is larger than node below it.");
+      prevCardinality = currentCardinality;
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("reduceCardinalityByRuntimeFilter from " + getDisplayLabel() + " to "
+          + nodeStack.get(0).getDisplayLabel());
+    }
+
+    Pair<Long, Double> reducedCardinality =
+        getReducedCardinalityByFilter(nodeStack, reductionScale);
+    long scanCardinalityAfterFilter = reducedCardinality.getFirst();
+
+    // Estimate scanRangeSelectivity_ based on partition selectivity.
+    // This assumes that scan ranges are uniformly distributed across partitions.
+    // If num ranges > 0, cap it to estimate at least 1 scan range read.
+    long numRanges = getEffectiveNumScanRanges();
+    scanRangeSelectivity_ = reducedCardinality.getSecond();
+    if (numRanges > 0 && scanRangeSelectivity_ < (1.0 / numRanges)) {
+      scanRangeSelectivity_ = 1.0 / numRanges;
+    }
+
+    // Apply 'scanRangeSelectivity_' towards scan's 'inputCardinality_'.
+    // Do not directly assign with the much lower 'scanCardinalityAfterFilter' here
+    // since non-partition filter still requires scanner to open and read a scan range.
+    // Kudu scan is an exception because Kudu does the row-level filtering for Impala.
+    long inputCardinalityEst = (this instanceof KuduScanNode) ?
+        scanCardinalityAfterFilter :
+        Math.max(scanCardinalityAfterFilter,
+            (long) Math.ceil(inputCardinality_ * scanRangeSelectivity_));
+    if (inputCardinality_ > inputCardinalityEst) {
+      filteredInputCardinality_ = inputCardinalityEst;
+    }
+
+    if (cardinality_ > scanCardinalityAfterFilter) {
+      // Replace this scan's cardinality with 'scanCardinalityAfterFilter'.
+      setFilteredCardinality(scanCardinalityAfterFilter);
+
+      while (nodeStack.size() > 1
+          && nodeStack.peek().cardinality_ > scanCardinalityAfterFilter) {
+        // Update each joinNode's cardinality_ in the stack.
+        // Stop when there is only 1 join node left in the stack (which is the join node
+        // with the least cardinality that must not be reduced anymore) or when the next
+        // join node cardinality is lower than 'scanCardinalityAfterFilter'.
+        PlanNode node = nodeStack.pop();
+        node.setFilteredCardinality(scanCardinalityAfterFilter);
+      }
+    }
+    nodeStack.clear();
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index c0633b83a..7be0861a1 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -1827,6 +1827,7 @@ public class Frontend {
 
     // Compute resource requirements of the final plans.
     TQueryExecRequest result = new TQueryExecRequest();
+    Planner.reduceCardinalityByRuntimeFilter(planRoots, planner.getPlannerCtx());
     Planner.computeProcessingCost(planRoots, result, planner.getPlannerCtx());
     Planner.computeResourceReqs(planRoots, queryCtx, result,
         planner.getPlannerCtx(), planner.getAnalysisResult().isQueryStmt());
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index fb2490ae7..485a34c00 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -1468,4 +1468,23 @@ public class PlannerTest extends PlannerTestBase {
     runPlannerTestFile(
         "agg-node-high-mem-estimate", "tpcds_parquet", options, testOptions);
   }
+
+  /**
+   * Test cardinality reduction by runtime filter.
+   */
+  @Test
+  public void testRuntimeFilterCardinalityReduction() {
+    runPlannerTestFile("runtime-filter-cardinality-reduction", "tpcds_parquet",
+        ImmutableSet.of(
+            PlannerTestOption.EXTENDED_EXPLAIN, PlannerTestOption.VALIDATE_CARDINALITY));
+  }
+
+  /**
+   * Test cardinality reduction by runtime filter against Kudu.
+   */
+  @Test
+  public void testRuntimeFilterCardinalityReductionOnKudu() {
+    runPlannerTestFile("runtime-filter-cardinality-reduction-on-kudu", "tpch_kudu",
+        ImmutableSet.of(PlannerTestOption.VALIDATE_CARDINALITY));
+  }
 }
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test b/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
index 5d10e41a2..ad48a6bc2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
@@ -1489,7 +1489,7 @@ PLAN-ROOT SINK
 00:SCAN HDFS [tpch_parquet.lineitem]
    partitions=1/1 files=3 size=193.60MB
    runtime filters: RF004 -> l_orderkey, RF005 -> l_returnflag
-   row-size=21B cardinality=6.00M
+   row-size=21B cardinality=5.76M(filtered from 6.00M)
 ====
 # IMPALA-4263: Grouping agg needs a merge step because the grouping exprs reference a
 # tuple that is made nullable in the join fragment.
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test b/testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test
index a8f648bcd..47fe62ced 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test
@@ -567,7 +567,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: o_orderkey = o_orderkey
 |  runtime filters: RF000[bloom] <- o_orderkey
 |  mem-estimate=59.44MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=0,2,4,6 row-size=32B cardinality=1
+|  tuple-ids=0,2,4,6 row-size=32B cardinality=5.76M
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--03:SCAN HDFS [tpch_parquet.orders]
@@ -585,7 +585,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: o_orderkey = o_orderkey
 |  runtime filters: RF002[bloom] <- o_orderkey
 |  mem-estimate=59.44MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=0,2,4 row-size=24B cardinality=1
+|  tuple-ids=0,2,4 row-size=24B cardinality=5.76M
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpch_parquet.orders]
@@ -604,7 +604,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: l_orderkey = o_orderkey
 |  runtime filters: RF004[bloom] <- o_orderkey
 |  mem-estimate=59.44MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=0,2 row-size=16B cardinality=1
+|  tuple-ids=0,2 row-size=16B cardinality=5.76M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpch_parquet.orders]
@@ -1158,7 +1158,7 @@ PLAN-ROOT SINK
 |  hash predicates: o_orderkey = o_orderkey
 |  fk/pk conjuncts: o_orderkey = o_orderkey
 |  mem-estimate=59.44MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=0,2,4,6 row-size=40B cardinality=1
+|  tuple-ids=0,2,4,6 row-size=40B cardinality=575.77K
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--03:SCAN HDFS [tpch_parquet.orders]
@@ -1175,7 +1175,7 @@ PLAN-ROOT SINK
 |  hash predicates: o_orderkey = o_orderkey
 |  fk/pk conjuncts: o_orderkey = o_orderkey
 |  mem-estimate=59.44MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=0,2,4 row-size=32B cardinality=1
+|  tuple-ids=0,2,4 row-size=32B cardinality=575.77K
 |  in pipelines: 00(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpch_parquet.orders]
@@ -1193,7 +1193,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: l_orderkey = o_orderkey
 |  runtime filters: RF004[bloom] <- o_orderkey
 |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,2 row-size=24B cardinality=1
+|  tuple-ids=0,2 row-size=24B cardinality=575.77K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpch_parquet.orders]
@@ -1217,7 +1217,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=8B cardinality=6.00M
+   tuple-ids=0 row-size=8B cardinality=575.77K(filtered from 6.00M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -1336,7 +1336,7 @@ Per-Host Resources: mem-estimate=91.08MB mem-reservation=12.75MB thread-reservat
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=8B cardinality=6.00M
+   tuple-ids=0 row-size=8B cardinality=575.77K(filtered from 6.00M)
    in pipelines: 00(GETNEXT)
 ====
 # IMPALA-12357: build fragment with union still eligible for filter elimination.
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/convert-to-cnf.test b/testdata/workloads/functional-planner/queries/PlannerTest/convert-to-cnf.test
index 73f82467c..853bddbbf 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/convert-to-cnf.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/convert-to-cnf.test
@@ -27,7 +27,7 @@ PLAN-ROOT SINK
    HDFS partitions=1/1 files=3 size=193.98MB
    predicates: l_partkey > 0, l_suppkey > 10 OR l_suppkey > 30
    runtime filters: RF000 -> l_orderkey
-   row-size=24B cardinality=600.12K
+   row-size=24B cardinality=575.77K(filtered from 600.12K)
 ====
 
 # outer join
@@ -87,7 +87,7 @@ PLAN-ROOT SINK
    HDFS partitions=1/1 files=3 size=193.98MB
    predicates: l_partkey > 0, l_suppkey <= 30 OR l_suppkey >= 30 AND l_suppkey <= 50, l_suppkey >= 10 OR l_suppkey >= 30 AND l_suppkey <= 50
    runtime filters: RF000 -> l_orderkey
-   row-size=24B cardinality=600.12K
+   row-size=24B cardinality=575.77K(filtered from 600.12K)
 ====
 
 # equality predicates on l_suppkey should eventually be converted to IN
@@ -121,7 +121,7 @@ PLAN-ROOT SINK
    HDFS partitions=1/1 files=3 size=193.98MB
    predicates: l_partkey > 0, l_suppkey IN (10, 30, 50)
    runtime filters: RF000 -> l_orderkey
-   row-size=24B cardinality=586
+   row-size=24B cardinality=349(filtered from 586)
 ====
 
 # NOT predicate
@@ -151,7 +151,7 @@ PLAN-ROOT SINK
 01:SCAN HDFS [tpch_parquet.orders]
    HDFS partitions=1/1 files=2 size=54.21MB
    runtime filters: RF000 -> o_orderkey
-   row-size=16B cardinality=1.50M
+   row-size=16B cardinality=600.12K(filtered from 1.50M)
 ====
 
 # set the max_cnf_exprs limit
@@ -185,7 +185,7 @@ PLAN-ROOT SINK
    HDFS partitions=1/1 files=3 size=193.98MB
    predicates: l_partkey > 0
    runtime filters: RF000 -> l_orderkey
-   row-size=24B cardinality=600.12K
+   row-size=24B cardinality=575.77K(filtered from 600.12K)
 ====
 
 # reset the max_cnf_exprs limit to unlimited (-1)
@@ -219,7 +219,7 @@ PLAN-ROOT SINK
    HDFS partitions=1/1 files=3 size=193.98MB
    predicates: l_partkey > 0, l_suppkey <= 50 OR l_suppkey >= 30 AND l_suppkey <= 90, l_suppkey >= 10 OR l_suppkey >= 30 AND l_suppkey <= 90
    runtime filters: RF000 -> l_orderkey
-   row-size=24B cardinality=600.12K
+   row-size=24B cardinality=575.77K(filtered from 600.12K)
 ====
 
 # same as above, but set max_cnf_exprs to 0 (also implies unlimited)
@@ -253,7 +253,7 @@ PLAN-ROOT SINK
    HDFS partitions=1/1 files=3 size=193.98MB
    predicates: l_partkey > 0, l_suppkey <= 50 OR l_suppkey >= 30 AND l_suppkey <= 90, l_suppkey >= 10 OR l_suppkey >= 30 AND l_suppkey <= 90
    runtime filters: RF000 -> l_orderkey
-   row-size=24B cardinality=600.12K
+   row-size=24B cardinality=575.77K(filtered from 600.12K)
 ====
 
 # disable the rewrite, so no conversion is expected
@@ -285,7 +285,7 @@ PLAN-ROOT SINK
 01:SCAN HDFS [tpch_parquet.orders]
    HDFS partitions=1/1 files=2 size=54.21MB
    runtime filters: RF000 -> o_orderkey
-   row-size=16B cardinality=1.50M
+   row-size=16B cardinality=600.12K(filtered from 1.50M)
 ====
 # IMPALA-9620: query1
 # Test predicates in the SELECT and GROUP-BY
@@ -402,7 +402,7 @@ PLAN-ROOT SINK
 01:SCAN HDFS [tpch_parquet.orders]
    HDFS partitions=1/1 files=2 size=54.21MB
    runtime filters: RF000 -> o_orderkey
-   row-size=28B cardinality=1.50M
+   row-size=28B cardinality=600.12K(filtered from 1.50M)
 ====
 
 # IMPALA-11274: Functions like CAST should still be eligible for CNF
@@ -433,7 +433,7 @@ PLAN-ROOT SINK
    HDFS partitions=1/1 files=3 size=193.99MB
    predicates: l_partkey > 0, CAST(l_returnflag AS VARCHAR(2)) IN ('Y', 'N')
    runtime filters: RF000 -> l_orderkey
-   row-size=29B cardinality=600.12K
+   row-size=29B cardinality=575.77K(filtered from 600.12K)
 ====
 
 # IMPALA-11274: Simple arithmetic expressions should still be eligible for CNF
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/explain-verbose-mt_dop.test b/testdata/workloads/functional-planner/queries/PlannerTest/explain-verbose-mt_dop.test
index b713e4cfd..58173febf 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/explain-verbose-mt_dop.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/explain-verbose-mt_dop.test
@@ -289,10 +289,10 @@ Per-Instance Resources: mem-estimate=27.88MB mem-reservation=3.00MB thread-reser
        table: rows=2.88M size=199.44MB
        partitions: 1824/1824 rows=2.88M
        columns: all
-     extrapolated-rows=disabled max-scan-range-rows=130.09K
+     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
      file formats: [PARQUET]
      mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-     tuple-ids=0 row-size=20B cardinality=2.88M
+     tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
      in pipelines: 01(GETNEXT)
 
 F93:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -841,7 +841,7 @@ Per-Instance Resources: mem-estimate=59.88MB mem-reservation=6.00MB thread-reser
      extrapolated-rows=disabled max-scan-range-rows=650.14K
      file formats: [PARQUET]
      mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-     tuple-ids=34 row-size=20B cardinality=1.44M
+     tuple-ids=34 row-size=20B cardinality=85.31K(filtered from 1.44M)
      in pipelines: 43(GETNEXT)
 
 F105:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1382,7 +1382,7 @@ Per-Instance Resources: mem-estimate=43.88MB mem-reservation=6.00MB thread-reser
      extrapolated-rows=disabled max-scan-range-rows=644.77K
      file formats: [PARQUET]
      mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
-     tuple-ids=68 row-size=20B cardinality=719.38K
+     tuple-ids=68 row-size=20B cardinality=42.85K(filtered from 719.38K)
      in pipelines: 85(GETNEXT)
 
 F117:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
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 1e8e6ec4c..b8f123e1d 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
@@ -265,7 +265,7 @@ where ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number
   and d1.d_fy_week_seq = 1000
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=372.19MB mem-reservation=60.19MB thread-reservation=6 runtime-filters-memory=5.00MB
+|  Per-Host Resources: mem-estimate=370.31MB mem-reservation=58.31MB thread-reservation=6 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
@@ -340,9 +340,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=346.60MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=7(filtered from 1824)
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=0 row-size=24B cardinality=2.88M
+|     tuple-ids=0 row-size=24B cardinality=11.05K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 01:SCAN HDFS [tpcds.store_returns]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
index 7d5a3b3dc..ce20acd33 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
@@ -404,7 +404,7 @@ PLAN-ROOT SINK
    predicates: CAST(a.id AS STRING) > '123'
    kudu predicates: a.id > 10
    runtime filters: RF000 -> a.id, RF001 -> a.id
-   row-size=4B cardinality=730
+   row-size=4B cardinality=10(filtered from 730)
 ====
 # IMPALA-4662: Kudu analysis failure for NULL literal in IN list
 # NULL literal in values list results in applying predicate at scan node
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
index b5f1b7330..ccc9b6b34 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
@@ -250,7 +250,7 @@ Per-Host Resources: mem-estimate=123.37MB mem-reservation=81.00MB thread-reserva
 |
 04:EXCHANGE [HASH(l_orderkey)]
 |  mem-estimate=10.04MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0 row-size=8B cardinality=6.00M
+|  tuple-ids=0 row-size=8B cardinality=5.76M(filtered from 6.00M)
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -263,7 +263,7 @@ Per-Host Resources: mem-estimate=81.14MB mem-reservation=5.00MB thread-reservati
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=8B cardinality=6.00M
+   tuple-ids=0 row-size=8B cardinality=5.76M(filtered from 6.00M)
    in pipelines: 00(GETNEXT)
 ====
 # High NDV aggregation.
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
index 41b53fb74..5edbde3f5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
@@ -10,7 +10,7 @@ PLAN-ROOT SINK
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
-|  mem-estimate=100.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=2 row-size=8B cardinality=1
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
@@ -30,7 +30,7 @@ PLAN-ROOT SINK
 00:SCAN KUDU [functional_kudu.alltypes a]
    runtime filters: RF000[min_max] -> a.string_col, RF001[min_max] -> a.int_col
    mem-estimate=1.50MB mem-reservation=0B thread-reservation=1
-   tuple-ids=0 row-size=21B cardinality=7.30K
+   tuple-ids=0 row-size=21B cardinality=5.84K(filtered from 7.30K)
    in pipelines: 00(GETNEXT)
 ====
 # Filters are not created if the target isn't a bare Kudu column or if 'is (not) distinct'
@@ -85,7 +85,7 @@ PLAN-ROOT SINK
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
-|  mem-estimate=100.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=2 row-size=8B cardinality=1
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
@@ -105,7 +105,7 @@ PLAN-ROOT SINK
 00:SCAN KUDU [functional_kudu.alltypes a]
    runtime filters: RF002[min_max] -> a.tinyint_col
    mem-estimate=3.00MB mem-reservation=0B thread-reservation=1
-   tuple-ids=0 row-size=26B cardinality=7.30K
+   tuple-ids=0 row-size=26B cardinality=5.84K(filtered from 7.30K)
    in pipelines: 00(GETNEXT)
 ====
 # Query with both Kudu and HDFS filter targets.
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test b/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
index 508a0c532..a8a6c3517 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
@@ -111,7 +111,7 @@ PLAN-ROOT SINK
 05:SCAN HDFS [tpch_nested_parquet.customer c]
    HDFS partitions=1/1 files=4 size=289.02MB
    runtime filters: RF000 -> c_nationkey, RF001 -> c.c_comment, RF004 -> c.c_nationkey, RF005 -> c_comment
-   row-size=87B cardinality=150.00K
+   row-size=87B cardinality=10.16K(filtered from 150.00K)
 ====
 # Test subplans: Cross join of parent and relative ref.
 select a.id, b.item from functional.allcomplextypes a cross join a.int_array_col b
@@ -2502,7 +2502,7 @@ PLAN-ROOT SINK
 |  |  01:SCAN HDFS [tpch_nested_parquet.customer t2]
 |  |     HDFS partitions=1/1 files=4 size=289.02MB
 |  |     runtime filters: RF004 -> t2.c_custkey
-|  |     row-size=59B cardinality=150.00K
+|  |     row-size=59B cardinality=5(filtered from 150.00K)
 |  |
 |  02:SCAN HDFS [tpch_nested_parquet.region t3]
 |     HDFS partitions=1/1 files=1 size=3.59KB
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/outer-to-inner-joins.test b/testdata/workloads/functional-planner/queries/PlannerTest/outer-to-inner-joins.test
index eba8cdb8d..5a9b7349c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/outer-to-inner-joins.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/outer-to-inner-joins.test
@@ -755,7 +755,7 @@ PLAN-ROOT SINK
 |     HDFS partitions=1824/1824 files=1824 size=346.60MB
 |     predicates: sts.ss_net_paid > 0, sts.ss_net_profit > 1, sts.ss_quantity > 0
 |     runtime filters: RF012 -> sts.ss_item_sk, RF013 -> sts.ss_ticket_number, RF016 -> ss_sold_date_sk
-|     row-size=32B cardinality=288.04K
+|     row-size=32B cardinality=170.55K(filtered from 288.04K)
 |
 |--22:SELECT
 |  |  predicates: (rank() <= 10 OR rank() <= 10)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test b/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
index b136d965d..638c8e9a0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
@@ -593,7 +593,7 @@ PLAN-ROOT SINK
    HDFS partitions=2/24 files=2 size=38.07KB
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.`month`, RF001 -> a.`year`, RF002 -> a.id, RF003 -> a.tinyint_col
-   row-size=13B cardinality=59
+   row-size=13B cardinality=49(filtered from 56)
 ---- SCANRANGELOCATIONS
 NODE 0:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypes/year=2009/month=1/090101.txt 0:20433
@@ -637,7 +637,7 @@ PLAN-ROOT SINK
    HDFS partitions=2/24 files=2 size=38.07KB
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.`month`, RF001 -> a.`year`, RF002 -> a.id, RF003 -> a.tinyint_col
-   row-size=13B cardinality=59
+   row-size=13B cardinality=49(filtered from 56)
 ====
 # Same as above but with cross join
 select straight_join a.id, b.id
@@ -680,7 +680,7 @@ PLAN-ROOT SINK
    HDFS partitions=2/24 files=2 size=38.07KB
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.`month`, RF001 -> a.`year`, RF002 -> a.id, RF003 -> a.tinyint_col
-   row-size=13B cardinality=59
+   row-size=13B cardinality=49(filtered from 56)
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
@@ -717,7 +717,7 @@ PLAN-ROOT SINK
    HDFS partitions=2/24 files=2 size=38.07KB
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.`month`, RF001 -> a.`year`, RF002 -> a.id, RF003 -> a.tinyint_col
-   row-size=13B cardinality=59
+   row-size=13B cardinality=49(filtered from 56)
 ====
 # no propagation into select block with limit;
 # propagation out of that block is okay;
@@ -761,7 +761,7 @@ PLAN-ROOT SINK
    HDFS partitions=2/24 files=2 size=38.07KB
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.`month`, RF001 -> a.`year`, RF002 -> a.id, RF003 -> a.tinyint_col
-   row-size=13B cardinality=59
+   row-size=13B cardinality=49(filtered from 56)
 ---- SCANRANGELOCATIONS
 NODE 0:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypes/year=2009/month=1/090101.txt 0:20433
@@ -813,7 +813,7 @@ PLAN-ROOT SINK
    HDFS partitions=2/24 files=2 size=38.07KB
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.`month`, RF001 -> a.`year`, RF002 -> a.id, RF003 -> a.tinyint_col
-   row-size=13B cardinality=59
+   row-size=13B cardinality=49(filtered from 56)
 ====
 # Similar to the above, converts the cross join to a hash join
 select straight_join a.id, b.id
@@ -860,7 +860,7 @@ PLAN-ROOT SINK
    HDFS partitions=2/24 files=2 size=38.07KB
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.`month`, RF001 -> a.`year`, RF002 -> a.id, RF003 -> a.tinyint_col
-   row-size=13B cardinality=59
+   row-size=13B cardinality=49(filtered from 56)
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
 |
@@ -903,7 +903,7 @@ PLAN-ROOT SINK
    HDFS partitions=2/24 files=2 size=38.07KB
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.`month`, RF001 -> a.`year`, RF002 -> a.id, RF003 -> a.tinyint_col
-   row-size=13B cardinality=59
+   row-size=13B cardinality=49(filtered from 56)
 ====
 # propagation of z.month=1 to alltypesagg is prevented
 select straight_join x.int_col, z.int_col
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
index abf0e1417..6994b3544 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -3352,7 +3352,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=2.14M
 |     mem-estimate=80.00MB mem-reservation=24.00MB thread-reservation=1
-|     tuple-ids=3 row-size=66B cardinality=6.00M
+|     tuple-ids=3 row-size=66B cardinality=1.15M(filtered from 6.00M)
 |     in pipelines: 05(GETNEXT)
 |
 04:AGGREGATE [FINALIZE]
@@ -3498,7 +3498,7 @@ Per-Host Resources: mem-estimate=151.74MB mem-reservation=60.00MB thread-reserva
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=2.14M
 |     mem-estimate=80.00MB mem-reservation=24.00MB thread-reservation=1
-|     tuple-ids=3 row-size=66B cardinality=6.00M
+|     tuple-ids=3 row-size=66B cardinality=1.15M(filtered from 6.00M)
 |     in pipelines: 05(GETNEXT)
 |
 14:AGGREGATE [FINALIZE]
@@ -3688,7 +3688,7 @@ Per-Instance Resources: mem-estimate=80.28MB mem-reservation=24.00MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=2.14M
 |     mem-estimate=80.00MB mem-reservation=24.00MB thread-reservation=0
-|     tuple-ids=3 row-size=66B cardinality=6.00M
+|     tuple-ids=3 row-size=66B cardinality=1.15M(filtered from 6.00M)
 |     in pipelines: 05(GETNEXT)
 |
 14:AGGREGATE [FINALIZE]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-cardinality-reduction-on-kudu.test b/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-cardinality-reduction-on-kudu.test
new file mode 100644
index 000000000..72f72e9dc
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-cardinality-reduction-on-kudu.test
@@ -0,0 +1,113 @@
+# Test query crafted similar to one in runtime-filter-cardinality-reduction.test
+# Actual run results in 00:SCAN node to return 262.38K.
+# With cardinality reduction disabled, 00:SCAN estimate cardinality is 6.00M.
+select l_partkey, l_suppkey, sum(o_orderkey) o_qty, sum(l_linenumber) l_qty
+from lineitem
+  left join orders on l_orderkey = o_orderkey
+  inner join supplier on l_suppkey = s_suppkey
+where s_nationkey = 11
+group by l_partkey, l_suppkey
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=0.0
+---- PLAN
+PLAN-ROOT SINK
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(o_orderkey), sum(l_linenumber)
+|  group by: l_partkey, l_suppkey
+|  row-size=32B cardinality=240.05K
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: l_suppkey = s_suppkey
+|  runtime filters: RF000 <- s_suppkey, RF001 <- s_suppkey
+|  row-size=44B cardinality=240.05K
+|
+|--02:SCAN KUDU [tpch_kudu.supplier]
+|     kudu predicates: s_nationkey = 11
+|     row-size=8B cardinality=400
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: l_orderkey = o_orderkey
+|  row-size=36B cardinality=6.00M
+|
+|--01:SCAN KUDU [tpch_kudu.orders]
+|     row-size=8B cardinality=1.50M
+|
+00:SCAN KUDU [tpch_kudu.lineitem]
+   runtime filters: RF000 -> l_suppkey, RF001 -> l_suppkey
+   row-size=28B cardinality=6.00M
+====
+# With cardinality reduction fully enabled, 00:SCAN estimate cardinality is 247.17K.
+select l_partkey, l_suppkey, sum(o_orderkey) o_qty, sum(l_linenumber) l_qty
+from lineitem
+  left join orders on l_orderkey = o_orderkey
+  inner join supplier on l_suppkey = s_suppkey
+where s_nationkey = 11
+group by l_partkey, l_suppkey
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=1.0
+---- PLAN
+PLAN-ROOT SINK
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(o_orderkey), sum(l_linenumber)
+|  group by: l_partkey, l_suppkey
+|  row-size=32B cardinality=240.05K
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: l_suppkey = s_suppkey
+|  runtime filters: RF000 <- s_suppkey, RF001 <- s_suppkey
+|  row-size=44B cardinality=240.05K
+|
+|--02:SCAN KUDU [tpch_kudu.supplier]
+|     kudu predicates: s_nationkey = 11
+|     row-size=8B cardinality=400
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: l_orderkey = o_orderkey
+|  row-size=36B cardinality=247.17K(filtered from 6.00M)
+|
+|--01:SCAN KUDU [tpch_kudu.orders]
+|     row-size=8B cardinality=1.50M
+|
+00:SCAN KUDU [tpch_kudu.lineitem]
+   runtime filters: RF000 -> l_suppkey, RF001 -> l_suppkey
+   row-size=28B cardinality=247.17K(filtered from 6.00M)
+====
+# With cardinality reduction applied half, 00:SCAN estimate cardinality is 3.12M.
+select l_partkey, l_suppkey, sum(o_orderkey) o_qty, sum(l_linenumber) l_qty
+from lineitem
+  left join orders on l_orderkey = o_orderkey
+  inner join supplier on l_suppkey = s_suppkey
+where s_nationkey = 11
+group by l_partkey, l_suppkey
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=0.5
+---- PLAN
+PLAN-ROOT SINK
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(o_orderkey), sum(l_linenumber)
+|  group by: l_partkey, l_suppkey
+|  row-size=32B cardinality=240.05K
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: l_suppkey = s_suppkey
+|  runtime filters: RF000 <- s_suppkey, RF001 <- s_suppkey
+|  row-size=44B cardinality=240.05K
+|
+|--02:SCAN KUDU [tpch_kudu.supplier]
+|     kudu predicates: s_nationkey = 11
+|     row-size=8B cardinality=400
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: l_orderkey = o_orderkey
+|  row-size=36B cardinality=3.12M(filtered from 6.00M)
+|
+|--01:SCAN KUDU [tpch_kudu.orders]
+|     row-size=8B cardinality=1.50M
+|
+00:SCAN KUDU [tpch_kudu.lineitem]
+   runtime filters: RF000 -> l_suppkey, RF001 -> l_suppkey
+   row-size=28B cardinality=3.12M(filtered from 6.00M)
+====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-cardinality-reduction.test b/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-cardinality-reduction.test
new file mode 100644
index 000000000..822d6d7bc
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-cardinality-reduction.test
@@ -0,0 +1,792 @@
+# Case 01: This is a base case, a modified subquery from TPC-DS Q78.
+# Actual run results in 00:SCAN node to return 549.33K rows.
+# With cardinality reduction disabled, 00:SCAN estimate cardinality is 2.88M.
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from store_sales
+left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+join date_dim on ss_sold_date_sk = d_date_sk
+where sr_ticket_number is null
+  and d_year = 2002
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=0.0
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=195.94MB mem-reservation=57.94MB thread-reservation=4 runtime-filters-memory=1.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=31.46MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  group by: d_year, ss_item_sk, ss_customer_sk
+|  mem-estimate=38.20MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=589.03K
+|  in pipelines: 05(GETNEXT), 00(OPEN)
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  runtime filters: RF000[bloom] <- d_date_sk, RF001[min_max] <- d_date_sk
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2 row-size=60B cardinality=589.03K
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--02:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=2 row-size=8B cardinality=373
+|     in pipelines: 02(GETNEXT)
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  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
+|  other predicates: sr_ticket_number IS NULL
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0,1N row-size=52B cardinality=2.88M
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
+|     tuple-ids=1 row-size=16B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF001[min_max] -> ss_sold_date_sk, RF000[bloom] -> ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
+   tuple-ids=0 row-size=36B cardinality=2.88M
+   in pipelines: 00(GETNEXT)
+====
+# Case 02: With cardinality reduction fully enabled, 00:SCAN estimate cardinality is 589.03K.
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from store_sales
+left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+join date_dim on ss_sold_date_sk = d_date_sk
+where sr_ticket_number is null
+  and d_year = 2002
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=1.0
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=195.94MB mem-reservation=57.94MB thread-reservation=4 runtime-filters-memory=1.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=31.46MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  group by: d_year, ss_item_sk, ss_customer_sk
+|  mem-estimate=38.20MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=589.03K
+|  in pipelines: 05(GETNEXT), 00(OPEN)
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  runtime filters: RF000[bloom] <- d_date_sk, RF001[min_max] <- d_date_sk
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2 row-size=60B cardinality=589.03K
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--02:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=2 row-size=8B cardinality=373
+|     in pipelines: 02(GETNEXT)
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  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
+|  other predicates: sr_ticket_number IS NULL
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0,1N row-size=52B cardinality=589.03K(filtered from 2.88M)
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
+|     tuple-ids=1 row-size=16B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF001[min_max] -> ss_sold_date_sk, RF000[bloom] -> ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
+   mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
+   in pipelines: 00(GETNEXT)
+====
+# Case 03: With cardinality reduction applied half, 00:SCAN estimate cardinality is 1.73M.
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from store_sales
+left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+join date_dim on ss_sold_date_sk = d_date_sk
+where sr_ticket_number is null
+  and d_year = 2002
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=0.5
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=195.94MB mem-reservation=57.94MB thread-reservation=4 runtime-filters-memory=1.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=31.46MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  group by: d_year, ss_item_sk, ss_customer_sk
+|  mem-estimate=38.20MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=589.03K
+|  in pipelines: 05(GETNEXT), 00(OPEN)
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  runtime filters: RF000[bloom] <- d_date_sk, RF001[min_max] <- d_date_sk
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2 row-size=60B cardinality=589.03K
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--02:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=2 row-size=8B cardinality=373
+|     in pipelines: 02(GETNEXT)
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  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
+|  other predicates: sr_ticket_number IS NULL
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0,1N row-size=52B cardinality=1.73M(filtered from 2.88M)
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
+|     tuple-ids=1 row-size=16B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF001[min_max] -> ss_sold_date_sk, RF000[bloom] -> ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1099(filtered from 1824)
+   mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
+   tuple-ids=0 row-size=36B cardinality=1.73M(filtered from 2.88M)
+   in pipelines: 00(GETNEXT)
+====
+# Case 04: Removing sr_ticket_number=ss_ticket_number predicate will turn 03:HASH JOIN into
+# an expanding join and makes probe pipeline ineligible for cardinality reduction.
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from store_sales
+left join store_returns on ss_item_sk=sr_item_sk
+join date_dim on ss_sold_date_sk = d_date_sk
+where sr_ticket_number is null
+  and d_year = 2002
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=1.0
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=637.75MB mem-reservation=55.94MB thread-reservation=4 runtime-filters-memory=1.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  group by: d_year, ss_item_sk, ss_customer_sk
+|  mem-estimate=536.75MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=9.42M
+|  in pipelines: 05(GETNEXT), 00(OPEN)
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  runtime filters: RF000[bloom] <- d_date_sk, RF001[min_max] <- d_date_sk
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2 row-size=52B cardinality=9.42M
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--02:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=2 row-size=8B cardinality=373
+|     in pipelines: 02(GETNEXT)
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  hash predicates: ss_item_sk = sr_item_sk
+|  fk/pk conjuncts: none
+|  other predicates: sr_ticket_number IS NULL
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0,1N row-size=44B cardinality=46.07M
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
+|     tuple-ids=1 row-size=16B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF001[min_max] -> ss_sold_date_sk, RF000[bloom] -> ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=1
+   tuple-ids=0 row-size=28B cardinality=2.88M
+   in pipelines: 00(GETNEXT)
+====
+# Case 05: Add sr_returned_date_sk = d_date_sk predicate so that build side scan (01:SCAN HDFS)
+# is targetted by a runtime filter. No cardinality reduction should happen in build scan
+# because cardinality reduction only applied at probe pipeline.
+# TODO: RF000[bloom] is duplicate of RF001[bloom].
+#       RF002[min_max] is duplicate of RF003[min_max].
+#       Consider deduplicating it in the future.
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from store_sales
+left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+join date_dim on ss_sold_date_sk = d_date_sk and sr_returned_date_sk = d_date_sk
+where sr_ticket_number is null
+  and d_year = 2002
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=1.0
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=220.94MB mem-reservation=58.94MB thread-reservation=4 runtime-filters-memory=2.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=28.79MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  group by: d_year, ss_item_sk, ss_customer_sk
+|  mem-estimate=34.96MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=539.08K
+|  in pipelines: 05(GETNEXT), 00(OPEN)
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: sr_returned_date_sk = d_date_sk, ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: sr_returned_date_sk = d_date_sk, ss_sold_date_sk = d_date_sk
+|  runtime filters: RF000[bloom] <- d_date_sk, RF001[bloom] <- d_date_sk, RF002[min_max] <- 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,1N,2 row-size=64B cardinality=539.08K
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--02:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=2 row-size=8B cardinality=373
+|     in pipelines: 02(GETNEXT)
+|
+03:HASH JOIN [LEFT OUTER JOIN]
+|  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
+|  other predicates: sr_ticket_number IS NULL
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0,1N row-size=56B cardinality=589.03K(filtered from 2.88M)
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     runtime filters: RF000[bloom] -> sr_returned_date_sk, RF001[bloom] -> tpcds_parquet.store_returns.sr_returned_date_sk
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=72.00MB mem-reservation=4.00MB thread-reservation=1
+|     tuple-ids=1 row-size=20B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF002[min_max] -> tpcds_parquet.store_sales.ss_sold_date_sk, RF003[min_max] -> ss_sold_date_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_sold_date_sk, RF001[bloom] -> ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
+   mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
+   in pipelines: 00(GETNEXT)
+====
+# Case 06: Additionally join against time_dim. 00:SCAN will receive 2 bloom runtime filter
+# and estimate cardinality is 61 (actual is 91).
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from store_sales
+left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+join date_dim on ss_sold_date_sk = d_date_sk
+join time_dim on ss_sold_time_sk = t_time_sk
+where sr_ticket_number is null
+  and d_year = 2002
+  and t_time = 36360
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=1.0
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=246.88MB mem-reservation=32.38MB thread-reservation=5 runtime-filters-memory=2.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+07:AGGREGATE [FINALIZE]
+|  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  group by: d_year, ss_item_sk, ss_customer_sk
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=4 row-size=56B cardinality=13
+|  in pipelines: 07(GETNEXT), 00(OPEN)
+|
+06:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  runtime filters: RF000[bloom] <- d_date_sk, RF001[min_max] <- d_date_sk
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,3,2 row-size=72B cardinality=13
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--02:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=2 row-size=8B cardinality=373
+|     in pipelines: 02(GETNEXT)
+|
+05:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_sold_time_sk = t_time_sk
+|  fk/pk conjuncts: ss_sold_time_sk = t_time_sk
+|  runtime filters: RF002[bloom] <- t_time_sk
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,3 row-size=64B cardinality=61(filtered from 62)
+|  in pipelines: 00(GETNEXT), 03(OPEN)
+|
+|--03:SCAN HDFS [tpcds_parquet.time_dim]
+|     HDFS partitions=1/1 files=1 size=1.31MB
+|     predicates: t_time = CAST(36360 AS INT)
+|     stored statistics:
+|       table: rows=86.40K size=1.31MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=86.40K
+|     parquet statistics predicates: t_time = CAST(36360 AS INT)
+|     parquet dictionary predicates: t_time = CAST(36360 AS INT)
+|     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
+|     tuple-ids=3 row-size=8B cardinality=1
+|     in pipelines: 03(GETNEXT)
+|
+04:HASH JOIN [LEFT OUTER JOIN]
+|  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
+|  other predicates: sr_ticket_number IS NULL
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0,1N row-size=56B cardinality=61(filtered from 2.88M)
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
+|     tuple-ids=1 row-size=16B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF001[min_max] -> ss_sold_date_sk, RF000[bloom] -> ss_sold_date_sk, RF002[bloom] -> ss_sold_time_sk
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
+   mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
+   tuple-ids=0 row-size=40B cardinality=61(filtered from 2.88M)
+   in pipelines: 00(GETNEXT)
+====
+# Case 07: Same as Case 06, but make the RHS of join predicate an expression.
+# No cardinality reduction should happen.
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from store_sales
+left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+join date_dim on ss_sold_date_sk = (d_date_sk + 1)
+join time_dim on ss_sold_time_sk = (t_time_sk - 1)
+where sr_ticket_number is null
+  and d_year = 2002
+  and t_time = 36360
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=1.0
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=321.67MB mem-reservation=60.88MB thread-reservation=5 runtime-filters-memory=2.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+07:AGGREGATE [FINALIZE]
+|  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  group by: d_year, ss_item_sk, ss_customer_sk
+|  mem-estimate=186.79MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=4 row-size=56B cardinality=2.88M
+|  in pipelines: 07(GETNEXT), 00(OPEN)
+|
+06:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_sold_time_sk = (t_time_sk - 1)
+|  fk/pk conjuncts: assumed fk/pk
+|  runtime filters: RF000[bloom] <- (t_time_sk - 1)
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2,3 row-size=72B cardinality=2.88M
+|  in pipelines: 00(GETNEXT), 03(OPEN)
+|
+|--03:SCAN HDFS [tpcds_parquet.time_dim]
+|     HDFS partitions=1/1 files=1 size=1.31MB
+|     predicates: t_time = CAST(36360 AS INT)
+|     stored statistics:
+|       table: rows=86.40K size=1.31MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=86.40K
+|     parquet statistics predicates: t_time = CAST(36360 AS INT)
+|     parquet dictionary predicates: t_time = CAST(36360 AS INT)
+|     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
+|     tuple-ids=3 row-size=8B cardinality=1
+|     in pipelines: 03(GETNEXT)
+|
+05:HASH JOIN [INNER JOIN]
+|  hash predicates: ss_sold_date_sk = (d_date_sk + 1)
+|  fk/pk conjuncts: assumed fk/pk
+|  runtime filters: RF002[bloom] <- (d_date_sk + 1), RF003[min_max] <- (d_date_sk + 1)
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2 row-size=64B cardinality=2.88M
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--02:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=2 row-size=8B cardinality=373
+|     in pipelines: 02(GETNEXT)
+|
+04:HASH JOIN [LEFT OUTER JOIN]
+|  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
+|  other predicates: sr_ticket_number IS NULL
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0,1N row-size=56B cardinality=2.88M
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
+|     tuple-ids=1 row-size=16B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF003[min_max] -> ss_sold_date_sk, RF000[bloom] -> ss_sold_time_sk, RF002[bloom] -> ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
+   tuple-ids=0 row-size=40B cardinality=2.88M
+   in pipelines: 00(GETNEXT)
+====
+# Case 08: Same as Case 06, but make the LHS of join predicate an expression.
+# No cardinality reduction should happen.
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from store_sales
+left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk
+join date_dim on (ss_sold_date_sk + 1) = d_date_sk
+join time_dim on (ss_sold_time_sk - 1) = t_time_sk
+where sr_ticket_number is null
+  and d_year = 2002
+  and t_time = 36360
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=1.0
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=321.67MB mem-reservation=60.88MB thread-reservation=5 runtime-filters-memory=2.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+07:AGGREGATE [FINALIZE]
+|  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  group by: d_year, ss_item_sk, ss_customer_sk
+|  mem-estimate=186.79MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=4 row-size=56B cardinality=2.88M
+|  in pipelines: 07(GETNEXT), 00(OPEN)
+|
+06:HASH JOIN [INNER JOIN]
+|  hash predicates: (ss_sold_time_sk - 1) = t_time_sk
+|  fk/pk conjuncts: assumed fk/pk
+|  runtime filters: RF000[bloom] <- t_time_sk
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2,3 row-size=72B cardinality=2.88M
+|  in pipelines: 00(GETNEXT), 03(OPEN)
+|
+|--03:SCAN HDFS [tpcds_parquet.time_dim]
+|     HDFS partitions=1/1 files=1 size=1.31MB
+|     predicates: t_time = CAST(36360 AS INT)
+|     stored statistics:
+|       table: rows=86.40K size=1.31MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=86.40K
+|     parquet statistics predicates: t_time = CAST(36360 AS INT)
+|     parquet dictionary predicates: t_time = CAST(36360 AS INT)
+|     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
+|     tuple-ids=3 row-size=8B cardinality=1
+|     in pipelines: 03(GETNEXT)
+|
+05:HASH JOIN [INNER JOIN]
+|  hash predicates: (ss_sold_date_sk + 1) = d_date_sk
+|  fk/pk conjuncts: assumed fk/pk
+|  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,2 row-size=64B cardinality=2.88M
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--02:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=2 row-size=8B cardinality=373
+|     in pipelines: 02(GETNEXT)
+|
+04:HASH JOIN [LEFT OUTER JOIN]
+|  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
+|  other predicates: sr_ticket_number IS NULL
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0,1N row-size=56B cardinality=2.88M
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
+|     tuple-ids=1 row-size=16B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF000[bloom] -> (ss_sold_time_sk - 1), RF002[bloom] -> (ss_sold_date_sk + 1)
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
+   tuple-ids=0 row-size=40B cardinality=2.88M
+   in pipelines: 00(GETNEXT)
+====
+# Case 09: Test against left anti join (a variant of left semi join).
+select d_year AS ss_sold_year, ss_item_sk,
+  ss_customer_sk,
+  sum(ss_quantity) ss_qty,
+  sum(ss_wholesale_cost) ss_wc,
+  sum(ss_sales_price) ss_sp
+from (
+  select * from store_sales
+  where ss_ticket_number not in (
+    select sr_ticket_number from store_returns)
+) ss
+join date_dim on ss_sold_date_sk = d_date_sk
+where d_year = 2002
+group by d_year, ss_item_sk, ss_customer_sk
+---- QUERYOPTIONS
+runtime_filter_cardinality_reduction_scale=1.0
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=174.94MB mem-reservation=60.94MB thread-reservation=4 runtime-filters-memory=1.00MB
+PLAN-ROOT SINK
+|  output exprs: d_year, ss_item_sk, ss_customer_sk, sum(ss_quantity), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  mem-estimate=31.46MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|
+05:AGGREGATE [FINALIZE]
+|  output: sum(CAST(tpcds_parquet.store_sales.ss_quantity AS BIGINT)), sum(tpcds_parquet.store_sales.ss_wholesale_cost), sum(tpcds_parquet.store_sales.ss_sales_price)
+|  group by: d_year, tpcds_parquet.store_sales.ss_item_sk, tpcds_parquet.store_sales.ss_customer_sk
+|  mem-estimate=38.20MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=5 row-size=56B cardinality=589.03K
+|  in pipelines: 05(GETNEXT), 00(OPEN)
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: tpcds_parquet.store_sales.ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: tpcds_parquet.store_sales.ss_sold_date_sk = d_date_sk
+|  runtime filters: RF000[bloom] <- d_date_sk, RF001[min_max] <- d_date_sk
+|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,4 row-size=44B cardinality=589.03K
+|  in pipelines: 00(GETNEXT), 03(OPEN)
+|
+|--03:SCAN HDFS [tpcds_parquet.date_dim]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2002 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2002 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2002 AS INT)
+|     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
+|     tuple-ids=4 row-size=8B cardinality=373
+|     in pipelines: 03(GETNEXT)
+|
+02:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
+|  hash predicates: ss_ticket_number = sr_ticket_number
+|  mem-estimate=20.00MB mem-reservation=20.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--01:SCAN HDFS [tpcds_parquet.store_returns]
+|     HDFS partitions=1/1 files=1 size=15.43MB
+|     stored statistics:
+|       table: rows=287.51K size=15.43MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|     mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1
+|     tuple-ids=1 row-size=8B cardinality=287.51K
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_sold_date_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=199.46MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
+   mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
+   in pipelines: 00(GETNEXT)
+====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
index df589a1b2..5ec35fb44 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
@@ -799,7 +799,7 @@ Per-Host Resources: mem-estimate=111.37MB mem-reservation=69.00MB thread-reserva
 |
 04:EXCHANGE [HASH(l_orderkey)]
 |  mem-estimate=10.04MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0 row-size=8B cardinality=6.00M
+|  tuple-ids=0 row-size=8B cardinality=5.76M(filtered from 6.00M)
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -812,7 +812,7 @@ Per-Host Resources: mem-estimate=81.14MB mem-reservation=5.00MB thread-reservati
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=8B cardinality=6.00M
+   tuple-ids=0 row-size=8B cardinality=5.76M(filtered from 6.00M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=120.00MB Threads=6
@@ -892,7 +892,7 @@ Per-Instance Resources: mem-estimate=66.70MB mem-reservation=34.00MB thread-rese
 |
 04:EXCHANGE [HASH(l_orderkey)]
 |  mem-estimate=10.04MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0 row-size=8B cardinality=6.00M
+|  tuple-ids=0 row-size=8B cardinality=5.76M(filtered from 6.00M)
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -906,7 +906,7 @@ Per-Instance Resources: mem-estimate=80.14MB mem-reservation=4.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=8B cardinality=6.00M
+   tuple-ids=0 row-size=8B cardinality=5.76M(filtered from 6.00M)
    in pipelines: 00(GETNEXT)
 ====
 # High NDV aggregation - should use default buffer size.
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-dist-method.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-dist-method.test
index 72d1bb04c..19122d4af 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-dist-method.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-dist-method.test
@@ -95,7 +95,7 @@ PLAN-ROOT SINK
 |  |  03:HASH JOIN [LEFT OUTER JOIN]
 |  |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  |  other predicates: sr_ticket_number IS NULL
-|  |  |  row-size=52B cardinality=2.88M
+|  |  |  row-size=52B cardinality=589.03K(filtered from 2.88M)
 |  |  |
 |  |  |--01:SCAN HDFS [tpcds.store_returns]
 |  |  |     HDFS partitions=1/1 files=1 size=31.19MB
@@ -104,7 +104,7 @@ PLAN-ROOT SINK
 |  |  00:SCAN HDFS [tpcds.store_sales]
 |  |     HDFS partitions=1824/1824 files=1824 size=346.60MB
 |  |     runtime filters: RF016 -> ss_sold_date_sk
-|  |     row-size=36B cardinality=2.88M
+|  |     row-size=36B cardinality=589.03K(filtered from 2.88M)
 |  |
 |  11:AGGREGATE [FINALIZE]
 |  |  output: sum(ws_quantity), sum(ws_wholesale_cost), sum(ws_sales_price)
@@ -233,7 +233,7 @@ PLAN-ROOT SINK
 |  |  03:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
 |  |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  |  other predicates: sr_ticket_number IS NULL
-|  |  |  row-size=52B cardinality=2.88M
+|  |  |  row-size=52B cardinality=589.03K(filtered from 2.88M)
 |  |  |
 |  |  |--JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
@@ -250,7 +250,7 @@ PLAN-ROOT SINK
 |  |  00:SCAN HDFS [tpcds.store_sales]
 |  |     HDFS partitions=1824/1824 files=1824 size=346.60MB
 |  |     runtime filters: RF016 -> ss_sold_date_sk
-|  |     row-size=36B cardinality=2.88M
+|  |     row-size=36B cardinality=589.03K(filtered from 2.88M)
 |  |
 |  28:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_quantity), sum:merge(ws_wholesale_cost), sum:merge(ws_sales_price)
@@ -417,7 +417,7 @@ PLAN-ROOT SINK
 |  |  03:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
 |  |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  |  other predicates: sr_ticket_number IS NULL
-|  |  |  row-size=52B cardinality=2.88M
+|  |  |  row-size=52B cardinality=589.03K(filtered from 2.88M)
 |  |  |
 |  |  |--JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
@@ -434,7 +434,7 @@ PLAN-ROOT SINK
 |  |  00:SCAN HDFS [tpcds.store_sales]
 |  |     HDFS partitions=1824/1824 files=1824 size=346.60MB
 |  |     runtime filters: RF016 -> ss_sold_date_sk
-|  |     row-size=36B cardinality=2.88M
+|  |     row-size=36B cardinality=589.03K(filtered from 2.88M)
 |  |
 |  28:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(ws_quantity), sum:merge(ws_wholesale_cost), sum:merge(ws_sales_price)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
index bd11760d4..8ce1dfeff 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
@@ -147,13 +147,13 @@ max-parallelism=6 segment-costs=[15408, 167] cpu-comparison-result=18 [max(18 (s
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 Per-Instance Resources: mem-estimate=17.60MB mem-reservation=1.00MB thread-reservation=1
-max-parallelism=12 segment-costs=[94125607]
+max-parallelism=12 segment-costs=[91203296]
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=01
 |  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=3.04K cost=2880404
+|  tuple-ids=1,2 row-size=60B cardinality=3.04K cost=3045
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -187,15 +187,15 @@ max-parallelism=12 segment-costs=[94125607]
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [tpcds_partitioned_parquet_snap.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   HDFS partitions=1824/1824 files=1824 size=199.36MB
    runtime filters: RF003[min_max] -> store_sales.ss_item_sk, RF000[bloom] -> store_sales.ss_sold_date_sk, RF002[bloom] -> store_sales.ss_item_sk
    stored statistics:
-     table: rows=2.88M size=200.96MB
+     table: rows=2.88M size=199.36MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=1 row-size=16B cardinality=2.88M cost=91245000
+   tuple-ids=1 row-size=16B cardinality=3.04K(filtered from 2.88M) cost=91200048
    in pipelines: 01(GETNEXT)
 ====
 # TPCDS-Q14a first of two
@@ -301,8 +301,8 @@ with  cross_items as
  order by channel,i_brand_id,i_class_id,i_category_id
 LIMIT 100
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=890.31MB Threads=212
-Per-Host Resource Estimates: Memory=3.47GB
+Max Per-Host Resource Reservation: Memory=846.44MB Threads=204
+Per-Host Resource Estimates: Memory=3.21GB
 F82:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 |  max-parallelism=1 segment-costs=[606] cpu-comparison-result=258 [max(1 (self) vs 258 (sum children))]
@@ -318,7 +318,7 @@ PLAN-ROOT SINK
 |  in pipelines: 129(GETNEXT)
 |
 F81:PLAN FRAGMENT [HASH(CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(channel) WHEN 105 THEN murmur_hash(channel) WHEN 106 THEN murmur_hash(channel) WHEN 107 THEN murmur_hash(channel) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(i_brand_id) WHEN 105 THEN murmur_hash(i_brand_id) WHEN 106 THEN murmur_hash(i_brand_id) WHEN 107 THEN murmur_hash(NULL) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WH [...]
-Per-Instance Resources: mem-estimate=70.05MB mem-reservation=30.44MB thread-reservation=1
+Per-Instance Resources: mem-estimate=68.52MB mem-reservation=30.44MB thread-reservation=1
 max-parallelism=6 segment-costs=[2025384, 2267013, 400, 6] cpu-comparison-result=258 [max(6 (self) vs 258 (sum children))]
 129:TOP-N [LIMIT=100]
 |  order by: CASE valid_tid(104,105,106,107,108) WHEN 104 THEN channel WHEN 105 THEN channel WHEN 106 THEN channel WHEN 107 THEN channel WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_brand_id WHEN 105 THEN i_brand_id WHEN 106 THEN i_brand_id WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_class_id WHEN 105 THEN i_class_id WHEN 106 THEN NULL WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(10 [...]
@@ -354,13 +354,13 @@ max-parallelism=6 segment-costs=[2025384, 2267013, 400, 6] cpu-comparison-result
 |  in pipelines: 215(GETNEXT), 151(OPEN), 179(OPEN), 207(OPEN)
 |
 214:EXCHANGE [HASH(CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(channel) WHEN 105 THEN murmur_hash(channel) WHEN 106 THEN murmur_hash(channel) WHEN 107 THEN murmur_hash(channel) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(i_brand_id) WHEN 105 THEN murmur_hash(i_brand_id) WHEN 106 THEN murmur_hash(i_brand_id) WHEN 107 THEN murmur_hash(NULL) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 10 [...]
-|  mem-estimate=13.05MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=11.52MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=104N,105N,106N,107N,108N row-size=240B cardinality=323.86K cost=82230
 |  in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT)
 |
-F80:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
-Per-Instance Resources: mem-estimate=68.27MB mem-reservation=25.94MB thread-reservation=1
-max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-comparison-result=258 [max(12 (self) vs 258 (sum children))]
+F80:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
+Per-Instance Resources: mem-estimate=82.03MB mem-reservation=42.88MB thread-reservation=1
+max-parallelism=6 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-comparison-result=258 [max(6 (self) vs 258 (sum children))]
 127:AGGREGATE [STREAMING]
 |  Class 0
 |    output: sum(sales), sum(number_sales)
@@ -377,7 +377,7 @@ max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-c
 |  Class 4
 |    output: sum(sales), sum(number_sales)
 |    group by: NULL, NULL, NULL, NULL
-|  mem-estimate=50.00MB mem-reservation=24.00MB thread-reservation=0
+|  mem-estimate=64.00MB mem-reservation=40.00MB thread-reservation=0
 |  tuple-ids=104N,105N,106N,107N,108N row-size=240B cardinality=323.86K cost=8292750
 |  in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT)
 |
@@ -590,8 +590,8 @@ max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-c
 |  |  in pipelines: 85(GETNEXT)
 |  |
 |  F56:PLAN FRAGMENT [HASH(ws_item_sk)] hosts=3 instances=6 (adjusted from 48)
-|  Per-Instance Resources: mem-estimate=12.78MB mem-reservation=2.00MB thread-reservation=1
-|  max-parallelism=6 segment-costs=[299836, 1664] cpu-comparison-result=54 [max(18 (self) vs 54 (sum children))]
+|  Per-Instance Resources: mem-estimate=11.61MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=6 segment-costs=[299836, 1664] cpu-comparison-result=54 [max(12 (self) vs 54 (sum children))]
 |  114:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price), count(*)
 |  |  group by: i_brand_id, i_class_id, i_category_id
@@ -1066,30 +1066,30 @@ max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-c
 |  |     in pipelines: 86(GETNEXT)
 |  |
 |  187:EXCHANGE [HASH(ws_item_sk)]
-|  |  mem-estimate=923.71KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=683.71KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=68,70 row-size=32B cardinality=42.60K cost=1664
 |  |  in pipelines: 85(GETNEXT)
 |  |
-|  F53:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F53:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=16.94MB mem-reservation=128.00KB thread-reservation=1
-|  max-parallelism=12 segment-costs=[91935096]
+|  max-parallelism=6 segment-costs=[5495092]
 |  110:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=47
 |  |  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=68,70 row-size=32B cardinality=42.60K cost=719384
+|  |  tuple-ids=68,70 row-size=32B cardinality=42.60K cost=42596
 |  |  in pipelines: 85(GETNEXT), 87(OPEN)
 |  |
 |  |--F130:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=47 plan-id=48 cohort-id=01
 |  |  |  build expressions: d_date_sk
 |  |  |  runtime filters: RF076[bloom] <- d_date_sk
-|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=108
 |  |  |
 |  |  186:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -1119,9 +1119,9 @@ max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-c
 |       table: rows=719.38K size=84.20MB
 |       partitions: 1824/1824 rows=719.38K
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=1.12K
+|     extrapolated-rows=disabled max-scan-range-rows=1.12K est-scan-range=109(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-|     tuple-ids=68 row-size=20B cardinality=719.38K cost=91214048
+|     tuple-ids=68 row-size=20B cardinality=42.60K(filtered from 719.38K) cost=5450832
 |     in pipelines: 85(GETNEXT)
 |
 |--84:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -1328,8 +1328,8 @@ max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-c
 |  |  in pipelines: 43(GETNEXT)
 |  |
 |  F29:PLAN FRAGMENT [HASH(cs_item_sk)] hosts=3 instances=6 (adjusted from 48)
-|  Per-Instance Resources: mem-estimate=13.21MB mem-reservation=2.00MB thread-reservation=1
-|  max-parallelism=6 segment-costs=[598525, 3322] cpu-comparison-result=54 [max(18 (self) vs 54 (sum children))]
+|  Per-Instance Resources: mem-estimate=12.04MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=6 segment-costs=[598525, 3322] cpu-comparison-result=54 [max(12 (self) vs 54 (sum children))]
 |  72:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price), count(*)
 |  |  group by: i_brand_id, i_class_id, i_category_id
@@ -1804,30 +1804,30 @@ max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-c
 |  |     in pipelines: 44(GETNEXT)
 |  |
 |  159:EXCHANGE [HASH(cs_item_sk)]
-|  |  mem-estimate=1.33MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=1.10MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=34,36 row-size=32B cardinality=85.03K cost=3322
 |  |  in pipelines: 43(GETNEXT)
 |  |
-|  F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=16.94MB mem-reservation=128.00KB thread-reservation=1
-|  max-parallelism=12 segment-costs=[93023030]
+|  max-parallelism=6 segment-costs=[5540012]
 |  68:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=35
 |  |  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=34,36 row-size=32B cardinality=85.03K cost=1441548
+|  |  tuple-ids=34,36 row-size=32B cardinality=85.03K cost=85029
 |  |  in pipelines: 43(GETNEXT), 45(OPEN)
 |  |
 |  |--F118:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=35 plan-id=36 cohort-id=01
 |  |  |  build expressions: d_date_sk
 |  |  |  runtime filters: RF040[bloom] <- d_date_sk
-|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=108
 |  |  |
 |  |  158:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -1857,9 +1857,9 @@ max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-c
 |       table: rows=1.44M size=151.13MB
 |       partitions: 1831/1831 rows=1.44M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=7.18K
+|     extrapolated-rows=disabled max-scan-range-rows=7.18K est-scan-range=109(filtered from 1831)
 |     mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-|     tuple-ids=34 row-size=20B cardinality=1.44M cost=91578160
+|     tuple-ids=34 row-size=20B cardinality=85.03K(filtered from 1.44M) cost=5451661
 |     in pipelines: 43(GETNEXT)
 |
 42:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -2056,19 +2056,19 @@ max-parallelism=12 segment-costs=[749813, 428467, 214644, 13268400, 82230] cpu-c
 151:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_quantity * ss_list_price), count:merge(*)
 |  group by: i_brand_id, i_class_id, i_category_id
-|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  tuple-ids=20 row-size=36B cardinality=148.80K cost=744000
 |  in pipelines: 151(GETNEXT), 01(OPEN)
 |
 150:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
-|  mem-estimate=2.17MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=1.94MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=20 row-size=36B cardinality=148.80K cost=5813
 |  in pipelines: 01(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
-Per-Instance Resources: mem-estimate=27.88MB mem-reservation=3.00MB thread-reservation=1
-max-parallelism=12 segment-costs=[95330517, 5813] cpu-comparison-result=51 [max(12 (self) vs 51 (sum children))]
+Per-Instance Resources: mem-estimate=26.94MB mem-reservation=3.00MB thread-reservation=1
+max-parallelism=6 segment-costs=[6817739, 5813] cpu-comparison-result=51 [max(6 (self) vs 51 (sum children))]
 30:AGGREGATE [STREAMING]
 |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*)
 |  group by: i_brand_id, i_class_id, i_category_id
@@ -2084,13 +2084,13 @@ max-parallelism=12 segment-costs=[95330517, 5813] cpu-comparison-result=51 [max(
 |  in pipelines: 01(GETNEXT), 148(OPEN)
 |
 |--F95:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=8.96MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=5.08MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  max-parallelism=3 segment-costs=[18608] cpu-comparison-result=43 [max(3 (self) vs 43 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=12 plan-id=13 cohort-id=01
 |  |  build expressions: tpcds_partitioned_parquet_snap.item.i_item_sk
 |  |  runtime filters: RF000[bloom] <- tpcds_partitioned_parquet_snap.item.i_item_sk, RF001[min_max] <- tpcds_partitioned_parquet_snap.item.i_item_sk
-|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=17975
 |  |
 |  149:EXCHANGE [BROADCAST]
 |  |  mem-estimate=212.43KB mem-reservation=0B thread-reservation=0
@@ -2525,13 +2525,13 @@ max-parallelism=12 segment-costs=[95330517, 5813] cpu-comparison-result=51 [max(
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F105:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=9.12MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=5.24MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  max-parallelism=3 segment-costs=[19266]
 |  JOIN BUILD
 |  |  join-table-id=22 plan-id=23 cohort-id=01
 |  |  build expressions: i_item_sk
 |  |  runtime filters: RF002[bloom] <- i_item_sk, RF003[min_max] <- i_item_sk
-|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=18000
 |  |
 |  131:EXCHANGE [BROADCAST]
 |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
@@ -2558,17 +2558,17 @@ max-parallelism=12 segment-costs=[95330517, 5813] cpu-comparison-result=51 [max(
 |  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,2 row-size=32B cardinality=170.55K cost=2880404
+|  tuple-ids=0,2 row-size=32B cardinality=170.55K cost=170551
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--F106:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  max-parallelism=3 segment-costs=[114]
 |  JOIN BUILD
 |  |  join-table-id=23 plan-id=24 cohort-id=01
 |  |  build expressions: d_date_sk
 |  |  runtime filters: RF004[bloom] <- d_date_sk
-|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=108
 |  |
 |  130:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -2598,9 +2598,9 @@ max-parallelism=12 segment-costs=[95330517, 5813] cpu-comparison-result=51 [max(
      table: rows=2.88M size=199.36MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=2.88M cost=91256256
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M) cost=5453331
    in pipelines: 01(GETNEXT)
 ====
 # TPCDS-Q14b second of two queries
@@ -4407,11 +4407,11 @@ from
          and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) y
  limit 100;
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=631.88MB Threads=88
-Per-Host Resource Estimates: Memory=1.58GB
+Max Per-Host Resource Reservation: Memory=546.00MB Threads=80
+Per-Host Resource Estimates: Memory=1.36GB
 F31:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1
-|  max-parallelism=1 segment-costs=[2, 0] cpu-comparison-result=102 [max(1 (self) vs 102 (sum children))]
+|  max-parallelism=1 segment-costs=[2, 0] cpu-comparison-result=84 [max(1 (self) vs 84 (sum children))]
 PLAN-ROOT SINK
 |  output exprs: sum(sales)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0 cost=0
@@ -4430,7 +4430,7 @@ PLAN-ROOT SINK
 |
 F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
 Per-Instance Resources: mem-estimate=10.93MB mem-reservation=1.94MB thread-reservation=1
-max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-result=102 [max(12 (self) vs 102 (sum children))]
+max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-result=84 [max(12 (self) vs 84 (sum children))]
 49:AGGREGATE
 |  output: sum(sales)
 |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -4451,7 +4451,7 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |
 |  |--F40:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
 |  |  |  Per-Instance Resources: mem-estimate=3.68MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  |  max-parallelism=12 segment-costs=[44427] cpu-comparison-result=30 [max(30 (self) vs 16 (sum children))]
+|  |  |  max-parallelism=12 segment-costs=[44427] cpu-comparison-result=24 [max(24 (self) vs 13 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=08 plan-id=09 cohort-id=01
 |  |  |  build expressions: ws_bill_customer_sk
@@ -4464,7 +4464,7 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  in pipelines: 25(GETNEXT)
 |  |  |
 |  |  F29:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=6 (adjusted from 48)
-|  |  Per-Instance Resources: mem-estimate=3.07MB mem-reservation=0B thread-reservation=1
+|  |  Per-Instance Resources: mem-estimate=2.81MB mem-reservation=0B thread-reservation=1
 |  |  max-parallelism=6 segment-costs=[46258]
 |  |  47:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
 |  |  |  hash-table-id=09
@@ -4475,7 +4475,7 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |
 |  |  |--F41:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=6 (adjusted from 48)
 |  |  |  |  Per-Instance Resources: mem-estimate=11.94MB mem-reservation=3.88MB thread-reservation=1
-|  |  |  |  max-parallelism=6 segment-costs=[18186, 17975] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  |  |  |  max-parallelism=6 segment-costs=[18186, 17975] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=09 plan-id=10 cohort-id=05
 |  |  |  |  build expressions: i_item_sk
@@ -4494,8 +4494,8 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  |  in pipelines: 83(GETNEXT)
 |  |  |  |
 |  |  |  F28:PLAN FRAGMENT [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)] hosts=3 instances=6 (adjusted from 48)
-|  |  |  Per-Instance Resources: mem-estimate=54.63MB mem-reservation=36.00MB thread-reservation=1
-|  |  |  max-parallelism=6 segment-costs=[9542330, 235454, 211] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  |  |  Per-Instance Resources: mem-estimate=54.47MB mem-reservation=36.00MB thread-reservation=1
+|  |  |  max-parallelism=6 segment-costs=[9542330, 235454, 211] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  |  |  46:AGGREGATE [STREAMING]
 |  |  |  |  group by: i_item_sk
 |  |  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
@@ -4511,14 +4511,14 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  |  in pipelines: 83(GETNEXT), 27(OPEN)
 |  |  |  |
 |  |  |  82:EXCHANGE [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)]
-|  |  |  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=10.47MB mem-reservation=0B thread-reservation=0
 |  |  |  |  tuple-ids=27 row-size=50B cardinality=2.35M cost=124166
 |  |  |  |  in pipelines: 27(GETNEXT)
 |  |  |  |
-|  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 (adjusted from 48)
 |  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  |  Per-Instance Resources: mem-estimate=34.27MB mem-reservation=17.50MB thread-reservation=1
-|  |  |  max-parallelism=12 segment-costs=[105886861, 124166] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
+|  |  |  max-parallelism=9 segment-costs=[88704838, 124166] cpu-comparison-result=9 [max(9 (self) vs 8 (sum children))]
 |  |  |  32:AGGREGATE [STREAMING]
 |  |  |  |  output: count(*)
 |  |  |  |  group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date
@@ -4535,13 +4535,13 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  |  in pipelines: 27(GETNEXT), 29(OPEN)
 |  |  |  |
 |  |  |  |--F42:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  Per-Instance Resources: mem-estimate=16.44MB mem-reservation=14.25MB thread-reservation=1
 |  |  |  |  |  max-parallelism=3 segment-costs=[24564]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=06
 |  |  |  |  |  build expressions: i_item_sk
 |  |  |  |  |  runtime filters: RF029[min_max] <- i_item_sk
-|  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |  |  |  mem-estimate=14.25MB mem-reservation=14.25MB spill-buffer=256.00KB thread-reservation=0 cost=18000
 |  |  |  |  |
 |  |  |  |  81:EXCHANGE [BROADCAST]
 |  |  |  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
@@ -4566,17 +4566,17 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  |  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=24,25 row-size=42B cardinality=2.35M cost=2880404
+|  |  |  |  tuple-ids=24,25 row-size=42B cardinality=2.35M cost=2354541
 |  |  |  |  in pipelines: 27(GETNEXT), 28(OPEN)
 |  |  |  |
 |  |  |  |--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  Per-Instance Resources: mem-estimate=6.89MB mem-reservation=6.81MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  |  |  max-parallelism=3 segment-costs=[1641]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=11 plan-id=12 cohort-id=06
 |  |  |  |  |  build expressions: d_date_sk
 |  |  |  |  |  runtime filters: RF030[bloom] <- d_date_sk
-|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |  mem-estimate=5.81MB mem-reservation=5.81MB spill-buffer=64.00KB thread-reservation=0 cost=1491
 |  |  |  |  |
 |  |  |  |  80:EXCHANGE [BROADCAST]
 |  |  |  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
@@ -4606,36 +4606,36 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |       table: rows=2.88M size=199.36MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |  |     tuple-ids=24 row-size=12B cardinality=2.88M cost=91233752
+|  |  |     tuple-ids=24 row-size=12B cardinality=2.35M(filtered from 2.88M) cost=74577592
 |  |  |     in pipelines: 27(GETNEXT)
 |  |  |
 |  |  86:EXCHANGE [HASH(ws_item_sk)]
-|  |  |  mem-estimate=1.00MB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=763.17KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=22,23 row-size=36B cardinality=42.60K cost=1831
 |  |  |  in pipelines: 25(GETNEXT)
 |  |  |
-|  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  Per-Instance Resources: mem-estimate=17.03MB mem-reservation=128.00KB thread-reservation=1
-|  |  max-parallelism=12 segment-costs=[91938079]
+|  |  max-parallelism=6 segment-costs=[5495426]
 |  |  45:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  |  hash-table-id=12
 |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
 |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=22,23 row-size=36B cardinality=42.60K cost=719384
+|  |  |  tuple-ids=22,23 row-size=36B cardinality=42.60K cost=42596
 |  |  |  in pipelines: 25(GETNEXT), 26(OPEN)
 |  |  |
 |  |  |--F44:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  |  max-parallelism=3 segment-costs=[114]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=12 plan-id=13 cohort-id=05
 |  |  |  |  build expressions: d_date_sk
 |  |  |  |  runtime filters: RF026[bloom] <- d_date_sk
-|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=108
 |  |  |  |
 |  |  |  79:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -4665,9 +4665,9 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |       table: rows=719.38K size=84.20MB
 |  |       partitions: 1824/1824 rows=719.38K
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=1.12K
+|  |     extrapolated-rows=disabled max-scan-range-rows=1.12K est-scan-range=109(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-|  |     tuple-ids=22 row-size=24B cardinality=719.38K cost=91216864
+|  |     tuple-ids=22 row-size=24B cardinality=42.60K(filtered from 719.38K) cost=5450999
 |  |     in pipelines: 25(GETNEXT)
 |  |
 |  44:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -4679,7 +4679,7 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |
 |  |--F45:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
-|  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
+|  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=9 [max(3 (self) vs 9 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=13 plan-id=14 cohort-id=01
 |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -4691,7 +4691,7 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |
 |  |  F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
-|  |  max-parallelism=2 segment-costs=[2, 1] cpu-comparison-result=12 [max(2 (self) vs 12 (sum children))]
+|  |  max-parallelism=2 segment-costs=[2, 1] cpu-comparison-result=9 [max(2 (self) vs 9 (sum children))]
 |  |  77:AGGREGATE [FINALIZE]
 |  |  |  output: max:merge(csales)
 |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -4704,8 +4704,8 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  in pipelines: 43(GETNEXT)
 |  |  |
 |  |  F21:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-|  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=1.94MB thread-reservation=1
-|  |  max-parallelism=6 segment-costs=[202344, 100000, 1] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  |  Per-Instance Resources: mem-estimate=10.85MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=6 segment-costs=[202344, 100000, 1] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  |  43:AGGREGATE
 |  |  |  output: max(sum(ss_quantity * ss_sales_price))
 |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -4720,14 +4720,14 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  in pipelines: 75(GETNEXT), 37(OPEN)
 |  |  |
 |  |  74:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=867.04KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=37 row-size=20B cardinality=100.00K cost=2344
 |  |  |  in pipelines: 37(GETNEXT)
 |  |  |
-|  |  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  |  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 (adjusted from 48)
 |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1
-|  |  max-parallelism=12 segment-costs=[101189027, 2344] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
+|  |  max-parallelism=9 segment-costs=[84004956, 2344] cpu-comparison-result=9 [max(9 (self) vs 8 (sum children))]
 |  |  42:AGGREGATE [STREAMING]
 |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  |  group by: c_customer_sk
@@ -4744,13 +4744,13 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  in pipelines: 37(GETNEXT), 38(OPEN)
 |  |  |
 |  |  |--F46:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  Per-Instance Resources: mem-estimate=14.64MB mem-reservation=14.25MB thread-reservation=1
 |  |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=14 plan-id=15 cohort-id=07
 |  |  |  |  build expressions: c_customer_sk
 |  |  |  |  runtime filters: RF021[min_max] <- c_customer_sk
-|  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  mem-estimate=14.25MB mem-reservation=14.25MB spill-buffer=256.00KB thread-reservation=0 cost=100000
 |  |  |  |
 |  |  |  73:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
@@ -4775,17 +4775,17 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  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=34,36 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  tuple-ids=34,36 row-size=24B cardinality=2.35M cost=2354541
 |  |  |  in pipelines: 37(GETNEXT), 39(OPEN)
 |  |  |
 |  |  |--F47:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  Per-Instance Resources: mem-estimate=6.84MB mem-reservation=6.81MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  |  max-parallelism=3 segment-costs=[1545]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=15 plan-id=16 cohort-id=07
 |  |  |  |  build expressions: d_date_sk
 |  |  |  |  runtime filters: RF022[bloom] <- d_date_sk
-|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  mem-estimate=5.81MB mem-reservation=5.81MB spill-buffer=64.00KB thread-reservation=0 cost=1491
 |  |  |  |
 |  |  |  72:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
@@ -4815,9 +4815,9 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |       table: rows=2.88M size=199.36MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=34 row-size=16B cardinality=2.88M cost=91245000
+|  |     tuple-ids=34 row-size=16B cardinality=2.35M(filtered from 2.88M) cost=74586792
 |  |     in pipelines: 37(GETNEXT)
 |  |
 |  71:AGGREGATE [FINALIZE]
@@ -4900,8 +4900,8 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  in pipelines: 52(GETNEXT), 01(OPEN)
 |
 |--F32:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
-|  |  Per-Instance Resources: mem-estimate=4.43MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  max-parallelism=12 segment-costs=[88683] cpu-comparison-result=24 [max(24 (self) vs 16 (sum children))]
+|  |  Per-Instance Resources: mem-estimate=4.17MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=12 segment-costs=[88683] cpu-comparison-result=18 [max(18 (self) vs 13 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: cs_bill_customer_sk
@@ -4909,14 +4909,14 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=85029
 |  |
 |  68:EXCHANGE [HASH(cs_bill_customer_sk)]
-|  |  mem-estimate=1.49MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=1.23MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=0,1 row-size=36B cardinality=85.03K cost=3654
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=18.06MB mem-reservation=128.00KB thread-reservation=1
-|  max-parallelism=12 segment-costs=[93114023]
+|  max-parallelism=6 segment-costs=[5625705]
 |  23:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash-table-id=01
 |  |  hash predicates: cs_item_sk = i_item_sk
@@ -4925,13 +4925,13 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  in pipelines: 01(GETNEXT), 66(OPEN)
 |  |
 |  |--F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=7.96MB mem-reservation=7.75MB thread-reservation=1
-|  |  |  max-parallelism=3 segment-costs=[18608] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
+|  |  |  Per-Instance Resources: mem-estimate=4.08MB mem-reservation=3.88MB thread-reservation=1
+|  |  |  max-parallelism=3 segment-costs=[18608] cpu-comparison-result=9 [max(3 (self) vs 9 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: i_item_sk
 |  |  |  runtime filters: RF009[min_max] <- i_item_sk
-|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=17975
 |  |  |
 |  |  67:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=212.43KB mem-reservation=0B thread-reservation=0
@@ -4940,7 +4940,7 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |
 |  |  F14:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=6 (adjusted from 48)
 |  |  Per-Instance Resources: mem-estimate=10.12MB mem-reservation=1.94MB thread-reservation=1
-|  |  max-parallelism=6 segment-costs=[18186, 211] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  |  max-parallelism=6 segment-costs=[18186, 211] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  |  66:AGGREGATE [FINALIZE]
 |  |  |  group by: i_item_sk
 |  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
@@ -4953,8 +4953,8 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  in pipelines: 64(GETNEXT)
 |  |  |
 |  |  F13:PLAN FRAGMENT [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)] hosts=3 instances=6 (adjusted from 48)
-|  |  Per-Instance Resources: mem-estimate=54.63MB mem-reservation=36.00MB thread-reservation=1
-|  |  max-parallelism=6 segment-costs=[9542330, 235454, 211] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  |  Per-Instance Resources: mem-estimate=54.47MB mem-reservation=36.00MB thread-reservation=1
+|  |  max-parallelism=6 segment-costs=[9542330, 235454, 211] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  |  22:AGGREGATE [STREAMING]
 |  |  |  group by: i_item_sk
 |  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
@@ -4970,14 +4970,14 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  in pipelines: 64(GETNEXT), 03(OPEN)
 |  |  |
 |  |  63:EXCHANGE [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)]
-|  |  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=10.47MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=5 row-size=50B cardinality=2.35M cost=124166
 |  |  |  in pipelines: 03(GETNEXT)
 |  |  |
-|  |  F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  |  F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 (adjusted from 48)
 |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  Per-Instance Resources: mem-estimate=34.27MB mem-reservation=17.50MB thread-reservation=1
-|  |  max-parallelism=12 segment-costs=[105886861, 124166] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
+|  |  max-parallelism=9 segment-costs=[88704838, 124166] cpu-comparison-result=9 [max(9 (self) vs 8 (sum children))]
 |  |  08:AGGREGATE [STREAMING]
 |  |  |  output: count(*)
 |  |  |  group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date
@@ -4994,13 +4994,13 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  in pipelines: 03(GETNEXT), 05(OPEN)
 |  |  |
 |  |  |--F34:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.44MB mem-reservation=14.25MB thread-reservation=1
 |  |  |  |  max-parallelism=3 segment-costs=[24564]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: i_item_sk
 |  |  |  |  runtime filters: RF013[min_max] <- i_item_sk
-|  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |  |  mem-estimate=14.25MB mem-reservation=14.25MB spill-buffer=256.00KB thread-reservation=0 cost=18000
 |  |  |  |
 |  |  |  62:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
@@ -5025,17 +5025,17 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  |  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=2,3 row-size=42B cardinality=2.35M cost=2880404
+|  |  |  tuple-ids=2,3 row-size=42B cardinality=2.35M cost=2354541
 |  |  |  in pipelines: 03(GETNEXT), 04(OPEN)
 |  |  |
 |  |  |--F35:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  Per-Instance Resources: mem-estimate=6.89MB mem-reservation=6.81MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  |  max-parallelism=3 segment-costs=[1641]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  |  build expressions: d_date_sk
 |  |  |  |  runtime filters: RF014[bloom] <- d_date_sk
-|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  mem-estimate=5.81MB mem-reservation=5.81MB spill-buffer=64.00KB thread-reservation=0 cost=1491
 |  |  |  |
 |  |  |  61:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
@@ -5065,9 +5065,9 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |       table: rows=2.88M size=199.36MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=2 row-size=12B cardinality=2.88M cost=91233752
+|  |     tuple-ids=2 row-size=12B cardinality=2.35M(filtered from 2.88M) cost=74577592
 |  |     in pipelines: 03(GETNEXT)
 |  |
 |  21:HASH JOIN [INNER JOIN, BROADCAST]
@@ -5075,17 +5075,17 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  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 row-size=36B cardinality=85.03K cost=1441548
+|  |  tuple-ids=0,1 row-size=36B cardinality=85.03K cost=85029
 |  |  in pipelines: 01(GETNEXT), 02(OPEN)
 |  |
 |  |--F36:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=02
 |  |  |  build expressions: d_date_sk
 |  |  |  runtime filters: RF010[bloom] <- d_date_sk
-|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=108
 |  |  |
 |  |  60:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -5115,9 +5115,9 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |       table: rows=1.44M size=151.13MB
 |       partitions: 1831/1831 rows=1.44M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=7.18K
+|     extrapolated-rows=disabled max-scan-range-rows=7.18K est-scan-range=109(filtered from 1831)
 |     mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-|     tuple-ids=0 row-size=24B cardinality=1.44M cost=91583792
+|     tuple-ids=0 row-size=24B cardinality=85.03K(filtered from 1.44M) cost=5451993
 |     in pipelines: 01(GETNEXT)
 |
 20:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -5129,7 +5129,7 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |
 |--F37:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
-|  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
+|  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=9 [max(3 (self) vs 9 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -5141,7 +5141,7 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |
 |  F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
-|  max-parallelism=2 segment-costs=[2, 1] cpu-comparison-result=12 [max(2 (self) vs 12 (sum children))]
+|  max-parallelism=2 segment-costs=[2, 1] cpu-comparison-result=9 [max(2 (self) vs 9 (sum children))]
 |  58:AGGREGATE [FINALIZE]
 |  |  output: max:merge(csales)
 |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -5154,8 +5154,8 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  in pipelines: 19(GETNEXT)
 |  |
 |  F06:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-|  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=1.94MB thread-reservation=1
-|  max-parallelism=6 segment-costs=[202344, 100000, 1] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  Per-Instance Resources: mem-estimate=10.85MB mem-reservation=1.94MB thread-reservation=1
+|  max-parallelism=6 segment-costs=[202344, 100000, 1] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  19:AGGREGATE
 |  |  output: max(sum(ss_quantity * ss_sales_price))
 |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -5170,14 +5170,14 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  in pipelines: 56(GETNEXT), 13(OPEN)
 |  |
 |  55:EXCHANGE [HASH(c_customer_sk)]
-|  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=867.04KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=15 row-size=20B cardinality=100.00K cost=2344
 |  |  in pipelines: 13(GETNEXT)
 |  |
-|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1
-|  max-parallelism=12 segment-costs=[101189027, 2344] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
+|  max-parallelism=9 segment-costs=[84004956, 2344] cpu-comparison-result=9 [max(9 (self) vs 8 (sum children))]
 |  18:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  group by: c_customer_sk
@@ -5194,13 +5194,13 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  in pipelines: 13(GETNEXT), 14(OPEN)
 |  |
 |  |--F38:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=20.39MB mem-reservation=20.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=15.64MB mem-reservation=15.25MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  JOIN BUILD
 |  |  |  join-table-id=06 plan-id=07 cohort-id=04
 |  |  |  build expressions: c_customer_sk
 |  |  |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
-|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  mem-estimate=14.25MB mem-reservation=14.25MB spill-buffer=256.00KB thread-reservation=0 cost=100000
 |  |  |
 |  |  54:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
@@ -5225,17 +5225,17 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |  |  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=12,14 row-size=24B cardinality=2.35M cost=2880404
+|  |  tuple-ids=12,14 row-size=24B cardinality=2.35M cost=2354541
 |  |  in pipelines: 13(GETNEXT), 15(OPEN)
 |  |
 |  |--F39:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=6.84MB mem-reservation=6.81MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  max-parallelism=3 segment-costs=[1545]
 |  |  JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=04
 |  |  |  build expressions: d_date_sk
 |  |  |  runtime filters: RF006[bloom] <- d_date_sk
-|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  mem-estimate=5.81MB mem-reservation=5.81MB spill-buffer=64.00KB thread-reservation=0 cost=1491
 |  |  |
 |  |  53:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
@@ -5265,9 +5265,9 @@ max-parallelism=12 segment-costs=[202344, 202344, 582875, 1] cpu-comparison-resu
 |       table: rows=2.88M size=199.36MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=12 row-size=16B cardinality=2.88M cost=91245000
+|     tuple-ids=12 row-size=16B cardinality=2.35M(filtered from 2.88M) cost=74586792
 |     in pipelines: 13(GETNEXT)
 |
 52:AGGREGATE [FINALIZE]
@@ -5428,11 +5428,11 @@ ORDER BY c_last_name,
          sales
 LIMIT 100;
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=663.88MB Threads=93
-Per-Host Resource Estimates: Memory=1.63GB
+Max Per-Host Resource Reservation: Memory=581.88MB Threads=85
+Per-Host Resource Estimates: Memory=1.41GB
 F35:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
-|  max-parallelism=1 segment-costs=[306] cpu-comparison-result=78 [max(1 (self) vs 78 (sum children))]
+|  max-parallelism=1 segment-costs=[306] cpu-comparison-result=66 [max(1 (self) vs 66 (sum children))]
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, sales
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=300
@@ -5446,7 +5446,7 @@ PLAN-ROOT SINK
 |
 F34:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reservation=1
-max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78 [max(6 (self) vs 78 (sum children))]
+max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=66 [max(6 (self) vs 66 (sum children))]
 53:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, c_first_name ASC, sales ASC
 |  mem-estimate=5.08KB mem-reservation=0B thread-reservation=0
@@ -5473,7 +5473,7 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |
 |  F32:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=6 (adjusted from 48)
 |  Per-Instance Resources: mem-estimate=15.37MB mem-reservation=2.00MB thread-reservation=1
-|  max-parallelism=6 segment-costs=[276575, 523] cpu-comparison-result=39 [max(6 (self) vs 39 (sum children))]
+|  max-parallelism=6 segment-costs=[276575, 523] cpu-comparison-result=33 [max(6 (self) vs 33 (sum children))]
 |  52:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price)
 |  |  group by: c_last_name, c_first_name
@@ -5491,7 +5491,7 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |
 |  |--F46:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=6 (adjusted from 48)
 |  |  |  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  |  max-parallelism=6 segment-costs=[10764] cpu-comparison-result=27 [max(24 (self) vs 27 (sum children))]
+|  |  |  max-parallelism=6 segment-costs=[10764] cpu-comparison-result=24 [max(18 (self) vs 24 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=10 plan-id=11 cohort-id=05
 |  |  |  build expressions: ws_item_sk
@@ -5504,8 +5504,8 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  in pipelines: 27(GETNEXT)
 |  |  |
 |  |  F30:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-|  |  Per-Instance Resources: mem-estimate=4.00MB mem-reservation=0B thread-reservation=1
-|  |  max-parallelism=6 segment-costs=[55191]
+|  |  Per-Instance Resources: mem-estimate=3.74MB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=6 segment-costs=[55190]
 |  |  50:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  |  hash-table-id=11
 |  |  |  hash predicates: ws_bill_customer_sk = customer.c_customer_sk
@@ -5546,12 +5546,12 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
 |  |  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=21,23,39 row-size=72B cardinality=9.57K cost=42596
+|  |  |  tuple-ids=21,23,39 row-size=72B cardinality=9.57K cost=42595
 |  |  |  in pipelines: 27(GETNEXT), 89(OPEN)
 |  |  |
 |  |  |--F48:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 (adjusted from 48)
 |  |  |  |  Per-Instance Resources: mem-estimate=12.94MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  |  |  max-parallelism=6 segment-costs=[303907, 10000] cpu-comparison-result=16 [max(6 (self) vs 16 (sum children))]
+|  |  |  |  max-parallelism=6 segment-costs=[303907, 10000] cpu-comparison-result=13 [max(6 (self) vs 13 (sum children))]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=12 plan-id=13 cohort-id=06
 |  |  |  |  build expressions: c_customer_sk
@@ -5573,7 +5573,7 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  |
 |  |  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
 |  |  |  Per-Instance Resources: mem-estimate=26.94MB mem-reservation=10.00MB thread-reservation=1
-|  |  |  max-parallelism=12 segment-costs=[102755368, 3907] cpu-comparison-result=16 [max(12 (self) vs 16 (sum children))]
+|  |  |  max-parallelism=12 segment-costs=[102755368, 3907] cpu-comparison-result=13 [max(12 (self) vs 13 (sum children))]
 |  |  |  47:AGGREGATE [STREAMING]
 |  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales))
 |  |  |  |  group by: c_customer_sk
@@ -5589,7 +5589,7 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  |
 |  |  |  |--F49:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
-|  |  |  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
+|  |  |  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=9 [max(3 (self) vs 9 (sum children))]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=13 plan-id=14 cohort-id=07
 |  |  |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -5601,7 +5601,7 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  |  |
 |  |  |  |  F29:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  |  |  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
-|  |  |  |  max-parallelism=2 segment-costs=[2, 1] cpu-comparison-result=12 [max(2 (self) vs 12 (sum children))]
+|  |  |  |  max-parallelism=2 segment-costs=[2, 1] cpu-comparison-result=9 [max(2 (self) vs 9 (sum children))]
 |  |  |  |  86:AGGREGATE [FINALIZE]
 |  |  |  |  |  output: max:merge(csales)
 |  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -5614,8 +5614,8 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  |  |  in pipelines: 44(GETNEXT)
 |  |  |  |  |
 |  |  |  |  F28:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-|  |  |  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=1.94MB thread-reservation=1
-|  |  |  |  max-parallelism=6 segment-costs=[202344, 100000, 1] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  |  |  |  Per-Instance Resources: mem-estimate=10.85MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  |  max-parallelism=6 segment-costs=[202344, 100000, 1] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  |  |  |  44:AGGREGATE
 |  |  |  |  |  output: max(sum(ss_quantity * ss_sales_price))
 |  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -5630,14 +5630,14 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  |  |  in pipelines: 84(GETNEXT), 38(OPEN)
 |  |  |  |  |
 |  |  |  |  83:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  mem-estimate=867.04KB mem-reservation=0B thread-reservation=0
 |  |  |  |  |  tuple-ids=35 row-size=20B cardinality=100.00K cost=2344
 |  |  |  |  |  in pipelines: 38(GETNEXT)
 |  |  |  |  |
-|  |  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  |  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 (adjusted from 48)
 |  |  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  |  |  Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1
-|  |  |  |  max-parallelism=12 segment-costs=[101189027, 2344] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
+|  |  |  |  max-parallelism=9 segment-costs=[84004956, 2344] cpu-comparison-result=9 [max(9 (self) vs 8 (sum children))]
 |  |  |  |  43:AGGREGATE [STREAMING]
 |  |  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  |  |  |  group by: c_customer_sk
@@ -5654,13 +5654,13 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  |  |  in pipelines: 38(GETNEXT), 39(OPEN)
 |  |  |  |  |
 |  |  |  |  |--F50:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  |  Per-Instance Resources: mem-estimate=14.64MB mem-reservation=14.25MB thread-reservation=1
 |  |  |  |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  |  |  |  JOIN BUILD
 |  |  |  |  |  |  join-table-id=14 plan-id=15 cohort-id=08
 |  |  |  |  |  |  build expressions: c_customer_sk
 |  |  |  |  |  |  runtime filters: RF033[min_max] <- c_customer_sk
-|  |  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  |  |  mem-estimate=14.25MB mem-reservation=14.25MB spill-buffer=256.00KB thread-reservation=0 cost=100000
 |  |  |  |  |  |
 |  |  |  |  |  82:EXCHANGE [BROADCAST]
 |  |  |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
@@ -5685,17 +5685,17 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  |  |  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=32,34 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  |  |  tuple-ids=32,34 row-size=24B cardinality=2.35M cost=2354541
 |  |  |  |  |  in pipelines: 38(GETNEXT), 40(OPEN)
 |  |  |  |  |
 |  |  |  |  |--F51:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  |  Per-Instance Resources: mem-estimate=6.84MB mem-reservation=6.81MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  |  |  |  max-parallelism=3 segment-costs=[1545]
 |  |  |  |  |  JOIN BUILD
 |  |  |  |  |  |  join-table-id=15 plan-id=16 cohort-id=08
 |  |  |  |  |  |  build expressions: d_date_sk
 |  |  |  |  |  |  runtime filters: RF034[bloom] <- d_date_sk
-|  |  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |  |  mem-estimate=5.81MB mem-reservation=5.81MB spill-buffer=64.00KB thread-reservation=0 cost=1491
 |  |  |  |  |  |
 |  |  |  |  |  81:EXCHANGE [BROADCAST]
 |  |  |  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
@@ -5725,9 +5725,9 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |  |       table: rows=2.88M size=199.36MB
 |  |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |  |       columns: all
-|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |  |  |     tuple-ids=32 row-size=16B cardinality=2.88M cost=91245000
+|  |  |  |     tuple-ids=32 row-size=16B cardinality=2.35M(filtered from 2.88M) cost=74586792
 |  |  |  |     in pipelines: 38(GETNEXT)
 |  |  |  |
 |  |  |  45:HASH JOIN [INNER JOIN, BROADCAST]
@@ -5779,30 +5779,30 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  |     in pipelines: 36(GETNEXT)
 |  |  |
 |  |  90:EXCHANGE [HASH(ws_bill_customer_sk)]
-|  |  |  mem-estimate=1.00MB mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=21,23 row-size=36B cardinality=42.60K cost=1831
+|  |  |  mem-estimate=763.17KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=21,23 row-size=36B cardinality=42.59K(filtered from 42.60K) cost=1831
 |  |  |  in pipelines: 27(GETNEXT)
 |  |  |
-|  |  F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  |  F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  |  Per-Instance Resources: mem-estimate=17.03MB mem-reservation=128.00KB thread-reservation=1
-|  |  max-parallelism=12 segment-costs=[91938079]
+|  |  max-parallelism=6 segment-costs=[5495425]
 |  |  48:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  |  hash-table-id=17
 |  |  |  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=21,23 row-size=36B cardinality=42.60K cost=719384
+|  |  |  tuple-ids=21,23 row-size=36B cardinality=42.59K(filtered from 42.60K) cost=42595
 |  |  |  in pipelines: 27(GETNEXT), 29(OPEN)
 |  |  |
 |  |  |--F53:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  |  max-parallelism=3 segment-costs=[114]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=17 plan-id=18 cohort-id=06
 |  |  |  |  build expressions: d_date_sk
 |  |  |  |  runtime filters: RF028[bloom] <- d_date_sk
-|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=108
 |  |  |  |
 |  |  |  79:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -5832,9 +5832,9 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |       table: rows=719.38K size=84.20MB
 |  |       partitions: 1824/1824 rows=719.38K
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=1.12K
+|  |     extrapolated-rows=disabled max-scan-range-rows=1.12K est-scan-range=109(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-|  |     tuple-ids=21 row-size=24B cardinality=719.38K cost=91216864
+|  |     tuple-ids=21 row-size=24B cardinality=42.59K(filtered from 719.38K) cost=5450999
 |  |     in pipelines: 27(GETNEXT)
 |  |
 |  92:EXCHANGE [HASH(i_item_sk)]
@@ -5843,8 +5843,8 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  in pipelines: 78(GETNEXT)
 |  |
 |  F20:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=6 (adjusted from 48)
-|  Per-Instance Resources: mem-estimate=44.63MB mem-reservation=34.00MB thread-reservation=1
-|  max-parallelism=6 segment-costs=[9542330, 12417] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  Per-Instance Resources: mem-estimate=44.47MB mem-reservation=34.00MB thread-reservation=1
+|  max-parallelism=6 segment-costs=[9542330, 12417] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  78:AGGREGATE [FINALIZE]
 |  |  output: count:merge(*)
 |  |  group by: itemdesc, i_item_sk, d_date
@@ -5854,14 +5854,14 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  in pipelines: 78(GETNEXT), 30(OPEN)
 |  |
 |  77:EXCHANGE [HASH(itemdesc,i_item_sk,d_date)]
-|  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=10.47MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=28 row-size=50B cardinality=2.35M cost=124166
 |  |  in pipelines: 30(GETNEXT)
 |  |
-|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  Per-Instance Resources: mem-estimate=34.27MB mem-reservation=17.50MB thread-reservation=1
-|  max-parallelism=12 segment-costs=[105886861, 124166] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
+|  max-parallelism=9 segment-costs=[88704838, 124166] cpu-comparison-result=9 [max(9 (self) vs 8 (sum children))]
 |  35:AGGREGATE [STREAMING]
 |  |  output: count(*)
 |  |  group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_partitioned_parquet_snap.item.i_item_sk, d_date
@@ -5878,13 +5878,13 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  in pipelines: 30(GETNEXT), 32(OPEN)
 |  |
 |  |--F54:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  Per-Instance Resources: mem-estimate=16.44MB mem-reservation=14.25MB thread-reservation=1
 |  |  |  max-parallelism=3 segment-costs=[24564]
 |  |  JOIN BUILD
 |  |  |  join-table-id=18 plan-id=19 cohort-id=05
 |  |  |  build expressions: tpcds_partitioned_parquet_snap.item.i_item_sk
 |  |  |  runtime filters: RF021[min_max] <- tpcds_partitioned_parquet_snap.item.i_item_sk
-|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |  mem-estimate=14.25MB mem-reservation=14.25MB spill-buffer=256.00KB thread-reservation=0 cost=18000
 |  |  |
 |  |  76:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
@@ -5911,17 +5911,17 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |  |  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=24,25 row-size=42B cardinality=2.35M cost=2880404
+|  |  tuple-ids=24,25 row-size=42B cardinality=2.35M cost=2354541
 |  |  in pipelines: 30(GETNEXT), 31(OPEN)
 |  |
 |  |--F55:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=6.89MB mem-reservation=6.81MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  max-parallelism=3 segment-costs=[1641]
 |  |  JOIN BUILD
 |  |  |  join-table-id=19 plan-id=20 cohort-id=05
 |  |  |  build expressions: d_date_sk
 |  |  |  runtime filters: RF022[bloom] <- d_date_sk
-|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  mem-estimate=5.81MB mem-reservation=5.81MB spill-buffer=64.00KB thread-reservation=0 cost=1491
 |  |  |
 |  |  75:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
@@ -5951,9 +5951,9 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |       table: rows=2.88M size=199.36MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|     tuple-ids=24 row-size=12B cardinality=2.88M cost=91233752
+|     tuple-ids=24 row-size=12B cardinality=2.35M(filtered from 2.88M) cost=74577592
 |     in pipelines: 30(GETNEXT)
 |
 74:AGGREGATE [FINALIZE]
@@ -5970,7 +5970,7 @@ max-parallelism=6 segment-costs=[32050, 29227, 300, 6] cpu-comparison-result=78
 |
 F15:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=6 (adjusted from 48)
 Per-Instance Resources: mem-estimate=15.37MB mem-reservation=2.00MB thread-reservation=1
-max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (self) vs 39 (sum children))]
+max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=33 [max(6 (self) vs 33 (sum children))]
 26:AGGREGATE [STREAMING]
 |  output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price)
 |  group by: c_last_name, c_first_name
@@ -5988,7 +5988,7 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |
 |--F36:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=6 (adjusted from 48)
 |  |  Per-Instance Resources: mem-estimate=4.06MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  max-parallelism=6 segment-costs=[11804] cpu-comparison-result=27 [max(24 (self) vs 27 (sum children))]
+|  |  max-parallelism=6 segment-costs=[11804] cpu-comparison-result=24 [max(18 (self) vs 24 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: cs_item_sk
@@ -6001,7 +6001,7 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  in pipelines: 01(GETNEXT)
 |  |
 |  F13:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-|  Per-Instance Resources: mem-estimate=4.49MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.23MB mem-reservation=0B thread-reservation=1
 |  max-parallelism=6 segment-costs=[100487]
 |  24:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=01
@@ -6048,7 +6048,7 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |
 |  |--F38:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 (adjusted from 48)
 |  |  |  Per-Instance Resources: mem-estimate=12.94MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  |  max-parallelism=6 segment-costs=[303907, 10000] cpu-comparison-result=16 [max(6 (self) vs 16 (sum children))]
+|  |  |  max-parallelism=6 segment-costs=[303907, 10000] cpu-comparison-result=13 [max(6 (self) vs 13 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  build expressions: c_customer_sk
@@ -6070,7 +6070,7 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  |
 |  |  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
 |  |  Per-Instance Resources: mem-estimate=26.94MB mem-reservation=10.00MB thread-reservation=1
-|  |  max-parallelism=12 segment-costs=[102755368, 3907] cpu-comparison-result=16 [max(12 (self) vs 16 (sum children))]
+|  |  max-parallelism=12 segment-costs=[102755368, 3907] cpu-comparison-result=13 [max(12 (self) vs 13 (sum children))]
 |  |  21:AGGREGATE [STREAMING]
 |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales))
 |  |  |  group by: c_customer_sk
@@ -6086,7 +6086,7 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  |
 |  |  |--F39:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
-|  |  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
+|  |  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=9 [max(3 (self) vs 9 (sum children))]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -6098,7 +6098,7 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  |  |
 |  |  |  F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  |  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
-|  |  |  max-parallelism=2 segment-costs=[2, 1] cpu-comparison-result=12 [max(2 (self) vs 12 (sum children))]
+|  |  |  max-parallelism=2 segment-costs=[2, 1] cpu-comparison-result=9 [max(2 (self) vs 9 (sum children))]
 |  |  |  65:AGGREGATE [FINALIZE]
 |  |  |  |  output: max:merge(csales)
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -6111,8 +6111,8 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  |  |  in pipelines: 18(GETNEXT)
 |  |  |  |
 |  |  |  F11:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-|  |  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=1.94MB thread-reservation=1
-|  |  |  max-parallelism=6 segment-costs=[202344, 100000, 1] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+|  |  |  Per-Instance Resources: mem-estimate=10.85MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  max-parallelism=6 segment-costs=[202344, 100000, 1] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 |  |  |  18:AGGREGATE
 |  |  |  |  output: max(sum(ss_quantity * ss_sales_price))
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -6127,14 +6127,14 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  |  |  in pipelines: 63(GETNEXT), 12(OPEN)
 |  |  |  |
 |  |  |  62:EXCHANGE [HASH(c_customer_sk)]
-|  |  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=867.04KB mem-reservation=0B thread-reservation=0
 |  |  |  |  tuple-ids=14 row-size=20B cardinality=100.00K cost=2344
 |  |  |  |  in pipelines: 12(GETNEXT)
 |  |  |  |
-|  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 (adjusted from 48)
 |  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  |  Per-Instance Resources: mem-estimate=26.56MB mem-reservation=6.00MB thread-reservation=1
-|  |  |  max-parallelism=12 segment-costs=[101189027, 2344] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
+|  |  |  max-parallelism=9 segment-costs=[84004956, 2344] cpu-comparison-result=9 [max(9 (self) vs 8 (sum children))]
 |  |  |  17:AGGREGATE [STREAMING]
 |  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  |  |  group by: c_customer_sk
@@ -6151,13 +6151,13 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  |  |  in pipelines: 12(GETNEXT), 13(OPEN)
 |  |  |  |
 |  |  |  |--F40:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  Per-Instance Resources: mem-estimate=14.64MB mem-reservation=14.25MB thread-reservation=1
 |  |  |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=04 plan-id=05 cohort-id=04
 |  |  |  |  |  build expressions: c_customer_sk
 |  |  |  |  |  runtime filters: RF015[min_max] <- c_customer_sk
-|  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  |  mem-estimate=14.25MB mem-reservation=14.25MB spill-buffer=256.00KB thread-reservation=0 cost=100000
 |  |  |  |  |
 |  |  |  |  61:EXCHANGE [BROADCAST]
 |  |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
@@ -6182,17 +6182,17 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  |  |  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=11,13 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  |  tuple-ids=11,13 row-size=24B cardinality=2.35M cost=2354541
 |  |  |  |  in pipelines: 12(GETNEXT), 14(OPEN)
 |  |  |  |
 |  |  |  |--F41:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  Per-Instance Resources: mem-estimate=6.84MB mem-reservation=6.81MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  |  |  max-parallelism=3 segment-costs=[1545]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=05 plan-id=06 cohort-id=04
 |  |  |  |  |  build expressions: d_date_sk
 |  |  |  |  |  runtime filters: RF016[bloom] <- d_date_sk
-|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |  mem-estimate=5.81MB mem-reservation=5.81MB spill-buffer=64.00KB thread-reservation=0 cost=1491
 |  |  |  |  |
 |  |  |  |  60:EXCHANGE [BROADCAST]
 |  |  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
@@ -6222,9 +6222,9 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |  |       table: rows=2.88M size=199.36MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |  |     tuple-ids=11 row-size=16B cardinality=2.88M cost=91245000
+|  |  |     tuple-ids=11 row-size=16B cardinality=2.35M(filtered from 2.88M) cost=74586792
 |  |  |     in pipelines: 12(GETNEXT)
 |  |  |
 |  |  19:HASH JOIN [INNER JOIN, BROADCAST]
@@ -6276,30 +6276,30 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  |     in pipelines: 10(GETNEXT)
 |  |
 |  69:EXCHANGE [HASH(cs_bill_customer_sk)]
-|  |  mem-estimate=1.49MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=1.23MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=0,2 row-size=36B cardinality=85.03K cost=3654
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  Per-Instance Resources: mem-estimate=17.03MB mem-reservation=128.00KB thread-reservation=1
-|  max-parallelism=12 segment-costs=[93028994]
+|  max-parallelism=6 segment-costs=[5540676]
 |  22:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=07
 |  |  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,2 row-size=36B cardinality=85.03K cost=1441548
+|  |  tuple-ids=0,2 row-size=36B cardinality=85.03K cost=85029
 |  |  in pipelines: 01(GETNEXT), 03(OPEN)
 |  |
 |  |--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=02
 |  |  |  build expressions: d_date_sk
 |  |  |  runtime filters: RF010[bloom] <- d_date_sk
-|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=108
 |  |  |
 |  |  58:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -6329,9 +6329,9 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |       table: rows=1.44M size=151.13MB
 |       partitions: 1831/1831 rows=1.44M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=7.18K
+|     extrapolated-rows=disabled max-scan-range-rows=7.18K est-scan-range=109(filtered from 1831)
 |     mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-|     tuple-ids=0 row-size=24B cardinality=1.44M cost=91583792
+|     tuple-ids=0 row-size=24B cardinality=85.03K(filtered from 1.44M) cost=5451993
 |     in pipelines: 01(GETNEXT)
 |
 71:EXCHANGE [HASH(i_item_sk)]
@@ -6340,8 +6340,8 @@ max-parallelism=6 segment-costs=[279347, 574] cpu-comparison-result=39 [max(6 (s
 |  in pipelines: 57(GETNEXT)
 |
 F03:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=6 (adjusted from 48)
-Per-Instance Resources: mem-estimate=44.63MB mem-reservation=34.00MB thread-reservation=1
-max-parallelism=6 segment-costs=[9542330, 12417] cpu-comparison-result=12 [max(6 (self) vs 12 (sum children))]
+Per-Instance Resources: mem-estimate=44.47MB mem-reservation=34.00MB thread-reservation=1
+max-parallelism=6 segment-costs=[9542330, 12417] cpu-comparison-result=9 [max(6 (self) vs 9 (sum children))]
 57:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |  group by: itemdesc, i_item_sk, d_date
@@ -6351,14 +6351,14 @@ max-parallelism=6 segment-costs=[9542330, 12417] cpu-comparison-result=12 [max(6
 |  in pipelines: 57(GETNEXT), 04(OPEN)
 |
 56:EXCHANGE [HASH(itemdesc,i_item_sk,d_date)]
-|  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.47MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=7 row-size=50B cardinality=2.35M cost=124166
 |  in pipelines: 04(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9 (adjusted from 48)
 Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 Per-Instance Resources: mem-estimate=34.27MB mem-reservation=17.50MB thread-reservation=1
-max-parallelism=12 segment-costs=[105886861, 124166] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
+max-parallelism=9 segment-costs=[88704838, 124166] cpu-comparison-result=9 [max(9 (self) vs 8 (sum children))]
 09:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_partitioned_parquet_snap.item.i_item_sk, d_date
@@ -6375,13 +6375,13 @@ max-parallelism=12 segment-costs=[105886861, 124166] cpu-comparison-result=12 [m
 |  in pipelines: 04(GETNEXT), 06(OPEN)
 |
 |--F44:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  Per-Instance Resources: mem-estimate=16.44MB mem-reservation=14.25MB thread-reservation=1
 |  |  max-parallelism=3 segment-costs=[24564]
 |  JOIN BUILD
 |  |  join-table-id=08 plan-id=09 cohort-id=01
 |  |  build expressions: tpcds_partitioned_parquet_snap.item.i_item_sk
 |  |  runtime filters: RF003[min_max] <- tpcds_partitioned_parquet_snap.item.i_item_sk
-|  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  mem-estimate=14.25MB mem-reservation=14.25MB spill-buffer=256.00KB thread-reservation=0 cost=18000
 |  |
 |  55:EXCHANGE [BROADCAST]
 |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
@@ -6408,17 +6408,17 @@ max-parallelism=12 segment-costs=[105886861, 124166] cpu-comparison-result=12 [m
 |  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=3,4 row-size=42B cardinality=2.35M cost=2880404
+|  tuple-ids=3,4 row-size=42B cardinality=2.35M cost=2354541
 |  in pipelines: 04(GETNEXT), 05(OPEN)
 |
 |--F45:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=6.89MB mem-reservation=6.81MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  max-parallelism=3 segment-costs=[1641]
 |  JOIN BUILD
 |  |  join-table-id=09 plan-id=10 cohort-id=01
 |  |  build expressions: d_date_sk
 |  |  runtime filters: RF004[bloom] <- d_date_sk
-|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  mem-estimate=5.81MB mem-reservation=5.81MB spill-buffer=64.00KB thread-reservation=0 cost=1491
 |  |
 |  54:EXCHANGE [BROADCAST]
 |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
@@ -6448,9 +6448,9 @@ max-parallelism=12 segment-costs=[105886861, 124166] cpu-comparison-result=12 [m
      table: rows=2.88M size=199.36MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-   tuple-ids=3 row-size=12B cardinality=2.88M cost=91233752
+   tuple-ids=3 row-size=12B cardinality=2.35M(filtered from 2.88M) cost=74577592
    in pipelines: 04(GETNEXT)
 ====
 # TPCDS-Q49
@@ -6557,8 +6557,8 @@ ORDER BY 1,
 LIMIT 100;
 
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=125.44MB Threads=49
-Per-Host Resource Estimates: Memory=684MB
+Max Per-Host Resource Reservation: Memory=116.69MB Threads=43
+Per-Host Resource Estimates: Memory=582MB
 F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=24.84MB mem-reservation=15.94MB thread-reservation=1
 |  max-parallelism=1 segment-costs=[8837, 17638, 35376, 119276, 400, 500] cpu-comparison-result=66 [max(1 (self) vs 66 (sum children))]
@@ -6641,8 +6641,8 @@ PLAN-ROOT SINK
 |  |  in pipelines: 23(GETNEXT)
 |  |
 |  F14:PLAN FRAGMENT [HASH(sts.ss_item_sk,sts.ss_ticket_number)] hosts=3 instances=6 (adjusted from 48)
-|  Per-Instance Resources: mem-estimate=14.76MB mem-reservation=2.00MB thread-reservation=1
-|  max-parallelism=6 segment-costs=[959522, 1000] cpu-comparison-result=22 [max(18 (self) vs 22 (sum children))]
+|  Per-Instance Resources: mem-estimate=14.55MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=6 segment-costs=[607052, 1000] cpu-comparison-result=22 [max(12 (self) vs 22 (sum children))]
 |  28:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(coalesce(sr.sr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(sts.ss_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(sr.sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(sts.ss_net_paid, CAST(0 AS DECIMAL(7,2))))
 |  |  group by: sts.ss_item_sk
@@ -6655,7 +6655,7 @@ PLAN-ROOT SINK
 |  |  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=16,17N,18 row-size=68B cardinality=17.05K cost=288040
+|  |  tuple-ids=16,17N,18 row-size=68B cardinality=17.05K cost=170550
 |  |  in pipelines: 23(GETNEXT), 25(OPEN)
 |  |
 |  |--F23:PLAN FRAGMENT [HASH(sts.ss_item_sk,sts.ss_ticket_number)] hosts=3 instances=3
@@ -6694,7 +6694,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: sts.ss_item_sk = sr.sr_item_sk, sts.ss_ticket_number = sr.sr_ticket_number
 |  |  other predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=16,17N row-size=56B cardinality=288.04K cost=576080
+|  |  tuple-ids=16,17N row-size=56B cardinality=170.55K(filtered from 288.04K) cost=341100
 |  |  in pipelines: 23(GETNEXT), 24(OPEN)
 |  |
 |  |--F24:PLAN FRAGMENT [HASH(sts.ss_item_sk,sts.ss_ticket_number)] hosts=3 instances=6 (adjusted from 48)
@@ -6728,14 +6728,14 @@ PLAN-ROOT SINK
 |  |     in pipelines: 24(GETNEXT)
 |  |
 |  48:EXCHANGE [HASH(sts.ss_item_sk,sts.ss_ticket_number)]
-|  |  mem-estimate=3.35MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=16 row-size=32B cardinality=288.04K cost=10127
+|  |  mem-estimate=3.14MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=16 row-size=32B cardinality=170.55K(filtered from 288.04K) cost=10127
 |  |  in pipelines: 23(GETNEXT)
 |  |
-|  F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F12:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=16.84MB mem-reservation=4.00MB thread-reservation=1
-|  max-parallelism=12 segment-costs=[99941351]
+|  max-parallelism=6 segment-costs=[5972976]
 |  23:SCAN HDFS [tpcds_partitioned_parquet_snap.store_sales sts, RANDOM]
 |     HDFS partitions=1824/1824 files=1824 size=199.36MB
 |     predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
@@ -6744,11 +6744,11 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=199.36MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     parquet statistics predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
 |     parquet dictionary predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=16 row-size=32B cardinality=288.04K cost=99931224
+|     tuple-ids=16 row-size=32B cardinality=170.55K(filtered from 288.04K) cost=5966980
 |     in pipelines: 23(GETNEXT)
 |
 |--22:SELECT
@@ -6809,8 +6809,8 @@ PLAN-ROOT SINK
 |  |  in pipelines: 12(GETNEXT)
 |  |
 |  F08:PLAN FRAGMENT [HASH(cs.cs_item_sk,cs.cs_order_number)] hosts=3 instances=6 (adjusted from 48)
-|  Per-Instance Resources: mem-estimate=13.29MB mem-reservation=2.00MB thread-reservation=1
-|  max-parallelism=6 segment-costs=[480048, 499] cpu-comparison-result=22 [max(18 (self) vs 22 (sum children))]
+|  Per-Instance Resources: mem-estimate=13.08MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=6 segment-costs=[302670, 499] cpu-comparison-result=22 [max(12 (self) vs 22 (sum children))]
 |  17:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(coalesce(cr.cr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(cs.cs_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(cr.cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(cs.cs_net_paid, CAST(0 AS DECIMAL(7,2))))
 |  |  group by: cs.cs_item_sk
@@ -6823,7 +6823,7 @@ PLAN-ROOT SINK
 |  |  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,10 row-size=68B cardinality=8.50K cost=144155
+|  |  tuple-ids=8,9N,10 row-size=68B cardinality=8.50K cost=85029
 |  |  in pipelines: 12(GETNEXT), 14(OPEN)
 |  |
 |  |--F21:PLAN FRAGMENT [HASH(cs.cs_item_sk,cs.cs_order_number)] hosts=3 instances=3
@@ -6862,7 +6862,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs.cs_item_sk = cr.cr_item_sk, cs.cs_order_number = cr.cr_order_number
 |  |  other predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N row-size=56B cardinality=144.16K cost=288310
+|  |  tuple-ids=8,9N row-size=56B cardinality=85.03K(filtered from 144.16K) cost=170058
 |  |  in pipelines: 12(GETNEXT), 13(OPEN)
 |  |
 |  |--F22:PLAN FRAGMENT [HASH(cs.cs_item_sk,cs.cs_order_number)] hosts=3 instances=6 (adjusted from 48)
@@ -6896,14 +6896,14 @@ PLAN-ROOT SINK
 |  |     in pipelines: 13(GETNEXT)
 |  |
 |  42:EXCHANGE [HASH(cs.cs_item_sk,cs.cs_order_number)]
-|  |  mem-estimate=1.89MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=8 row-size=32B cardinality=144.16K cost=5068
+|  |  mem-estimate=1.68MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=32B cardinality=85.03K(filtered from 144.16K) cost=5068
 |  |  in pipelines: 12(GETNEXT)
 |  |
-|  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=16.84MB mem-reservation=256.00KB thread-reservation=1
-|  max-parallelism=12 segment-costs=[95924764]
+|  max-parallelism=6 segment-costs=[5710734]
 |  12:SCAN HDFS [tpcds_partitioned_parquet_snap.catalog_sales cs, RANDOM]
 |     HDFS partitions=1831/1831 files=1831 size=151.13MB
 |     predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
@@ -6912,11 +6912,11 @@ PLAN-ROOT SINK
 |       table: rows=1.44M size=151.13MB
 |       partitions: 1831/1831 rows=1.44M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=7.18K
+|     extrapolated-rows=disabled max-scan-range-rows=7.18K est-scan-range=109(filtered from 1831)
 |     parquet statistics predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
 |     parquet dictionary predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=8 row-size=32B cardinality=144.16K cost=95919696
+|     tuple-ids=8 row-size=32B cardinality=85.03K(filtered from 144.16K) cost=5707744
 |     in pipelines: 12(GETNEXT)
 |
 11:SELECT
@@ -6977,8 +6977,8 @@ max-parallelism=6 segment-costs=[21550, 4260, 317] cpu-comparison-result=22 [max
 |  in pipelines: 01(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(ws.ws_item_sk,ws.ws_order_number)] hosts=3 instances=6 (adjusted from 48)
-Per-Instance Resources: mem-estimate=12.56MB mem-reservation=2.00MB thread-reservation=1
-max-parallelism=6 segment-costs=[239644, 250] cpu-comparison-result=22 [max(18 (self) vs 22 (sum children))]
+Per-Instance Resources: mem-estimate=12.35MB mem-reservation=2.00MB thread-reservation=1
+max-parallelism=6 segment-costs=[151615, 250] cpu-comparison-result=22 [max(12 (self) vs 22 (sum children))]
 06:AGGREGATE [STREAMING]
 |  output: sum(CAST(coalesce(wr.wr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(ws.ws_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(wr.wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(ws.ws_net_paid, CAST(0 AS DECIMAL(7,2))))
 |  group by: ws.ws_item_sk
@@ -6991,7 +6991,7 @@ max-parallelism=6 segment-costs=[239644, 250] cpu-comparison-result=22 [max(18 (
 |  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=0,1N,2 row-size=68B cardinality=4.26K cost=71938
+|  tuple-ids=0,1N,2 row-size=68B cardinality=4.26K cost=42595
 |  in pipelines: 01(GETNEXT), 03(OPEN)
 |
 |--F19:PLAN FRAGMENT [HASH(ws.ws_item_sk,ws.ws_order_number)] hosts=3 instances=3
@@ -7030,7 +7030,7 @@ max-parallelism=6 segment-costs=[239644, 250] cpu-comparison-result=22 [max(18 (
 |  fk/pk conjuncts: ws.ws_item_sk = wr.wr_item_sk, ws.ws_order_number = wr.wr_order_number
 |  other predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N row-size=56B cardinality=71.94K cost=143876
+|  tuple-ids=0,1N row-size=56B cardinality=42.59K(filtered from 71.94K) cost=85190
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F20:PLAN FRAGMENT [HASH(ws.ws_item_sk,ws.ws_order_number)] hosts=3 instances=6 (adjusted from 48)
@@ -7064,14 +7064,14 @@ max-parallelism=6 segment-costs=[239644, 250] cpu-comparison-result=22 [max(18 (
 |     in pipelines: 02(GETNEXT)
 |
 36:EXCHANGE [HASH(ws.ws_item_sk,ws.ws_order_number)]
-|  mem-estimate=1.15MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0 row-size=32B cardinality=71.94K cost=2530
+|  mem-estimate=965.35KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0 row-size=32B cardinality=42.59K(filtered from 71.94K) cost=2530
 |  in pipelines: 01(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 Per-Instance Resources: mem-estimate=16.84MB mem-reservation=128.00KB thread-reservation=1
-max-parallelism=12 segment-costs=[93383162]
+max-parallelism=6 segment-costs=[5580615]
 01:SCAN HDFS [tpcds_partitioned_parquet_snap.web_sales ws, RANDOM]
    HDFS partitions=1824/1824 files=1824 size=84.20MB
    predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
@@ -7080,11 +7080,11 @@ max-parallelism=12 segment-costs=[93383162]
      table: rows=719.38K size=84.20MB
      partitions: 1824/1824 rows=719.38K
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=1.12K
+   extrapolated-rows=disabled max-scan-range-rows=1.12K est-scan-range=109(filtered from 1824)
    parquet statistics predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
    parquet dictionary predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
    mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-   tuple-ids=0 row-size=32B cardinality=71.94K cost=93380632
+   tuple-ids=0 row-size=32B cardinality=42.59K(filtered from 71.94K) cost=5579117
    in pipelines: 01(GETNEXT)
 ====
 # TPCDS-Q76
@@ -7575,8 +7575,8 @@ ORDER BY ss_sold_year,
          ratio
 LIMIT 100;
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=274.88MB Threads=44
-Per-Host Resource Estimates: Memory=863MB
+Max Per-Host Resource Reservation: Memory=188.00MB Threads=38
+Per-Host Resource Estimates: Memory=624MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 |  max-parallelism=1 segment-costs=[1012] cpu-comparison-result=57 [max(1 (self) vs 57 (sum children))]
@@ -7592,7 +7592,7 @@ PLAN-ROOT SINK
 |  in pipelines: 20(GETNEXT)
 |
 F03:PLAN FRAGMENT [HASH(d_year,cs_item_sk,cs_bill_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-Per-Instance Resources: mem-estimate=15.93MB mem-reservation=4.75MB thread-reservation=1
+Per-Instance Resources: mem-estimate=15.58MB mem-reservation=4.75MB thread-reservation=1
 max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [max(6 (self) vs 57 (sum children))]
 20:TOP-N [LIMIT=100]
 |  order by: ss_sold_year ASC, ss_item_sk ASC, ss_customer_sk ASC, ss_qty DESC, ss_wc DESC, ss_sp DESC, coalesce(ws_qty, 0) + coalesce(cs_qty, 0) ASC, coalesce(ws_wc, 0) + coalesce(cs_wc, 0) ASC, coalesce(ws_sp, 0) + coalesce(cs_sp, 0) ASC, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2) ASC
@@ -7625,7 +7625,7 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |  in pipelines: 28(GETNEXT)
 |  |
 |  F07:PLAN FRAGMENT [HASH(d_year,ws_item_sk,ws_bill_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-|  Per-Instance Resources: mem-estimate=13.32MB mem-reservation=2.88MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=12.97MB mem-reservation=2.88MB thread-reservation=1
 |  max-parallelism=6 segment-costs=[891286, 441686] cpu-comparison-result=38 [max(6 (self) vs 38 (sum children))]
 |  18:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
 |  |  hash-table-id=01
@@ -7650,7 +7650,7 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |  |  in pipelines: 32(GETNEXT)
 |  |  |
 |  |  F11:PLAN FRAGMENT [HASH(d_year,ss_item_sk,ss_customer_sk)] hosts=3 instances=6 (adjusted from 48)
-|  |  Per-Instance Resources: mem-estimate=20.70MB mem-reservation=8.50MB thread-reservation=1
+|  |  Per-Instance Resources: mem-estimate=20.35MB mem-reservation=8.50MB thread-reservation=1
 |  |  max-parallelism=6 segment-costs=[3568694, 177] cpu-comparison-result=19 [max(6 (self) vs 19 (sum children))]
 |  |  32:AGGREGATE [FINALIZE]
 |  |  |  output: sum:merge(ss_quantity), sum:merge(ss_wholesale_cost), sum:merge(ss_sales_price)
@@ -7661,18 +7661,18 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |  |  in pipelines: 32(GETNEXT), 00(OPEN)
 |  |  |
 |  |  31:EXCHANGE [HASH(d_year,ss_item_sk,ss_customer_sk)]
-|  |  |  mem-estimate=10.70MB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=10.35MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=3 row-size=56B cardinality=589.03K cost=34514
 |  |  |  in pipelines: 00(GETNEXT)
 |  |  |
-|  |  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  |  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-|  |  Per-Instance Resources: mem-estimate=27.41MB mem-reservation=9.00MB thread-reservation=1
-|  |  max-parallelism=12 segment-costs=[103476656, 34514] cpu-comparison-result=19 [max(12 (self) vs 19 (sum children))]
+|  |  Per-Instance Resources: mem-estimate=27.41MB mem-reservation=13.00MB thread-reservation=1
+|  |  max-parallelism=6 segment-costs=[24021978, 34514] cpu-comparison-result=19 [max(6 (self) vs 19 (sum children))]
 |  |  05:AGGREGATE [STREAMING]
 |  |  |  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
 |  |  |  group by: d_year, ss_item_sk, ss_customer_sk
-|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
 |  |  |  tuple-ids=3 row-size=56B cardinality=589.03K cost=3534180
 |  |  |  in pipelines: 00(GETNEXT)
 |  |  |
@@ -7681,17 +7681,17 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |  |  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,2 row-size=60B cardinality=589.03K cost=2880404
+|  |  |  tuple-ids=0,1N,2 row-size=60B cardinality=589.03K cost=589030
 |  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
 |  |  |
 |  |  |--F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  |  max-parallelism=3 segment-costs=[388]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: d_date_sk
 |  |  |  |  runtime filters: RF016[bloom] <- d_date_sk
-|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=373
 |  |  |  |
 |  |  |  30:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -7720,16 +7720,16 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |  |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
 |  |  |  other predicates: sr_ticket_number IS NULL
 |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
-|  |  |  tuple-ids=0,1N row-size=52B cardinality=2.88M cost=5760808
+|  |  |  tuple-ids=0,1N row-size=52B cardinality=589.03K(filtered from 2.88M) cost=1178060
 |  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
 |  |  |
 |  |  |--F16:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  |  Per-Instance Resources: mem-estimate=72.62MB mem-reservation=68.00MB thread-reservation=1
+|  |  |  |  Per-Instance Resources: mem-estimate=38.62MB mem-reservation=34.00MB thread-reservation=1
 |  |  |  |  max-parallelism=3 segment-costs=[591876]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  |  build expressions: sr_item_sk, sr_ticket_number
-|  |  |  |  mem-estimate=68.00MB mem-reservation=68.00MB spill-buffer=1.00MB thread-reservation=0 cost=575028
+|  |  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=1.00MB thread-reservation=0 cost=575028
 |  |  |  |
 |  |  |  29:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=4.62MB mem-reservation=0B thread-reservation=0
@@ -7757,9 +7757,9 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |       table: rows=2.88M size=199.36MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|  |     tuple-ids=0 row-size=36B cardinality=2.88M cost=91301264
+|  |     tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M) cost=18720708
 |  |     in pipelines: 00(GETNEXT)
 |  |
 |  28:AGGREGATE [FINALIZE]
@@ -7770,18 +7770,18 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |  in pipelines: 28(GETNEXT), 06(OPEN)
 |  |
 |  27:EXCHANGE [HASH(d_year,ws_item_sk,ws_bill_customer_sk)]
-|  |  mem-estimate=3.32MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=2.97MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=8 row-size=56B cardinality=147.11K cost=8620
 |  |  in pipelines: 06(GETNEXT)
 |  |
-|  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+|  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
-|  Per-Instance Resources: mem-estimate=27.41MB mem-reservation=2.12MB thread-reservation=1
-|  max-parallelism=12 segment-costs=[94266106, 8620] cpu-comparison-result=19 [max(12 (self) vs 19 (sum children))]
+|  Per-Instance Resources: mem-estimate=27.41MB mem-reservation=3.12MB thread-reservation=1
+|  max-parallelism=6 segment-costs=[20029171, 8620] cpu-comparison-result=19 [max(6 (self) vs 19 (sum children))]
 |  11:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ws_quantity AS BIGINT)), sum(ws_wholesale_cost), sum(ws_sales_price)
 |  |  group by: d_year, ws_item_sk, ws_bill_customer_sk
-|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  mem-estimate=10.00MB mem-reservation=3.00MB spill-buffer=128.00KB thread-reservation=0
 |  |  tuple-ids=8 row-size=56B cardinality=147.11K cost=882666
 |  |  in pipelines: 06(GETNEXT)
 |  |
@@ -7790,17 +7790,17 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |  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=5,6N,7 row-size=60B cardinality=147.11K cost=719384
+|  |  tuple-ids=5,6N,7 row-size=60B cardinality=147.11K cost=147111
 |  |  in pipelines: 06(GETNEXT), 08(OPEN)
 |  |
 |  |--F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  max-parallelism=3 segment-costs=[388]
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=02
 |  |  |  build expressions: d_date_sk
 |  |  |  runtime filters: RF014[bloom] <- d_date_sk
-|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=373
 |  |  |
 |  |  26:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -7831,16 +7831,16 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  |  fk/pk conjuncts: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
 |  |  other predicates: wr_order_number IS NULL
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
-|  |  tuple-ids=5,6N row-size=52B cardinality=719.38K cost=1438768
+|  |  tuple-ids=5,6N row-size=52B cardinality=147.11K(filtered from 719.38K) cost=294222
 |  |  in pipelines: 06(GETNEXT), 07(OPEN)
 |  |
 |  |--F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  |  Per-Instance Resources: mem-estimate=20.33MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  Per-Instance Resources: mem-estimate=10.83MB mem-reservation=9.50MB thread-reservation=1
 |  |  |  max-parallelism=3 segment-costs=[147732]
 |  |  JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=02
 |  |  |  build expressions: wr_item_sk, wr_order_number
-|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=143526
+|  |  |  mem-estimate=9.50MB mem-reservation=9.50MB spill-buffer=256.00KB thread-reservation=0 cost=143526
 |  |  |
 |  |  25:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=1.33MB mem-reservation=0B thread-reservation=0
@@ -7870,9 +7870,9 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |       table: rows=719.38K size=84.20MB
 |       partitions: 1824/1824 rows=719.38K
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=1.12K
+|     extrapolated-rows=disabled max-scan-range-rows=1.12K est-scan-range=374(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-|     tuple-ids=5 row-size=36B cardinality=719.38K cost=91225288
+|     tuple-ids=5 row-size=36B cardinality=147.11K(filtered from 719.38K) cost=18705172
 |     in pipelines: 06(GETNEXT)
 |
 24:AGGREGATE [FINALIZE]
@@ -7883,18 +7883,18 @@ max-parallelism=6 segment-costs=[1779191, 881992, 12] cpu-comparison-result=57 [
 |  in pipelines: 24(GETNEXT), 12(OPEN)
 |
 23:EXCHANGE [HASH(d_year,cs_item_sk,cs_bill_customer_sk)]
-|  mem-estimate=5.93MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=5.58MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=13 row-size=56B cardinality=293.66K cost=17207
 |  in pipelines: 12(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6 (adjusted from 48)
 Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
-Per-Instance Resources: mem-estimate=27.41MB mem-reservation=3.25MB thread-reservation=1
-max-parallelism=12 segment-costs=[97687316, 17207] cpu-comparison-result=19 [max(12 (self) vs 19 (sum children))]
+Per-Instance Resources: mem-estimate=27.41MB mem-reservation=5.25MB thread-reservation=1
+max-parallelism=6 segment-costs=[21353300, 17207] cpu-comparison-result=19 [max(6 (self) vs 19 (sum children))]
 17:AGGREGATE [STREAMING]
 |  output: sum(CAST(cs_quantity AS BIGINT)), sum(cs_wholesale_cost), sum(cs_sales_price)
 |  group by: d_year, cs_item_sk, cs_bill_customer_sk
-|  mem-estimate=10.00MB mem-reservation=3.00MB spill-buffer=128.00KB thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
 |  tuple-ids=13 row-size=56B cardinality=293.66K cost=1761984
 |  in pipelines: 12(GETNEXT)
 |
@@ -7903,17 +7903,17 @@ max-parallelism=12 segment-costs=[97687316, 17207] cpu-comparison-result=19 [max
 |  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=10,11N,12 row-size=60B cardinality=293.66K cost=1441548
+|  tuple-ids=10,11N,12 row-size=60B cardinality=293.66K cost=293664
 |  in pipelines: 12(GETNEXT), 14(OPEN)
 |
 |--F19:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  max-parallelism=3 segment-costs=[388]
 |  JOIN BUILD
 |  |  join-table-id=06 plan-id=07 cohort-id=01
 |  |  build expressions: d_date_sk
 |  |  runtime filters: RF006[bloom] <- d_date_sk
-|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0 cost=373
 |  |
 |  22:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -7944,16 +7944,16 @@ max-parallelism=12 segment-costs=[97687316, 17207] cpu-comparison-result=19 [max
 |  fk/pk conjuncts: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
 |  other predicates: cr_order_number IS NULL
 |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=10,11N row-size=52B cardinality=1.44M cost=2883096
+|  tuple-ids=10,11N row-size=52B cardinality=293.66K(filtered from 1.44M) cost=587328
 |  in pipelines: 12(GETNEXT), 13(OPEN)
 |
 |--F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  |  Per-Instance Resources: mem-estimate=36.43MB mem-reservation=34.00MB thread-reservation=1
+|  |  Per-Instance Resources: mem-estimate=19.43MB mem-reservation=17.00MB thread-reservation=1
 |  |  max-parallelism=3 segment-costs=[296576]
 |  JOIN BUILD
 |  |  join-table-id=07 plan-id=08 cohort-id=01
 |  |  build expressions: cr_item_sk, cr_order_number
-|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=512.00KB thread-reservation=0 cost=288134
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=512.00KB thread-reservation=0 cost=288134
 |  |
 |  21:EXCHANGE [BROADCAST]
 |  |  mem-estimate=2.43MB mem-reservation=0B thread-reservation=0
@@ -7983,9 +7983,9 @@ max-parallelism=12 segment-costs=[97687316, 17207] cpu-comparison-result=19 [max
      table: rows=1.44M size=151.13MB
      partitions: 1831/1831 rows=1.44M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=7.18K
+   extrapolated-rows=disabled max-scan-range-rows=7.18K est-scan-range=374(filtered from 1831)
    mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-   tuple-ids=10 row-size=36B cardinality=1.44M cost=91600688
+   tuple-ids=10 row-size=36B cardinality=293.66K(filtered from 1.44M) cost=18710324
    in pipelines: 12(GETNEXT)
 ====
 # TPCDS-Q80
@@ -8171,7 +8171,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 |  F14:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  Per-Instance Resources: mem-estimate=29.75MB mem-reservation=2.12MB thread-reservation=1
-|  max-parallelism=12 segment-costs=[93875583, 2] cpu-comparison-result=31 [max(12 (self) vs 31 (sum children))]
+|  max-parallelism=12 segment-costs=[91910793, 2] cpu-comparison-result=31 [max(12 (self) vs 31 (sum children))]
 |  36:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_sales_price), sum(coalesce(wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(ws_net_profit - coalesce(wr_net_loss, CAST(0 AS DECIMAL(7,2))))
 |  |  group by: web_site_id
@@ -8294,7 +8294,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 |  |  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,17N,20 row-size=72B cardinality=71.94K cost=719384
+|  |  tuple-ids=16,17N,20 row-size=72B cardinality=71.94K cost=72038
 |  |  in pipelines: 25(GETNEXT), 29(OPEN)
 |  |
 |  |--F37:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -8332,7 +8332,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 |  |  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=256.00KB thread-reservation=0
-|  |  tuple-ids=16,17N row-size=60B cardinality=719.38K cost=1438768
+|  |  tuple-ids=16,17N row-size=60B cardinality=72.04K(filtered from 719.38K) cost=144076
 |  |  in pipelines: 25(GETNEXT), 26(OPEN)
 |  |
 |  |--F38:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -8373,7 +8373,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.12K
 |     mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=0
-|     tuple-ids=16 row-size=36B cardinality=719.38K cost=91225288
+|     tuple-ids=16 row-size=36B cardinality=72.04K(filtered from 719.38K) cost=91202536
 |     in pipelines: 25(GETNEXT)
 |
 |--53:AGGREGATE [FINALIZE]
@@ -8391,7 +8391,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
 |  Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB
 |  Per-Instance Resources: mem-estimate=29.75MB mem-reservation=2.25MB thread-reservation=1
-|  max-parallelism=12 segment-costs=[96911509, 902] cpu-comparison-result=31 [max(12 (self) vs 31 (sum children))]
+|  max-parallelism=12 segment-costs=[92974322, 902] cpu-comparison-result=31 [max(12 (self) vs 31 (sum children))]
 |  24:AGGREGATE [STREAMING]
 |  |  output: sum(cs_ext_sales_price), sum(coalesce(cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(cs_net_profit - coalesce(cr_net_loss, CAST(0 AS DECIMAL(7,2))))
 |  |  group by: cp_catalog_page_id
@@ -8515,7 +8515,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 |  |  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,9N,12 row-size=72B cardinality=144.16K cost=1441548
+|  |  tuple-ids=8,9N,12 row-size=72B cardinality=144.16K cost=144355
 |  |  in pipelines: 13(GETNEXT), 17(OPEN)
 |  |
 |  |--F32:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -8553,7 +8553,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 |  |  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=512.00KB thread-reservation=0
-|  |  tuple-ids=8,9N row-size=60B cardinality=1.44M cost=2883096
+|  |  tuple-ids=8,9N row-size=60B cardinality=144.35K(filtered from 1.44M) cost=288710
 |  |  in pipelines: 13(GETNEXT), 14(OPEN)
 |  |
 |  |--F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -8594,7 +8594,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=7.18K
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=8 row-size=36B cardinality=1.44M cost=91600688
+|     tuple-ids=8 row-size=36B cardinality=144.35K(filtered from 1.44M) cost=91555080
 |     in pipelines: 13(GETNEXT)
 |
 46:AGGREGATE [FINALIZE]
@@ -8612,7 +8612,7 @@ max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-re
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12 (adjusted from 48)
 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB
 Per-Instance Resources: mem-estimate=29.75MB mem-reservation=6.00MB thread-reservation=1
-max-parallelism=12 segment-costs=[101912983, 1] cpu-comparison-result=31 [max(12 (self) vs 31 (sum children))]
+max-parallelism=12 segment-costs=[94045966, 1] cpu-comparison-result=31 [max(12 (self) vs 31 (sum children))]
 12:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price), sum(coalesce(sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(ss_net_profit - coalesce(sr_net_loss, CAST(0 AS DECIMAL(7,2))))
 |  group by: s_store_id
@@ -8736,7 +8736,7 @@ max-parallelism=12 segment-costs=[101912983, 1] cpu-comparison-result=31 [max(12
 |  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,1N,4 row-size=72B cardinality=288.04K cost=2880404
+|  tuple-ids=0,1N,4 row-size=72B cardinality=288.04K cost=288441
 |  in pipelines: 01(GETNEXT), 05(OPEN)
 |
 |--F27:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -8774,7 +8774,7 @@ max-parallelism=12 segment-costs=[101912983, 1] cpu-comparison-result=31 [max(12
 |  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=1.00MB thread-reservation=0
-|  tuple-ids=0,1N row-size=60B cardinality=2.88M cost=5760808
+|  tuple-ids=0,1N row-size=60B cardinality=288.44K(filtered from 2.88M) cost=576882
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -8815,7 +8815,7 @@ max-parallelism=12 segment-costs=[101912983, 1] cpu-comparison-result=31 [max(12
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=36B cardinality=2.88M cost=91301264
+   tuple-ids=0 row-size=36B cardinality=288.44K(filtered from 2.88M) cost=91210136
    in pipelines: 01(GETNEXT)
 ====
 # IMPALA-12192: Test that scan fragment parallelism can scale beyond the scan node cost.
@@ -8918,9 +8918,9 @@ max-parallelism=21 segment-costs=[194962048, 2276043]
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpcds_partitioned_parquet_snap.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=199.46MB
+   HDFS partitions=1824/1824 files=1824 size=199.36MB
    stored statistics:
-     table: rows=2.88M size=199.46MB
+     table: rows=2.88M size=199.36MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test
index 65cb845ac..2fab8270c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test
@@ -88,7 +88,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=6 row-size=16B cardinality=287.51K
+|     tuple-ids=6 row-size=16B cardinality=53.81K(filtered from 287.51K)
 |     in pipelines: 06(GETNEXT)
 |
 12:HASH JOIN [INNER JOIN]
@@ -157,7 +157,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=0 row-size=16B cardinality=287.51K
+|     tuple-ids=0 row-size=16B cardinality=53.81K(filtered from 287.51K)
 |     in pipelines: 00(GETNEXT)
 |
 05:SCAN HDFS [tpcds_parquet.customer]
@@ -168,7 +168,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=100.00K
    mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=5 row-size=32B cardinality=100.00K
+   tuple-ids=5 row-size=32B cardinality=53.81K(filtered from 100.00K)
    in pipelines: 05(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=49.20MB Threads=16
@@ -287,7 +287,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=6 row-size=16B cardinality=287.51K
+|     tuple-ids=6 row-size=16B cardinality=53.81K(filtered from 287.51K)
 |     in pipelines: 06(GETNEXT)
 |
 12:HASH JOIN [INNER JOIN, BROADCAST]
@@ -391,7 +391,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=0 row-size=16B cardinality=287.51K
+|     tuple-ids=0 row-size=16B cardinality=53.81K(filtered from 287.51K)
 |     in pipelines: 00(GETNEXT)
 |
 05:SCAN HDFS [tpcds_parquet.customer, RANDOM]
@@ -402,7 +402,7 @@ Per-Host Resources: mem-estimate=44.92MB mem-reservation=13.62MB thread-reservat
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=100.00K
    mem-estimate=32.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=5 row-size=32B cardinality=100.00K
+   tuple-ids=5 row-size=32B cardinality=53.81K(filtered from 100.00K)
    in pipelines: 05(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=64.70MB Threads=15
@@ -539,7 +539,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |     mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=6 row-size=16B cardinality=287.51K
+|     tuple-ids=6 row-size=16B cardinality=53.81K(filtered from 287.51K)
 |     in pipelines: 06(GETNEXT)
 |
 12:HASH JOIN [INNER JOIN, BROADCAST]
@@ -669,7 +669,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |     mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=0 row-size=16B cardinality=287.51K
+|     tuple-ids=0 row-size=16B cardinality=53.81K(filtered from 287.51K)
 |     in pipelines: 00(GETNEXT)
 |
 05:SCAN HDFS [tpcds_parquet.customer, RANDOM]
@@ -680,6 +680,6 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=2.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=100.00K
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=5 row-size=32B cardinality=100.00K
+   tuple-ids=5 row-size=32B cardinality=53.81K(filtered from 100.00K)
    in pipelines: 05(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test
index 66213f4ef..8bc2397f2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test
@@ -95,7 +95,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=3.04K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=16.56MB Threads=9
@@ -214,7 +214,7 @@ Per-Host Resources: mem-estimate=36.74MB mem-reservation=4.94MB thread-reservati
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=3.04K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=27.38MB Threads=12
@@ -350,6 +350,6 @@ Per-Instance Resources: mem-estimate=17.60MB mem-reservation=1.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=3.04K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
index 80d88016f..a3bb209b8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
@@ -199,7 +199,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |  |     mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
-|  |  |     tuple-ids=79 row-size=24B cardinality=719.38K
+|  |  |     tuple-ids=79 row-size=24B cardinality=148.00K(filtered from 719.38K)
 |  |  |     in pipelines: 37(GETNEXT)
 |  |  |
 |  |  36:SCAN HDFS [tpcds_parquet.customer]
@@ -281,9 +281,9 @@ PLAN-ROOT SINK
 |  |       table: rows=2.88M size=200.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=24B cardinality=2.88M
+|  |     tuple-ids=1 row-size=24B cardinality=589.03K(filtered from 2.88M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  43:HASH JOIN [INNER JOIN]
@@ -344,7 +344,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |  |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|  |  |     tuple-ids=33 row-size=24B cardinality=1.44M
+|  |  |     tuple-ids=33 row-size=24B cardinality=294.63K(filtered from 1.44M)
 |  |  |     in pipelines: 16(GETNEXT)
 |  |  |
 |  |  15:SCAN HDFS [tpcds_parquet.customer]
@@ -416,7 +416,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |  |     mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
-|  |  |     tuple-ids=65 row-size=24B cardinality=719.38K
+|  |  |     tuple-ids=65 row-size=24B cardinality=148.00K(filtered from 719.38K)
 |  |  |     in pipelines: 30(GETNEXT)
 |  |  |
 |  |  29:SCAN HDFS [tpcds_parquet.customer]
@@ -489,9 +489,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=15 row-size=24B cardinality=2.88M
+|     tuple-ids=15 row-size=24B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 09(GETNEXT)
 |
 21:UNION
@@ -543,7 +543,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=47 row-size=24B cardinality=1.44M
+|     tuple-ids=47 row-size=24B cardinality=294.63K(filtered from 1.44M)
 |     in pipelines: 23(GETNEXT)
 |
 22:SCAN HDFS [tpcds_parquet.customer]
@@ -698,7 +698,7 @@ Per-Host Resources: mem-estimate=19.90MB mem-reservation=13.38MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
-|  |     tuple-ids=79 row-size=24B cardinality=719.38K
+|  |     tuple-ids=79 row-size=24B cardinality=148.00K(filtered from 719.38K)
 |  |     in pipelines: 37(GETNEXT)
 |  |
 |  44:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -811,9 +811,9 @@ Per-Host Resources: mem-estimate=19.90MB mem-reservation=13.38MB thread-reservat
 |  |       table: rows=2.88M size=200.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=24B cardinality=2.88M
+|  |     tuple-ids=1 row-size=24B cardinality=589.03K(filtered from 2.88M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  75:EXCHANGE [HASH(customer_id)]
@@ -934,7 +934,7 @@ Per-Host Resources: mem-estimate=19.90MB mem-reservation=13.38MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|  |     tuple-ids=33 row-size=24B cardinality=1.44M
+|  |     tuple-ids=33 row-size=24B cardinality=294.63K(filtered from 1.44M)
 |  |     in pipelines: 16(GETNEXT)
 |  |
 |  42:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1048,7 +1048,7 @@ Per-Host Resources: mem-estimate=19.90MB mem-reservation=13.38MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
-|  |     tuple-ids=65 row-size=24B cardinality=719.38K
+|  |     tuple-ids=65 row-size=24B cardinality=148.00K(filtered from 719.38K)
 |  |     in pipelines: 30(GETNEXT)
 |  |
 |  07:UNION
@@ -1145,9 +1145,9 @@ Per-Host Resources: mem-estimate=19.90MB mem-reservation=13.38MB thread-reservat
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=15 row-size=24B cardinality=2.88M
+|     tuple-ids=15 row-size=24B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 09(GETNEXT)
 |
 83:EXCHANGE [HASH(customer_id)]
@@ -1252,7 +1252,7 @@ Per-Host Resources: mem-estimate=292.41MB mem-reservation=19.94MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=47 row-size=24B cardinality=1.44M
+   tuple-ids=47 row-size=24B cardinality=294.63K(filtered from 1.44M)
    in pipelines: 23(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=645.25MB Threads=57
@@ -1429,7 +1429,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-|  |     tuple-ids=79 row-size=24B cardinality=719.38K
+|  |     tuple-ids=79 row-size=24B cardinality=148.00K(filtered from 719.38K)
 |  |     in pipelines: 37(GETNEXT)
 |  |
 |  44:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1567,9 +1567,9 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio
 |  |       table: rows=2.88M size=200.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|  |     tuple-ids=1 row-size=24B cardinality=2.88M
+|  |     tuple-ids=1 row-size=24B cardinality=589.03K(filtered from 2.88M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  75:EXCHANGE [HASH(customer_id)]
@@ -1715,7 +1715,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
-|  |     tuple-ids=33 row-size=24B cardinality=1.44M
+|  |     tuple-ids=33 row-size=24B cardinality=294.63K(filtered from 1.44M)
 |  |     in pipelines: 16(GETNEXT)
 |  |
 |  42:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1855,7 +1855,7 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-|  |     tuple-ids=65 row-size=24B cardinality=719.38K
+|  |     tuple-ids=65 row-size=24B cardinality=148.00K(filtered from 719.38K)
 |  |     in pipelines: 30(GETNEXT)
 |  |
 |  07:UNION
@@ -1970,9 +1970,9 @@ Per-Instance Resources: mem-estimate=4.27MB mem-reservation=0B thread-reservatio
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=15 row-size=24B cardinality=2.88M
+|     tuple-ids=15 row-size=24B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 09(GETNEXT)
 |
 83:EXCHANGE [HASH(customer_id)]
@@ -2095,6 +2095,6 @@ Per-Instance Resources: mem-estimate=48.47MB mem-reservation=16.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
-   tuple-ids=47 row-size=24B cardinality=1.44M
+   tuple-ids=47 row-size=24B cardinality=294.63K(filtered from 1.44M)
    in pipelines: 23(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test
index ab0ac7aee..05b448df1 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test
@@ -135,9 +135,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=39.28MB Threads=12
@@ -298,9 +298,9 @@ Per-Host Resources: mem-estimate=145.85MB mem-reservation=24.31MB thread-reserva
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=65.53MB Threads=13
@@ -494,8 +494,8 @@ Per-Instance Resources: mem-estimate=27.50MB mem-reservation=6.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
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 0d6e55e92..1a7f665a9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
@@ -150,7 +150,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=6 row-size=21B cardinality=50.00K
+|  |     tuple-ids=6 row-size=21B cardinality=49.54K(filtered from 50.00K)
 |  |     in pipelines: 05(GETNEXT)
 |  |
 |  04:AGGREGATE [FINALIZE]
@@ -218,9 +218,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=0 row-size=12B cardinality=2.88M
+   tuple-ids=0 row-size=12B cardinality=293.73K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=35.96MB Threads=15
@@ -346,7 +346,7 @@ Per-Host Resources: mem-estimate=51.24MB mem-reservation=11.81MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |  |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=6 row-size=21B cardinality=50.00K
+|  |     tuple-ids=6 row-size=21B cardinality=49.54K(filtered from 50.00K)
 |  |     in pipelines: 05(GETNEXT)
 |  |
 |  18:AGGREGATE [FINALIZE]
@@ -441,9 +441,9 @@ Per-Host Resources: mem-estimate=51.24MB mem-reservation=11.81MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=0 row-size=12B cardinality=2.88M
+   tuple-ids=0 row-size=12B cardinality=293.73K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=54.46MB Threads=16
@@ -595,7 +595,7 @@ Per-Instance Resources: mem-estimate=26.85MB mem-reservation=3.00MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=6 row-size=21B cardinality=50.00K
+|  |     tuple-ids=6 row-size=21B cardinality=49.54K(filtered from 50.00K)
 |  |     in pipelines: 05(GETNEXT)
 |  |
 |  18:AGGREGATE [FINALIZE]
@@ -707,8 +707,8 @@ Per-Instance Resources: mem-estimate=26.85MB mem-reservation=3.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=0 row-size=12B cardinality=2.88M
+   tuple-ids=0 row-size=12B cardinality=293.73K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test
index e0e47ee32..27983be2c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test
@@ -142,7 +142,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=2 row-size=87B cardinality=1.92M
+|  |     tuple-ids=2 row-size=87B cardinality=100.00K(filtered from 1.92M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  05:HASH JOIN [INNER JOIN]
@@ -173,9 +173,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=3 row-size=8B cardinality=2.88M
+|     tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
 |     in pipelines: 03(GETNEXT)
 |
 06:UNION
@@ -212,7 +212,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=8 row-size=8B cardinality=1.44M
+|     tuple-ids=8 row-size=8B cardinality=93.21K(filtered from 1.44M)
 |     in pipelines: 10(GETNEXT)
 |
 09:HASH JOIN [INNER JOIN]
@@ -244,7 +244,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=6 row-size=8B cardinality=719.38K
+   tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=52.62MB Threads=19
@@ -374,7 +374,7 @@ Per-Host Resources: mem-estimate=17.34MB mem-reservation=7.88MB thread-reservati
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=2 row-size=87B cardinality=1.92M
+|  |     tuple-ids=2 row-size=87B cardinality=100.00K(filtered from 1.92M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  24:EXCHANGE [HASH(ss_customer_sk)]
@@ -419,9 +419,9 @@ Per-Host Resources: mem-estimate=17.34MB mem-reservation=7.88MB thread-reservati
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=3 row-size=8B cardinality=2.88M
+|     tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
 |     in pipelines: 03(GETNEXT)
 |
 26:EXCHANGE [HASH(customer_sk)]
@@ -472,7 +472,7 @@ Per-Host Resources: mem-estimate=101.05MB mem-reservation=8.94MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=8 row-size=8B cardinality=1.44M
+|     tuple-ids=8 row-size=8B cardinality=93.21K(filtered from 1.44M)
 |     in pipelines: 10(GETNEXT)
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
@@ -511,7 +511,7 @@ Per-Host Resources: mem-estimate=101.05MB mem-reservation=8.94MB thread-reservat
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=6 row-size=8B cardinality=719.38K
+   tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
    in pipelines: 07(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=75.31MB Threads=19
@@ -675,7 +675,7 @@ Per-Instance Resources: mem-estimate=11.31MB mem-reservation=2.00MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=2 row-size=87B cardinality=1.92M
+|  |     tuple-ids=2 row-size=87B cardinality=100.00K(filtered from 1.92M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  24:EXCHANGE [HASH(ss_customer_sk)]
@@ -729,9 +729,9 @@ Per-Instance Resources: mem-estimate=11.31MB mem-reservation=2.00MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|     tuple-ids=3 row-size=8B cardinality=2.88M
+|     tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
 |     in pipelines: 03(GETNEXT)
 |
 26:EXCHANGE [HASH(customer_sk)]
@@ -791,7 +791,7 @@ Per-Instance Resources: mem-estimate=48.09MB mem-reservation=4.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=8 row-size=8B cardinality=1.44M
+|     tuple-ids=8 row-size=8B cardinality=93.21K(filtered from 1.44M)
 |     in pipelines: 10(GETNEXT)
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
@@ -838,6 +838,6 @@ Per-Instance Resources: mem-estimate=48.09MB mem-reservation=4.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=6 row-size=8B cardinality=719.38K
+   tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
    in pipelines: 07(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
index 4985bdb19..0a6c3ef5c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
@@ -223,9 +223,9 @@ PLAN-ROOT SINK
 |  |       table: rows=2.88M size=200.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |  |     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=16B cardinality=2.88M
+|  |     tuple-ids=1 row-size=16B cardinality=589.03K(filtered from 2.88M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  01:SCAN HDFS [tpcds_parquet.customer]
@@ -359,9 +359,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=11 row-size=16B cardinality=2.88M
+|     tuple-ids=11 row-size=16B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 09(GETNEXT)
 |
 08:SCAN HDFS [tpcds_parquet.customer]
@@ -473,7 +473,7 @@ Per-Host Resources: mem-estimate=11.90MB mem-reservation=4.75MB thread-reservati
 |  |  fk/pk conjuncts: none
 |  |  runtime filters: RF020[bloom] <- c_customer_sk, RF021[min_max] <- c_customer_sk
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=35,34 row-size=169B cardinality=719.38K
+|  |  tuple-ids=35,34 row-size=169B cardinality=148.00K(filtered from 719.38K)
 |  |  in pipelines: 23(GETNEXT), 22(OPEN)
 |  |
 |  |--50:EXCHANGE [HASH(c_customer_sk)]
@@ -495,7 +495,7 @@ Per-Host Resources: mem-estimate=11.90MB mem-reservation=4.75MB thread-reservati
 |  |
 |  49:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=35 row-size=16B cardinality=719.38K
+|  |  tuple-ids=35 row-size=16B cardinality=148.00K(filtered from 719.38K)
 |  |  in pipelines: 23(GETNEXT)
 |  |
 |  F18:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -508,7 +508,7 @@ Per-Host Resources: mem-estimate=11.90MB mem-reservation=4.75MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=35 row-size=16B cardinality=719.38K
+|     tuple-ids=35 row-size=16B cardinality=148.00K(filtered from 719.38K)
 |     in pipelines: 23(GETNEXT)
 |
 54:EXCHANGE [HASH(customer_id)]
@@ -628,9 +628,9 @@ Per-Host Resources: mem-estimate=56.52MB mem-reservation=42.69MB thread-reservat
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=1 row-size=16B cardinality=2.88M
+|     tuple-ids=1 row-size=16B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 02(GETNEXT)
 |
 28:HASH JOIN [INNER JOIN, BROADCAST]
@@ -708,7 +708,7 @@ Per-Host Resources: mem-estimate=56.52MB mem-reservation=42.69MB thread-reservat
 |  |  fk/pk conjuncts: none
 |  |  runtime filters: RF012[bloom] <- c_customer_sk, RF013[min_max] <- c_customer_sk
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=25,24 row-size=169B cardinality=719.38K
+|  |  tuple-ids=25,24 row-size=169B cardinality=148.00K(filtered from 719.38K)
 |  |  in pipelines: 16(GETNEXT), 15(OPEN)
 |  |
 |  |--38:EXCHANGE [HASH(c_customer_sk)]
@@ -731,7 +731,7 @@ Per-Host Resources: mem-estimate=56.52MB mem-reservation=42.69MB thread-reservat
 |  |
 |  37:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=25 row-size=16B cardinality=719.38K
+|  |  tuple-ids=25 row-size=16B cardinality=148.00K(filtered from 719.38K)
 |  |  in pipelines: 16(GETNEXT)
 |  |
 |  F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -744,7 +744,7 @@ Per-Host Resources: mem-estimate=56.52MB mem-reservation=42.69MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=25 row-size=16B cardinality=719.38K
+|     tuple-ids=25 row-size=16B cardinality=148.00K(filtered from 719.38K)
 |     in pipelines: 16(GETNEXT)
 |
 07:UNION
@@ -841,9 +841,9 @@ Per-Host Resources: mem-estimate=52.31MB mem-reservation=5.94MB thread-reservati
      table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=11 row-size=16B cardinality=2.88M
+   tuple-ids=11 row-size=16B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=444.62MB Threads=43
@@ -960,7 +960,7 @@ Per-Instance Resources: mem-estimate=4.53MB mem-reservation=0B thread-reservatio
 |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: none
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=35,34 row-size=169B cardinality=719.38K
+|  |  tuple-ids=35,34 row-size=169B cardinality=148.00K(filtered from 719.38K)
 |  |  in pipelines: 23(GETNEXT), 22(OPEN)
 |  |
 |  |--F28:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
@@ -990,7 +990,7 @@ Per-Instance Resources: mem-estimate=4.53MB mem-reservation=0B thread-reservatio
 |  |
 |  49:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=35 row-size=16B cardinality=719.38K
+|  |  tuple-ids=35 row-size=16B cardinality=148.00K(filtered from 719.38K)
 |  |  in pipelines: 23(GETNEXT)
 |  |
 |  F18:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1004,7 +1004,7 @@ Per-Instance Resources: mem-estimate=4.53MB mem-reservation=0B thread-reservatio
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=35 row-size=16B cardinality=719.38K
+|     tuple-ids=35 row-size=16B cardinality=148.00K(filtered from 719.38K)
 |     in pipelines: 23(GETNEXT)
 |
 54:EXCHANGE [HASH(customer_id)]
@@ -1149,9 +1149,9 @@ Per-Instance Resources: mem-estimate=49.54MB mem-reservation=34.00MB thread-rese
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-|     tuple-ids=1 row-size=16B cardinality=2.88M
+|     tuple-ids=1 row-size=16B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 02(GETNEXT)
 |
 28:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1245,7 +1245,7 @@ Per-Instance Resources: mem-estimate=49.54MB mem-reservation=34.00MB thread-rese
 |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
 |  |  fk/pk conjuncts: none
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
-|  |  tuple-ids=25,24 row-size=169B cardinality=719.38K
+|  |  tuple-ids=25,24 row-size=169B cardinality=148.00K(filtered from 719.38K)
 |  |  in pipelines: 16(GETNEXT), 15(OPEN)
 |  |
 |  |--F34:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
@@ -1277,7 +1277,7 @@ Per-Instance Resources: mem-estimate=49.54MB mem-reservation=34.00MB thread-rese
 |  |
 |  37:EXCHANGE [HASH(ws_bill_customer_sk)]
 |  |  mem-estimate=5.53MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=25 row-size=16B cardinality=719.38K
+|  |  tuple-ids=25 row-size=16B cardinality=148.00K(filtered from 719.38K)
 |  |  in pipelines: 16(GETNEXT)
 |  |
 |  F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1291,7 +1291,7 @@ Per-Instance Resources: mem-estimate=49.54MB mem-reservation=34.00MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=25 row-size=16B cardinality=719.38K
+|     tuple-ids=25 row-size=16B cardinality=148.00K(filtered from 719.38K)
 |     in pipelines: 16(GETNEXT)
 |
 07:UNION
@@ -1406,8 +1406,8 @@ Per-Instance Resources: mem-estimate=16.75MB mem-reservation=2.00MB thread-reser
      table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=11 row-size=16B cardinality=2.88M
+   tuple-ids=11 row-size=16B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 09(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test
index 1bc98d160..7fc942c0b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test
@@ -111,7 +111,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=16B cardinality=719.38K
+   tuple-ids=0 row-size=16B cardinality=216.12K(filtered from 719.38K)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=105.81MB Threads=8
@@ -235,7 +235,7 @@ Per-Host Resources: mem-estimate=138.39MB mem-reservation=48.81MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=16B cardinality=719.38K
+   tuple-ids=0 row-size=16B cardinality=216.12K(filtered from 719.38K)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=112.62MB Threads=7
@@ -376,6 +376,6 @@ Per-Instance Resources: mem-estimate=67.58MB mem-reservation=42.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=0 row-size=16B cardinality=719.38K
+   tuple-ids=0 row-size=16B cardinality=216.12K(filtered from 719.38K)
    in pipelines: 00(GETNEXT)
 ====
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 a4a7bcd8a..494e9118c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
@@ -177,7 +177,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=1.92M
    mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=2 row-size=39B cardinality=181.75K
+   tuple-ids=2 row-size=39B cardinality=3.01K(filtered from 181.75K)
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=36.08MB Threads=14
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test
index 31cda716d..12519ec26 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test
@@ -526,7 +526,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=68 row-size=20B cardinality=719.38K
+|     tuple-ids=68 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 85(GETNEXT)
 |
 |--84:NESTED LOOP JOIN [INNER JOIN]
@@ -908,7 +908,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=34 row-size=20B cardinality=1.44M
+|     tuple-ids=34 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 43(GETNEXT)
 |
 42:NESTED LOOP JOIN [INNER JOIN]
@@ -1290,9 +1290,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=366.44MB Threads=120
@@ -1947,7 +1947,7 @@ Per-Host Resources: mem-estimate=95.88MB mem-reservation=61.75MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=68 row-size=20B cardinality=719.38K
+|     tuple-ids=68 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 85(GETNEXT)
 |
 |--84:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -2520,7 +2520,7 @@ Per-Host Resources: mem-estimate=95.88MB mem-reservation=61.75MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=34 row-size=20B cardinality=1.44M
+|     tuple-ids=34 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 43(GETNEXT)
 |
 42:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -3093,9 +3093,9 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=12.81MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=546.75MB Threads=141
@@ -3878,7 +3878,7 @@ Per-Instance Resources: mem-estimate=82.03MB mem-reservation=42.88MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=68 row-size=20B cardinality=719.38K
+|     tuple-ids=68 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 85(GETNEXT)
 |
 |--84:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -4579,7 +4579,7 @@ Per-Instance Resources: mem-estimate=82.03MB mem-reservation=42.88MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=34 row-size=20B cardinality=1.44M
+|     tuple-ids=34 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 43(GETNEXT)
 |
 42:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -5282,8 +5282,8 @@ Per-Instance Resources: mem-estimate=26.94MB mem-reservation=3.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test
index c4550c7b1..d16d63f1d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test
@@ -115,7 +115,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=144.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=12B cardinality=1.44M
+   tuple-ids=0 row-size=12B cardinality=146.92K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=26.62MB Threads=12
@@ -264,7 +264,7 @@ Per-Host Resources: mem-estimate=148.31MB mem-reservation=7.94MB thread-reservat
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=144.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=12B cardinality=1.44M
+   tuple-ids=0 row-size=12B cardinality=146.92K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=29.56MB Threads=11
@@ -439,6 +439,6 @@ Per-Instance Resources: mem-estimate=48.38MB mem-reservation=4.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=12B cardinality=1.44M
+   tuple-ids=0 row-size=12B cardinality=146.92K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ====
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 7a86ca81b..436370c45 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
@@ -146,7 +146,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=336.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=0 row-size=32B cardinality=1.44M
+|     tuple-ids=0 row-size=32B cardinality=29.86K(filtered from 1.44M)
 |     in pipelines: 00(GETNEXT)
 |
 04:SCAN HDFS [tpcds_parquet.catalog_sales cs2]
@@ -330,7 +330,7 @@ Per-Host Resources: mem-estimate=22.30MB mem-reservation=8.69MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=336.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=0 row-size=32B cardinality=1.44M
+|     tuple-ids=0 row-size=32B cardinality=29.86K(filtered from 1.44M)
 |     in pipelines: 00(GETNEXT)
 |
 17:EXCHANGE [HASH(cs2.cs_order_number)]
@@ -562,7 +562,7 @@ Per-Instance Resources: mem-estimate=15.55MB mem-reservation=1.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
-|     tuple-ids=0 row-size=32B cardinality=1.44M
+|     tuple-ids=0 row-size=32B cardinality=29.86K(filtered from 1.44M)
 |     in pipelines: 00(GETNEXT)
 |
 17:EXCHANGE [HASH(cs2.cs_order_number)]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test
index 556055083..6f24124ee 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test
@@ -149,7 +149,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: sr_customer_sk = ss_customer_sk, sr_item_sk = ss_item_sk, sr_ticket_number = ss_ticket_number
 |  |  |  runtime filters: RF012[bloom] <- ss_customer_sk, RF013[bloom] <- ss_item_sk, RF014[bloom] <- ss_ticket_number, RF015[min_max] <- ss_customer_sk, RF016[min_max] <- ss_item_sk, RF017[min_max] <- ss_ticket_number
 |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  |  tuple-ids=1,0,3 row-size=82B cardinality=105.69K
+|  |  |  tuple-ids=1,0,3 row-size=82B cardinality=39.38K(filtered from 105.69K)
 |  |  |  in pipelines: 01(GETNEXT), 00(OPEN)
 |  |  |
 |  |  |--08:HASH JOIN [INNER JOIN]
@@ -180,9 +180,9 @@ PLAN-ROOT SINK
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=91(filtered from 1824)
 |  |  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  |     tuple-ids=0 row-size=32B cardinality=2.88M
+|  |  |     tuple-ids=0 row-size=32B cardinality=143.71K(filtered from 2.88M)
 |  |  |     in pipelines: 00(GETNEXT)
 |  |  |
 |  |  01:SCAN HDFS [tpcds_parquet.store_returns]
@@ -193,7 +193,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |  |     mem-estimate=120.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=28B cardinality=287.51K
+|  |     tuple-ids=1 row-size=28B cardinality=39.38K(filtered from 287.51K)
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.catalog_sales]
@@ -204,7 +204,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=2 row-size=20B cardinality=1.44M
+|     tuple-ids=2 row-size=20B cardinality=215.64K(filtered from 1.44M)
 |     in pipelines: 02(GETNEXT)
 |
 07:SCAN HDFS [tpcds_parquet.item]
@@ -215,7 +215,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=18.00K
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=7 row-size=148B cardinality=18.00K
+   tuple-ids=7 row-size=148B cardinality=2.03K(filtered from 18.00K)
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=70.52MB Threads=20
@@ -418,7 +418,7 @@ Per-Host Resources: mem-estimate=206.00MB mem-reservation=19.75MB thread-reserva
 |  |
 |  18:EXCHANGE [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)]
 |  |  mem-estimate=2.65MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,3 row-size=54B cardinality=143.71K
+|  |  tuple-ids=0,3 row-size=54B cardinality=143.70K(filtered from 143.71K)
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -428,7 +428,7 @@ Per-Host Resources: mem-estimate=206.00MB mem-reservation=19.75MB thread-reserva
 |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
 |  |  runtime filters: RF018[bloom] <- d1.d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3 row-size=54B cardinality=143.71K
+|  |  tuple-ids=0,3 row-size=54B cardinality=143.70K(filtered from 143.71K)
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--17:EXCHANGE [BROADCAST]
@@ -458,9 +458,9 @@ Per-Host Resources: mem-estimate=206.00MB mem-reservation=19.75MB thread-reserva
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=91(filtered from 1824)
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=0 row-size=32B cardinality=2.88M
+|     tuple-ids=0 row-size=32B cardinality=143.70K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
@@ -471,7 +471,7 @@ Per-Host Resources: mem-estimate=206.00MB mem-reservation=19.75MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=2 row-size=20B cardinality=1.44M
+   tuple-ids=2 row-size=20B cardinality=215.64K(filtered from 1.44M)
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=93.14MB Threads=22
@@ -724,7 +724,7 @@ Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reser
 |  |
 |  18:EXCHANGE [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)]
 |  |  mem-estimate=2.83MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,3 row-size=54B cardinality=143.71K
+|  |  tuple-ids=0,3 row-size=54B cardinality=143.70K(filtered from 143.71K)
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -735,7 +735,7 @@ Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reser
 |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3 row-size=54B cardinality=143.71K
+|  |  tuple-ids=0,3 row-size=54B cardinality=143.70K(filtered from 143.71K)
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -773,9 +773,9 @@ Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=91(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=0 row-size=32B cardinality=2.88M
+|     tuple-ids=0 row-size=32B cardinality=143.70K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
@@ -786,6 +786,6 @@ Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=2 row-size=20B cardinality=1.44M
+   tuple-ids=2 row-size=20B cardinality=215.64K(filtered from 1.44M)
    in pipelines: 02(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test
index a189fb254..d6799cb6f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test
@@ -178,7 +178,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=384.00MB mem-reservation=16.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=40B cardinality=1.44M
+|  |     tuple-ids=0 row-size=40B cardinality=294.63K(filtered from 1.44M)
 |  |     in pipelines: 00(GETNEXT)
 |  |
 |  01:SCAN HDFS [tpcds_parquet.customer_demographics cd1]
@@ -192,7 +192,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: cd1.cd_gender = 'M', cd1.cd_education_status = 'College'
 |     parquet dictionary predicates: cd1.cd_gender = 'M', cd1.cd_education_status = 'College'
 |     mem-estimate=64.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=1 row-size=43B cardinality=194.03K
+|     tuple-ids=1 row-size=43B cardinality=28.95K(filtered from 194.03K)
 |     in pipelines: 01(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.customer_demographics cd2]
@@ -203,7 +203,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=1.92M
    mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=2 row-size=4B cardinality=1.92M
+   tuple-ids=2 row-size=4B cardinality=28.95K(filtered from 1.92M)
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=166.00MB Threads=18
@@ -468,7 +468,7 @@ Per-Host Resources: mem-estimate=399.42MB mem-reservation=29.62MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=384.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=40B cardinality=1.44M
+   tuple-ids=0 row-size=40B cardinality=294.63K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=178.56MB Threads=17
@@ -783,6 +783,6 @@ Per-Instance Resources: mem-estimate=49.79MB mem-reservation=16.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
-   tuple-ids=0 row-size=40B cardinality=1.44M
+   tuple-ids=0 row-size=40B cardinality=294.63K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ====
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 294ef1339..c059e3a14 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
@@ -140,9 +140,9 @@ PLAN-ROOT SINK
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |  |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=24B cardinality=2.88M
+|  |     tuple-ids=1 row-size=24B cardinality=29.00K(filtered from 2.88M)
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  03:SCAN HDFS [tpcds_parquet.customer]
@@ -153,7 +153,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=3 row-size=8B cardinality=100.00K
+|     tuple-ids=3 row-size=8B cardinality=1.72K(filtered from 100.00K)
 |     in pipelines: 03(GETNEXT)
 |
 04:SCAN HDFS [tpcds_parquet.customer_address]
@@ -164,7 +164,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=50.00K
    mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=4 row-size=21B cardinality=50.00K
+   tuple-ids=4 row-size=21B cardinality=1.76K(filtered from 50.00K)
    in pipelines: 04(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=31.08MB Threads=16
@@ -365,9 +365,9 @@ Per-Host Resources: mem-estimate=73.28MB mem-reservation=9.88MB thread-reservati
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=1 row-size=24B cardinality=2.88M
+   tuple-ids=1 row-size=24B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=49.77MB Threads=21
@@ -610,8 +610,8 @@ Per-Instance Resources: mem-estimate=18.80MB mem-reservation=2.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=1 row-size=24B cardinality=2.88M
+   tuple-ids=1 row-size=24B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test
index 8d82b56ff..c5e253688 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test
@@ -113,7 +113,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=144.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=16B cardinality=1.44M
+   tuple-ids=0 row-size=16B cardinality=433.07K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=105.81MB Threads=8
@@ -237,7 +237,7 @@ Per-Host Resources: mem-estimate=187.18MB mem-reservation=48.81MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=144.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=16B cardinality=1.44M
+   tuple-ids=0 row-size=16B cardinality=433.07K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=112.62MB Threads=7
@@ -378,6 +378,6 @@ Per-Instance Resources: mem-estimate=84.37MB mem-reservation=42.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=0 row-size=16B cardinality=1.44M
+   tuple-ids=0 row-size=16B cardinality=433.07K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test
index 4e71a3d6c..cfb741524 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test
@@ -121,7 +121,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=1.18M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=38.20MB Threads=10
@@ -258,7 +258,7 @@ Per-Host Resources: mem-estimate=147.42MB mem-reservation=29.81MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=1.18M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=47.02MB Threads=9
@@ -420,6 +420,6 @@ Per-Instance Resources: mem-estimate=42.61MB mem-reservation=21.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=1.18M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test
index 44f67cb3a..84c4e6a8a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test
@@ -143,9 +143,9 @@ PLAN-ROOT SINK
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=24 row-size=12B cardinality=2.88M
+|  |  |     tuple-ids=24 row-size=12B cardinality=2.35M(filtered from 2.88M)
 |  |  |     in pipelines: 27(GETNEXT)
 |  |  |
 |  |  45:HASH JOIN [INNER JOIN]
@@ -177,7 +177,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=160.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=22 row-size=24B cardinality=719.38K
+|  |     tuple-ids=22 row-size=24B cardinality=42.85K(filtered from 719.38K)
 |  |     in pipelines: 25(GETNEXT)
 |  |
 |  44:NESTED LOOP JOIN [INNER JOIN]
@@ -245,9 +245,9 @@ PLAN-ROOT SINK
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=34 row-size=16B cardinality=2.88M
+|  |     tuple-ids=34 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |  |     in pipelines: 37(GETNEXT)
 |  |
 |  36:AGGREGATE [FINALIZE]
@@ -362,9 +362,9 @@ PLAN-ROOT SINK
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=12B cardinality=2.88M
+|  |     tuple-ids=2 row-size=12B cardinality=2.35M(filtered from 2.88M)
 |  |     in pipelines: 03(GETNEXT)
 |  |
 |  21:HASH JOIN [INNER JOIN]
@@ -396,7 +396,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=0 row-size=24B cardinality=1.44M
+|     tuple-ids=0 row-size=24B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 20:NESTED LOOP JOIN [INNER JOIN]
@@ -464,9 +464,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=12 row-size=16B cardinality=2.88M
+|     tuple-ids=12 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 12:AGGREGATE [FINALIZE]
@@ -669,9 +669,9 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |  |     tuple-ids=24 row-size=12B cardinality=2.88M
+|  |  |     tuple-ids=24 row-size=12B cardinality=2.35M(filtered from 2.88M)
 |  |  |     in pipelines: 27(GETNEXT)
 |  |  |
 |  |  45:HASH JOIN [INNER JOIN, BROADCAST]
@@ -710,7 +710,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=160.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=22 row-size=24B cardinality=719.38K
+|  |     tuple-ids=22 row-size=24B cardinality=42.85K(filtered from 719.38K)
 |  |     in pipelines: 25(GETNEXT)
 |  |
 |  44:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -826,9 +826,9 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=34 row-size=16B cardinality=2.88M
+|  |     tuple-ids=34 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |  |     in pipelines: 37(GETNEXT)
 |  |
 |  71:AGGREGATE [FINALIZE]
@@ -1019,9 +1019,9 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=2 row-size=12B cardinality=2.88M
+|  |     tuple-ids=2 row-size=12B cardinality=2.35M(filtered from 2.88M)
 |  |     in pipelines: 03(GETNEXT)
 |  |
 |  21:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1060,7 +1060,7 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=0 row-size=24B cardinality=1.44M
+|     tuple-ids=0 row-size=24B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 20:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -1176,9 +1176,9 @@ Per-Host Resources: mem-estimate=16.72MB mem-reservation=7.75MB thread-reservati
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=12 row-size=16B cardinality=2.88M
+|     tuple-ids=12 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 52:AGGREGATE [FINALIZE]
@@ -1436,9 +1436,9 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |  |     tuple-ids=24 row-size=12B cardinality=2.88M
+|  |  |     tuple-ids=24 row-size=12B cardinality=2.35M(filtered from 2.88M)
 |  |  |     in pipelines: 27(GETNEXT)
 |  |  |
 |  |  45:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1485,7 +1485,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=22 row-size=24B cardinality=719.38K
+|  |     tuple-ids=22 row-size=24B cardinality=42.85K(filtered from 719.38K)
 |  |     in pipelines: 25(GETNEXT)
 |  |
 |  44:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -1625,9 +1625,9 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=34 row-size=16B cardinality=2.88M
+|  |     tuple-ids=34 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |  |     in pipelines: 37(GETNEXT)
 |  |
 |  71:AGGREGATE [FINALIZE]
@@ -1862,9 +1862,9 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|  |     tuple-ids=2 row-size=12B cardinality=2.88M
+|  |     tuple-ids=2 row-size=12B cardinality=2.35M(filtered from 2.88M)
 |  |     in pipelines: 03(GETNEXT)
 |  |
 |  21:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1911,7 +1911,7 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=0 row-size=24B cardinality=1.44M
+|     tuple-ids=0 row-size=24B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 20:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -2051,9 +2051,9 @@ Per-Instance Resources: mem-estimate=10.79MB mem-reservation=1.94MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=12 row-size=16B cardinality=2.88M
+|     tuple-ids=12 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 52:AGGREGATE [FINALIZE]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test
index 3f74ceb9e..dfbe2a753 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test
@@ -270,7 +270,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |  |     mem-estimate=160.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  |     tuple-ids=21 row-size=24B cardinality=719.38K
+|  |  |     tuple-ids=21 row-size=24B cardinality=42.85K(filtered from 719.38K)
 |  |  |     in pipelines: 27(GETNEXT)
 |  |  |
 |  |  28:SCAN HDFS [tpcds_parquet.customer]
@@ -281,7 +281,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=22 row-size=40B cardinality=100.00K
+|  |     tuple-ids=22 row-size=40B cardinality=9.63K(filtered from 100.00K)
 |  |     in pipelines: 28(GETNEXT)
 |  |
 |  35:AGGREGATE [FINALIZE]
@@ -446,9 +446,9 @@ PLAN-ROOT SINK
 |  |  |  |       table: rows=2.88M size=200.96MB
 |  |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |  |       columns: all
-|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |  |  |     tuple-ids=11 row-size=16B cardinality=2.88M
+|  |  |  |     tuple-ids=11 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |  |  |  |     in pipelines: 12(GETNEXT)
 |  |  |  |
 |  |  |  19:HASH JOIN [INNER JOIN]
@@ -486,7 +486,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=0,2 row-size=36B cardinality=85.31K
+|  |  |  tuple-ids=0,2 row-size=36B cardinality=85.31K(filtered from 85.31K)
 |  |  |  in pipelines: 01(GETNEXT), 03(OPEN)
 |  |  |
 |  |  |--03:SCAN HDFS [tpcds_parquet.date_dim]
@@ -510,7 +510,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=24B cardinality=1.44M
+|  |     tuple-ids=0 row-size=24B cardinality=85.31K(filtered from 1.44M)
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.customer]
@@ -521,7 +521,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=1 row-size=40B cardinality=100.00K
+|     tuple-ids=1 row-size=40B cardinality=10.53K(filtered from 100.00K)
 |     in pipelines: 02(GETNEXT)
 |
 09:AGGREGATE [FINALIZE]
@@ -579,9 +579,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=3 row-size=12B cardinality=2.88M
+   tuple-ids=3 row-size=12B cardinality=2.35M(filtered from 2.88M)
    in pipelines: 04(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=293.69MB Threads=56
@@ -830,9 +830,9 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati
 |  |  |  |       table: rows=2.88M size=200.96MB
 |  |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |  |       columns: all
-|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |  |  |     tuple-ids=32 row-size=16B cardinality=2.88M
+|  |  |  |     tuple-ids=32 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |  |  |  |     in pipelines: 38(GETNEXT)
 |  |  |  |
 |  |  |  45:HASH JOIN [INNER JOIN, BROADCAST]
@@ -909,7 +909,7 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=160.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=21 row-size=24B cardinality=719.38K
+|  |     tuple-ids=21 row-size=24B cardinality=42.85K(filtered from 719.38K)
 |  |     in pipelines: 27(GETNEXT)
 |  |
 |  78:AGGREGATE [FINALIZE]
@@ -995,9 +995,9 @@ Per-Host Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservati
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=24 row-size=12B cardinality=2.88M
+|     tuple-ids=24 row-size=12B cardinality=2.35M(filtered from 2.88M)
 |     in pipelines: 30(GETNEXT)
 |
 74:AGGREGATE [FINALIZE]
@@ -1217,9 +1217,9 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|  |  |     tuple-ids=11 row-size=16B cardinality=2.88M
+|  |  |     tuple-ids=11 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |  |  |     in pipelines: 12(GETNEXT)
 |  |  |
 |  |  19:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1265,7 +1265,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat
 |  |  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=0,2 row-size=36B cardinality=85.31K
+|  |  tuple-ids=0,2 row-size=36B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 01(GETNEXT), 03(OPEN)
 |  |
 |  |--58:EXCHANGE [BROADCAST]
@@ -1296,7 +1296,7 @@ Per-Host Resources: mem-estimate=71.94MB mem-reservation=39.88MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=0 row-size=24B cardinality=1.44M
+|     tuple-ids=0 row-size=24B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 57:AGGREGATE [FINALIZE]
@@ -1382,9 +1382,9 @@ Per-Host Resources: mem-estimate=61.76MB mem-reservation=43.19MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=3 row-size=12B cardinality=2.88M
+   tuple-ids=3 row-size=12B cardinality=2.35M(filtered from 2.88M)
    in pipelines: 04(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=530.38MB Threads=67
@@ -1682,9 +1682,9 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser
 |  |  |  |       table: rows=2.88M size=200.96MB
 |  |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |  |       columns: all
-|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |  |  |     tuple-ids=32 row-size=16B cardinality=2.88M
+|  |  |  |     tuple-ids=32 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |  |  |  |     in pipelines: 38(GETNEXT)
 |  |  |  |
 |  |  |  45:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1777,7 +1777,7 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=21 row-size=24B cardinality=719.38K
+|  |     tuple-ids=21 row-size=24B cardinality=42.85K(filtered from 719.38K)
 |  |     in pipelines: 27(GETNEXT)
 |  |
 |  78:AGGREGATE [FINALIZE]
@@ -1881,9 +1881,9 @@ Per-Instance Resources: mem-estimate=10.51MB mem-reservation=1.94MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|     tuple-ids=24 row-size=12B cardinality=2.88M
+|     tuple-ids=24 row-size=12B cardinality=2.35M(filtered from 2.88M)
 |     in pipelines: 30(GETNEXT)
 |
 74:AGGREGATE [FINALIZE]
@@ -2152,9 +2152,9 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
 |  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |  |     tuple-ids=11 row-size=16B cardinality=2.88M
+|  |  |     tuple-ids=11 row-size=16B cardinality=2.35M(filtered from 2.88M)
 |  |  |     in pipelines: 12(GETNEXT)
 |  |  |
 |  |  19:HASH JOIN [INNER JOIN, BROADCAST]
@@ -2208,7 +2208,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese
 |  |  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,2 row-size=36B cardinality=85.31K
+|  |  tuple-ids=0,2 row-size=36B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 01(GETNEXT), 03(OPEN)
 |  |
 |  |--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -2247,7 +2247,7 @@ Per-Instance Resources: mem-estimate=54.32MB mem-reservation=36.00MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=0 row-size=24B cardinality=1.44M
+|     tuple-ids=0 row-size=24B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 57:AGGREGATE [FINALIZE]
@@ -2351,8 +2351,8 @@ Per-Instance Resources: mem-estimate=51.27MB mem-reservation=34.50MB thread-rese
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=1491(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-   tuple-ids=3 row-size=12B cardinality=2.88M
+   tuple-ids=3 row-size=12B cardinality=2.35M(filtered from 2.88M)
    in pipelines: 04(GETNEXT)
 ====
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 a4857b8be..6647fc34f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
@@ -191,7 +191,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=10 row-size=28B cardinality=2.88M
+|     tuple-ids=10 row-size=28B cardinality=960.13K(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 12:AGGREGATE [FINALIZE]
@@ -284,7 +284,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |  |  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  |     tuple-ids=0 row-size=28B cardinality=2.88M
+|  |  |     tuple-ids=0 row-size=28B cardinality=31.25K(filtered from 2.88M)
 |  |  |     in pipelines: 00(GETNEXT)
 |  |  |
 |  |  01:SCAN HDFS [tpcds_parquet.store_returns]
@@ -295,7 +295,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=16B cardinality=287.51K
+|  |     tuple-ids=1 row-size=16B cardinality=16.91K(filtered from 287.51K)
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  04:SCAN HDFS [tpcds_parquet.customer]
@@ -306,7 +306,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=4 row-size=65B cardinality=100.00K
+|     tuple-ids=4 row-size=65B cardinality=7.65K(filtered from 100.00K)
 |     in pipelines: 04(GETNEXT)
 |
 05:SCAN HDFS [tpcds_parquet.customer_address]
@@ -317,7 +317,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=50.00K
    mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=5 row-size=60B cardinality=50.00K
+   tuple-ids=5 row-size=60B cardinality=7.85K(filtered from 50.00K)
    in pipelines: 05(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=193.02MB Threads=32
@@ -536,7 +536,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=10 row-size=28B cardinality=2.88M
+|     tuple-ids=10 row-size=28B cardinality=960.13K(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 39:AGGREGATE [FINALIZE]
@@ -746,7 +746,7 @@ Per-Host Resources: mem-estimate=90.76MB mem-reservation=12.88MB thread-reservat
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=31.25K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=319.08MB Threads=42
@@ -1013,7 +1013,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=10 row-size=28B cardinality=2.88M
+|     tuple-ids=10 row-size=28B cardinality=960.13K(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 39:AGGREGATE [FINALIZE]
@@ -1267,6 +1267,6 @@ Per-Instance Resources: mem-estimate=19.78MB mem-reservation=4.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=31.25K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
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 92095cfb0..81b198b2d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
@@ -191,7 +191,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=10 row-size=28B cardinality=2.88M
+|     tuple-ids=10 row-size=28B cardinality=960.13K(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 12:AGGREGATE [FINALIZE]
@@ -284,7 +284,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |  |  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  |     tuple-ids=0 row-size=28B cardinality=2.88M
+|  |  |     tuple-ids=0 row-size=28B cardinality=31.25K(filtered from 2.88M)
 |  |  |     in pipelines: 00(GETNEXT)
 |  |  |
 |  |  01:SCAN HDFS [tpcds_parquet.store_returns]
@@ -295,7 +295,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=16B cardinality=287.51K
+|  |     tuple-ids=1 row-size=16B cardinality=16.91K(filtered from 287.51K)
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  04:SCAN HDFS [tpcds_parquet.customer]
@@ -306,7 +306,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |     mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=4 row-size=65B cardinality=100.00K
+|     tuple-ids=4 row-size=65B cardinality=7.65K(filtered from 100.00K)
 |     in pipelines: 04(GETNEXT)
 |
 05:SCAN HDFS [tpcds_parquet.customer_address]
@@ -317,7 +317,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=50.00K
    mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=5 row-size=60B cardinality=50.00K
+   tuple-ids=5 row-size=60B cardinality=7.85K(filtered from 50.00K)
    in pipelines: 05(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=193.02MB Threads=32
@@ -536,7 +536,7 @@ Per-Host Resources: mem-estimate=22.02MB mem-reservation=13.94MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=10 row-size=28B cardinality=2.88M
+|     tuple-ids=10 row-size=28B cardinality=960.13K(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 39:AGGREGATE [FINALIZE]
@@ -746,7 +746,7 @@ Per-Host Resources: mem-estimate=90.76MB mem-reservation=12.88MB thread-reservat
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=31.25K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=319.08MB Threads=42
@@ -1013,7 +1013,7 @@ Per-Instance Resources: mem-estimate=22.00MB mem-reservation=13.94MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=10 row-size=28B cardinality=2.88M
+|     tuple-ids=10 row-size=28B cardinality=960.13K(filtered from 2.88M)
 |     in pipelines: 13(GETNEXT)
 |
 39:AGGREGATE [FINALIZE]
@@ -1267,6 +1267,6 @@ Per-Instance Resources: mem-estimate=19.78MB mem-reservation=4.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=31.25K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test
index 7e439f2c0..b04feb248 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test
@@ -143,7 +143,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: sr_customer_sk = ss_customer_sk, sr_item_sk = ss_item_sk, sr_ticket_number = ss_ticket_number
 |  |  |  runtime filters: RF012[bloom] <- ss_customer_sk, RF013[bloom] <- ss_item_sk, RF014[bloom] <- ss_ticket_number, RF015[min_max] <- ss_customer_sk, RF016[min_max] <- ss_item_sk, RF017[min_max] <- ss_ticket_number
 |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  |  tuple-ids=1,0,3 row-size=72B cardinality=125.43K
+|  |  |  tuple-ids=1,0,3 row-size=72B cardinality=17.02K(filtered from 125.43K)
 |  |  |  in pipelines: 01(GETNEXT), 00(OPEN)
 |  |  |
 |  |  |--08:HASH JOIN [INNER JOIN]
@@ -174,9 +174,9 @@ PLAN-ROOT SINK
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |  |  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  |     tuple-ids=0 row-size=32B cardinality=2.88M
+|  |  |     tuple-ids=0 row-size=32B cardinality=170.55K(filtered from 2.88M)
 |  |  |     in pipelines: 00(GETNEXT)
 |  |  |
 |  |  01:SCAN HDFS [tpcds_parquet.store_returns]
@@ -187,7 +187,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |  |     mem-estimate=120.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=28B cardinality=287.51K
+|  |     tuple-ids=1 row-size=28B cardinality=17.02K(filtered from 287.51K)
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.catalog_sales]
@@ -198,7 +198,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=2 row-size=20B cardinality=1.44M
+|     tuple-ids=2 row-size=20B cardinality=93.21K(filtered from 1.44M)
 |     in pipelines: 02(GETNEXT)
 |
 07:SCAN HDFS [tpcds_parquet.item]
@@ -209,7 +209,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=18.00K
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=7 row-size=148B cardinality=18.00K
+   tuple-ids=7 row-size=148B cardinality=450(filtered from 18.00K)
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=69.59MB Threads=20
@@ -412,7 +412,7 @@ Per-Host Resources: mem-estimate=205.07MB mem-reservation=18.81MB thread-reserva
 |  |
 |  18:EXCHANGE [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)]
 |  |  mem-estimate=2.54MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K
+|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -422,7 +422,7 @@ Per-Host Resources: mem-estimate=205.07MB mem-reservation=18.81MB thread-reserva
 |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
 |  |  runtime filters: RF018[bloom] <- d1.d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K
+|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--17:EXCHANGE [BROADCAST]
@@ -452,9 +452,9 @@ Per-Host Resources: mem-estimate=205.07MB mem-reservation=18.81MB thread-reserva
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=0 row-size=32B cardinality=2.88M
+|     tuple-ids=0 row-size=32B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
@@ -465,7 +465,7 @@ Per-Host Resources: mem-estimate=205.07MB mem-reservation=18.81MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=2 row-size=20B cardinality=1.44M
+   tuple-ids=2 row-size=20B cardinality=93.21K(filtered from 1.44M)
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=91.27MB Threads=22
@@ -718,7 +718,7 @@ Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reser
 |  |
 |  18:EXCHANGE [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)]
 |  |  mem-estimate=2.69MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K
+|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -729,7 +729,7 @@ Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reser
 |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K
+|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -767,9 +767,9 @@ Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=0 row-size=32B cardinality=2.88M
+|     tuple-ids=0 row-size=32B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
@@ -780,6 +780,6 @@ Per-Instance Resources: mem-estimate=50.25MB mem-reservation=8.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=2 row-size=20B cardinality=1.44M
+   tuple-ids=2 row-size=20B cardinality=93.21K(filtered from 1.44M)
    in pipelines: 02(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test
index 2b5f184ca..462e84ae2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test
@@ -131,7 +131,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=384.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=36B cardinality=1.44M
+   tuple-ids=0 row-size=36B cardinality=294.63K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=50.53MB Threads=13
@@ -300,7 +300,7 @@ Per-Host Resources: mem-estimate=390.55MB mem-reservation=21.94MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=384.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=36B cardinality=1.44M
+   tuple-ids=0 row-size=36B cardinality=294.63K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=57.34MB Threads=12
@@ -502,6 +502,6 @@ Per-Instance Resources: mem-estimate=48.61MB mem-reservation=16.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
-   tuple-ids=0 row-size=36B cardinality=1.44M
+   tuple-ids=0 row-size=36B cardinality=294.63K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test
index 5a75e5dc9..aa3058aca 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test
@@ -143,9 +143,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=51.08MB Threads=12
@@ -329,9 +329,9 @@ Per-Host Resources: mem-estimate=165.85MB mem-reservation=30.31MB thread-reserva
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=89.14MB Threads=13
@@ -548,8 +548,8 @@ Per-Instance Resources: mem-estimate=51.02MB mem-reservation=12.00MB thread-rese
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test
index cc1ef30f1..ed025fbd2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test
@@ -143,7 +143,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: sr_customer_sk = ss_customer_sk, sr_item_sk = ss_item_sk, sr_ticket_number = ss_ticket_number
 |  |  |  runtime filters: RF012[bloom] <- ss_customer_sk, RF013[bloom] <- ss_item_sk, RF014[bloom] <- ss_ticket_number, RF015[min_max] <- ss_customer_sk, RF016[min_max] <- ss_item_sk, RF017[min_max] <- ss_ticket_number
 |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
-|  |  |  tuple-ids=1,0,3 row-size=72B cardinality=125.43K
+|  |  |  tuple-ids=1,0,3 row-size=72B cardinality=17.02K(filtered from 125.43K)
 |  |  |  in pipelines: 01(GETNEXT), 00(OPEN)
 |  |  |
 |  |  |--08:HASH JOIN [INNER JOIN]
@@ -174,9 +174,9 @@ PLAN-ROOT SINK
 |  |  |       table: rows=2.88M size=200.96MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
-|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |  |  |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  |     tuple-ids=0 row-size=32B cardinality=2.88M
+|  |  |     tuple-ids=0 row-size=32B cardinality=170.55K(filtered from 2.88M)
 |  |  |     in pipelines: 00(GETNEXT)
 |  |  |
 |  |  01:SCAN HDFS [tpcds_parquet.store_returns]
@@ -187,7 +187,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |  |     mem-estimate=120.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=1 row-size=28B cardinality=287.51K
+|  |     tuple-ids=1 row-size=28B cardinality=17.02K(filtered from 287.51K)
 |  |     in pipelines: 01(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.catalog_sales]
@@ -198,7 +198,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=2 row-size=20B cardinality=1.44M
+|     tuple-ids=2 row-size=20B cardinality=132.10K(filtered from 1.44M)
 |     in pipelines: 02(GETNEXT)
 |
 07:SCAN HDFS [tpcds_parquet.item]
@@ -209,7 +209,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=18.00K
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=7 row-size=148B cardinality=18.00K
+   tuple-ids=7 row-size=148B cardinality=4.26K(filtered from 18.00K)
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=69.59MB Threads=20
@@ -412,7 +412,7 @@ Per-Host Resources: mem-estimate=205.02MB mem-reservation=18.81MB thread-reserva
 |  |
 |  18:EXCHANGE [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)]
 |  |  mem-estimate=2.54MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K
+|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -422,7 +422,7 @@ Per-Host Resources: mem-estimate=205.02MB mem-reservation=18.81MB thread-reserva
 |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
 |  |  runtime filters: RF018[bloom] <- d1.d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K
+|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--17:EXCHANGE [BROADCAST]
@@ -452,9 +452,9 @@ Per-Host Resources: mem-estimate=205.02MB mem-reservation=18.81MB thread-reserva
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=0 row-size=32B cardinality=2.88M
+|     tuple-ids=0 row-size=32B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
@@ -465,7 +465,7 @@ Per-Host Resources: mem-estimate=205.02MB mem-reservation=18.81MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=2 row-size=20B cardinality=1.44M
+   tuple-ids=2 row-size=20B cardinality=132.10K(filtered from 1.44M)
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=91.27MB Threads=22
@@ -718,7 +718,7 @@ Per-Instance Resources: mem-estimate=50.21MB mem-reservation=8.00MB thread-reser
 |  |
 |  18:EXCHANGE [HASH(ss_customer_sk,ss_item_sk,ss_ticket_number)]
 |  |  mem-estimate=2.69MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K
+|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 00(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
@@ -729,7 +729,7 @@ Per-Instance Resources: mem-estimate=50.21MB mem-reservation=8.00MB thread-reser
 |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K
+|  |  tuple-ids=0,3 row-size=44B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -767,9 +767,9 @@ Per-Instance Resources: mem-estimate=50.21MB mem-reservation=8.00MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=0 row-size=32B cardinality=2.88M
+|     tuple-ids=0 row-size=32B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 02:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
@@ -780,6 +780,6 @@ Per-Instance Resources: mem-estimate=50.21MB mem-reservation=8.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=2 row-size=20B cardinality=1.44M
+   tuple-ids=2 row-size=20B cardinality=132.10K(filtered from 1.44M)
    in pipelines: 02(GETNEXT)
 ====
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 1397e384f..89b1c2456 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
@@ -122,7 +122,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=71.76K
 |  |     mem-estimate=64.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=7 row-size=16B cardinality=71.76K
+|  |     tuple-ids=7 row-size=16B cardinality=12.83K(filtered from 71.76K)
 |  |     in pipelines: 08(GETNEXT)
 |  |
 |  10:SCAN HDFS [tpcds_parquet.customer_address]
@@ -133,7 +133,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=9 row-size=18B cardinality=50.00K
+|     tuple-ids=9 row-size=18B cardinality=13.17K(filtered from 50.00K)
 |     in pipelines: 10(GETNEXT)
 |
 16:HASH JOIN [INNER JOIN]
@@ -188,7 +188,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=71.76K
 |  |     mem-estimate=64.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=16B cardinality=71.76K
+|  |     tuple-ids=0 row-size=16B cardinality=12.83K(filtered from 71.76K)
 |  |     in pipelines: 00(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.customer_address]
@@ -199,7 +199,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=18B cardinality=50.00K
+|     tuple-ids=2 row-size=18B cardinality=13.17K(filtered from 50.00K)
 |     in pipelines: 02(GETNEXT)
 |
 15:HASH JOIN [INNER JOIN]
@@ -231,7 +231,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=100.00K
    mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=6 row-size=203B cardinality=100.00K
+   tuple-ids=6 row-size=203B cardinality=2.26K(filtered from 100.00K)
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=49.12MB Threads=20
@@ -365,7 +365,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=71.76K
 |  |     mem-estimate=64.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=7 row-size=16B cardinality=71.76K
+|  |     tuple-ids=7 row-size=16B cardinality=12.83K(filtered from 71.76K)
 |  |     in pipelines: 08(GETNEXT)
 |  |
 |  10:SCAN HDFS [tpcds_parquet.customer_address, RANDOM]
@@ -376,7 +376,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=9 row-size=18B cardinality=50.00K
+|     tuple-ids=9 row-size=18B cardinality=13.17K(filtered from 50.00K)
 |     in pipelines: 10(GETNEXT)
 |
 16:HASH JOIN [INNER JOIN, BROADCAST]
@@ -466,7 +466,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=71.76K
 |  |     mem-estimate=64.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=16B cardinality=71.76K
+|  |     tuple-ids=0 row-size=16B cardinality=12.83K(filtered from 71.76K)
 |  |     in pipelines: 00(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM]
@@ -477,7 +477,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=32.00MB mem-reservation=256.00KB thread-reservation=1
-|     tuple-ids=2 row-size=18B cardinality=50.00K
+|     tuple-ids=2 row-size=18B cardinality=13.17K(filtered from 50.00K)
 |     in pipelines: 02(GETNEXT)
 |
 15:HASH JOIN [INNER JOIN, BROADCAST]
@@ -516,7 +516,7 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=100.00K
    mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=6 row-size=203B cardinality=100.00K
+   tuple-ids=6 row-size=203B cardinality=2.26K(filtered from 100.00K)
    in pipelines: 07(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=68.69MB Threads=19
@@ -677,7 +677,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=71.76K
 |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=7 row-size=16B cardinality=71.76K
+|  |     tuple-ids=7 row-size=16B cardinality=12.83K(filtered from 71.76K)
 |  |     in pipelines: 08(GETNEXT)
 |  |
 |  10:SCAN HDFS [tpcds_parquet.customer_address, RANDOM]
@@ -688,7 +688,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=9 row-size=18B cardinality=50.00K
+|     tuple-ids=9 row-size=18B cardinality=13.17K(filtered from 50.00K)
 |     in pipelines: 10(GETNEXT)
 |
 16:HASH JOIN [INNER JOIN, BROADCAST]
@@ -804,7 +804,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=71.76K
 |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=0 row-size=16B cardinality=71.76K
+|  |     tuple-ids=0 row-size=16B cardinality=12.83K(filtered from 71.76K)
 |  |     in pipelines: 00(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.customer_address, RANDOM]
@@ -815,7 +815,7 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
-|     tuple-ids=2 row-size=18B cardinality=50.00K
+|     tuple-ids=2 row-size=18B cardinality=13.17K(filtered from 50.00K)
 |     in pipelines: 02(GETNEXT)
 |
 15:HASH JOIN [INNER JOIN, BROADCAST]
@@ -862,6 +862,6 @@ Per-Instance Resources: mem-estimate=16.02MB mem-reservation=8.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=100.00K
    mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=6 row-size=203B cardinality=100.00K
+   tuple-ids=6 row-size=203B cardinality=2.26K(filtered from 100.00K)
    in pipelines: 07(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test
index a62690431..a629788a5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test
@@ -146,9 +146,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=10 row-size=12B cardinality=2.88M
+|     tuple-ids=10 row-size=12B cardinality=293.73K(filtered from 2.88M)
 |     in pipelines: 12(GETNEXT)
 |
 39:HASH JOIN [INNER JOIN]
@@ -214,9 +214,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=5 row-size=12B cardinality=2.88M
+|     tuple-ids=5 row-size=12B cardinality=293.73K(filtered from 2.88M)
 |     in pipelines: 06(GETNEXT)
 |
 38:HASH JOIN [INNER JOIN]
@@ -282,7 +282,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=25 row-size=12B cardinality=719.38K
+|     tuple-ids=25 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 30(GETNEXT)
 |
 37:HASH JOIN [INNER JOIN]
@@ -348,7 +348,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=20 row-size=12B cardinality=719.38K
+|     tuple-ids=20 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 24(GETNEXT)
 |
 36:HASH JOIN [INNER JOIN]
@@ -414,7 +414,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=15 row-size=12B cardinality=719.38K
+|     tuple-ids=15 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 18(GETNEXT)
 |
 05:AGGREGATE [FINALIZE]
@@ -471,9 +471,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=0 row-size=12B cardinality=2.88M
+   tuple-ids=0 row-size=12B cardinality=293.73K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=220.50MB Threads=47
@@ -595,9 +595,9 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=10 row-size=12B cardinality=2.88M
+|     tuple-ids=10 row-size=12B cardinality=293.73K(filtered from 2.88M)
 |     in pipelines: 12(GETNEXT)
 |
 39:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -698,9 +698,9 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=5 row-size=12B cardinality=2.88M
+|     tuple-ids=5 row-size=12B cardinality=293.73K(filtered from 2.88M)
 |     in pipelines: 06(GETNEXT)
 |
 38:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -808,7 +808,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=25 row-size=12B cardinality=719.38K
+|     tuple-ids=25 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 30(GETNEXT)
 |
 37:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -916,7 +916,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=20 row-size=12B cardinality=719.38K
+|     tuple-ids=20 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 24(GETNEXT)
 |
 36:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1024,7 +1024,7 @@ Per-Host Resources: mem-estimate=18.77PB mem-reservation=60.25MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=15 row-size=12B cardinality=719.38K
+|     tuple-ids=15 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 18(GETNEXT)
 |
 51:EXCHANGE [HASH(ca_county)]
@@ -1116,9 +1116,9 @@ Per-Host Resources: mem-estimate=52.16MB mem-reservation=18.69MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=0 row-size=12B cardinality=2.88M
+   tuple-ids=0 row-size=12B cardinality=293.73K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=277.38MB Threads=58
@@ -1265,9 +1265,9 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=10 row-size=12B cardinality=2.88M
+|     tuple-ids=10 row-size=12B cardinality=293.73K(filtered from 2.88M)
 |     in pipelines: 12(GETNEXT)
 |
 39:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1393,9 +1393,9 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=5 row-size=12B cardinality=2.88M
+|     tuple-ids=5 row-size=12B cardinality=293.73K(filtered from 2.88M)
 |     in pipelines: 06(GETNEXT)
 |
 38:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1528,7 +1528,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=25 row-size=12B cardinality=719.38K
+|     tuple-ids=25 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 30(GETNEXT)
 |
 37:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1661,7 +1661,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=20 row-size=12B cardinality=719.38K
+|     tuple-ids=20 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 24(GETNEXT)
 |
 36:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1794,7 +1794,7 @@ Per-Instance Resources: mem-estimate=9.38PB mem-reservation=12.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=15 row-size=12B cardinality=719.38K
+|     tuple-ids=15 row-size=12B cardinality=73.80K(filtered from 719.38K)
 |     in pipelines: 18(GETNEXT)
 |
 51:EXCHANGE [HASH(ca_county)]
@@ -1903,8 +1903,8 @@ Per-Instance Resources: mem-estimate=27.26MB mem-reservation=6.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=187(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=0 row-size=12B cardinality=2.88M
+   tuple-ids=0 row-size=12B cardinality=293.73K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test
index 29ac5b143..4a819285b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test
@@ -76,7 +76,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=144.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=0 row-size=16B cardinality=1.44M
+|  |     tuple-ids=0 row-size=16B cardinality=1.52K(filtered from 1.44M)
 |  |     in pipelines: 00(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.date_dim]
@@ -90,7 +90,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27'
 |     parquet dictionary predicates: CAST(d_date AS DATE) <= DATE '2000-04-26', CAST(d_date AS DATE) >= DATE '2000-01-27'
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=2 row-size=26B cardinality=7.30K
+|     tuple-ids=2 row-size=26B cardinality=1.52K(filtered from 7.30K)
 |     in pipelines: 02(GETNEXT)
 |
 06:AGGREGATE [FINALIZE]
@@ -246,7 +246,7 @@ Per-Host Resources: mem-estimate=13.31MB mem-reservation=4.88MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=144.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=0 row-size=16B cardinality=1.44M
+|     tuple-ids=0 row-size=16B cardinality=1.52K(filtered from 1.44M)
 |     in pipelines: 00(GETNEXT)
 |
 13:AGGREGATE [FINALIZE]
@@ -448,7 +448,7 @@ Per-Instance Resources: mem-estimate=10.15MB mem-reservation=1.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=0 row-size=16B cardinality=1.44M
+|     tuple-ids=0 row-size=16B cardinality=1.52K(filtered from 1.44M)
 |     in pipelines: 00(GETNEXT)
 |
 13:AGGREGATE [FINALIZE]
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 18b940849..93ca2f778 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
@@ -194,7 +194,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=16 row-size=20B cardinality=719.38K
+|     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 21(GETNEXT)
 |
 |--20:AGGREGATE [FINALIZE]
@@ -269,7 +269,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
 |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K
+|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |
 |  |--12:SCAN HDFS [tpcds_parquet.date_dim]
@@ -293,7 +293,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=8 row-size=20B cardinality=1.44M
+|     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 11(GETNEXT)
 |
 10:AGGREGATE [FINALIZE]
@@ -368,7 +368,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,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.date_dim]
@@ -391,9 +391,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=73.50MB Threads=35
@@ -590,7 +590,7 @@ Per-Host Resources: mem-estimate=20.35MB mem-reservation=3.94MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=16 row-size=20B cardinality=719.38K
+|     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 21(GETNEXT)
 |
 |--45:AGGREGATE [FINALIZE]
@@ -707,7 +707,7 @@ Per-Host Resources: mem-estimate=20.35MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
 |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K
+|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |
 |  |--39:EXCHANGE [BROADCAST]
@@ -738,7 +738,7 @@ Per-Host Resources: mem-estimate=20.35MB mem-reservation=3.94MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=8 row-size=20B cardinality=1.44M
+|     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 11(GETNEXT)
 |
 38:AGGREGATE [FINALIZE]
@@ -848,7 +848,7 @@ Per-Host Resources: mem-estimate=70.12MB mem-reservation=15.75MB thread-reservat
 |  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,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--33:EXCHANGE [BROADCAST]
@@ -878,9 +878,9 @@ Per-Host Resources: mem-estimate=70.12MB mem-reservation=15.75MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=109.75MB Threads=35
@@ -1110,7 +1110,7 @@ Per-Instance Resources: mem-estimate=20.70MB mem-reservation=3.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=16 row-size=20B cardinality=719.38K
+|     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 21(GETNEXT)
 |
 |--45:AGGREGATE [FINALIZE]
@@ -1253,7 +1253,7 @@ Per-Instance Resources: mem-estimate=20.70MB mem-reservation=3.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,9 row-size=32B cardinality=85.31K
+|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |
 |  |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1292,7 +1292,7 @@ Per-Instance Resources: mem-estimate=20.70MB mem-reservation=3.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=8 row-size=20B cardinality=1.44M
+|     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 11(GETNEXT)
 |
 38:AGGREGATE [FINALIZE]
@@ -1428,7 +1428,7 @@ Per-Instance Resources: mem-estimate=26.56MB 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,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1466,8 +1466,8 @@ Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test
index b85075e84..c9c1b7186 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test
@@ -120,7 +120,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
 |  |  runtime filters: RF006[bloom] <- household_demographics.hd_demo_sk, RF007[min_max] <- household_demographics.hd_demo_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=0,3 row-size=56B cardinality=162.45K
+|  |  tuple-ids=0,3 row-size=56B cardinality=162.45K(filtered from 162.45K)
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--03:SCAN HDFS [tpcds_parquet.household_demographics]
@@ -143,9 +143,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=354(filtered from 1824)
 |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=0 row-size=24B cardinality=2.88M
+|     tuple-ids=0 row-size=24B cardinality=162.45K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 08:SCAN HDFS [tpcds_parquet.customer]
@@ -156,7 +156,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=100.00K
    mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=6 row-size=68B cardinality=100.00K
+   tuple-ids=6 row-size=68B cardinality=3.15K(filtered from 100.00K)
    in pipelines: 08(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=27.08MB Threads=13
@@ -297,7 +297,7 @@ Per-Host Resources: mem-estimate=84.09MB mem-reservation=13.81MB thread-reservat
 |  fk/pk conjuncts: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
 |  runtime filters: RF006[bloom] <- household_demographics.hd_demo_sk, RF007[min_max] <- household_demographics.hd_demo_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=56B cardinality=162.45K
+|  tuple-ids=0,3 row-size=56B cardinality=162.45K(filtered from 162.45K)
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--11:EXCHANGE [BROADCAST]
@@ -327,9 +327,9 @@ Per-Host Resources: mem-estimate=84.09MB mem-reservation=13.81MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=354(filtered from 1824)
    mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=24B cardinality=2.88M
+   tuple-ids=0 row-size=24B cardinality=162.45K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=43.83MB Threads=16
@@ -495,7 +495,7 @@ Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reser
 |  hash predicates: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
 |  fk/pk conjuncts: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=56B cardinality=162.45K
+|  tuple-ids=0,3 row-size=56B cardinality=162.45K(filtered from 162.45K)
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -533,8 +533,8 @@ Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=354(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=0 row-size=24B cardinality=2.88M
+   tuple-ids=0 row-size=24B cardinality=162.45K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test
index 3f2793c95..fd8357b15 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test
@@ -120,7 +120,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=8 row-size=8B cardinality=1.44M
+|  |     tuple-ids=8 row-size=8B cardinality=93.21K(filtered from 1.44M)
 |  |     in pipelines: 10(GETNEXT)
 |  |
 |  09:HASH JOIN [INNER JOIN]
@@ -152,7 +152,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=6 row-size=8B cardinality=719.38K
+|     tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
 |     in pipelines: 07(GETNEXT)
 |
 15:HASH JOIN [LEFT SEMI JOIN]
@@ -190,9 +190,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=3 row-size=8B cardinality=2.88M
+|     tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
 |     in pipelines: 03(GETNEXT)
 |
 14:HASH JOIN [INNER JOIN]
@@ -240,7 +240,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=1.92M
    mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=2 row-size=42B cardinality=1.92M
+   tuple-ids=2 row-size=42B cardinality=100.00K(filtered from 1.92M)
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=65.75MB Threads=19
@@ -342,7 +342,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=8 row-size=8B cardinality=1.44M
+|  |     tuple-ids=8 row-size=8B cardinality=93.21K(filtered from 1.44M)
 |  |     in pipelines: 10(GETNEXT)
 |  |
 |  09:HASH JOIN [INNER JOIN, BROADCAST]
@@ -381,7 +381,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=6 row-size=8B cardinality=719.38K
+|     tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
 |     in pipelines: 07(GETNEXT)
 |
 15:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
@@ -457,7 +457,7 @@ Per-Host Resources: mem-estimate=28.04MB mem-reservation=14.62MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=2 row-size=42B cardinality=1.92M
+|     tuple-ids=2 row-size=42B cardinality=100.00K(filtered from 1.92M)
 |     in pipelines: 02(GETNEXT)
 |
 22:EXCHANGE [HASH(ss_customer_sk)]
@@ -502,9 +502,9 @@ Per-Host Resources: mem-estimate=21.27MB mem-reservation=5.44MB thread-reservati
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=3 row-size=8B cardinality=2.88M
+   tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
    in pipelines: 03(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=97.25MB Threads=23
@@ -623,7 +623,7 @@ Per-Instance Resources: mem-estimate=13.97MB mem-reservation=3.00MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-|  |     tuple-ids=8 row-size=8B cardinality=1.44M
+|  |     tuple-ids=8 row-size=8B cardinality=93.21K(filtered from 1.44M)
 |  |     in pipelines: 10(GETNEXT)
 |  |
 |  09:HASH JOIN [INNER JOIN, BROADCAST]
@@ -670,7 +670,7 @@ Per-Instance Resources: mem-estimate=13.97MB mem-reservation=3.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=6 row-size=8B cardinality=719.38K
+|     tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
 |     in pipelines: 07(GETNEXT)
 |
 15:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
@@ -772,7 +772,7 @@ Per-Instance Resources: mem-estimate=13.97MB mem-reservation=3.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=2 row-size=42B cardinality=1.92M
+|     tuple-ids=2 row-size=42B cardinality=100.00K(filtered from 1.92M)
 |     in pipelines: 02(GETNEXT)
 |
 22:EXCHANGE [HASH(ss_customer_sk)]
@@ -826,8 +826,8 @@ Per-Instance Resources: mem-estimate=16.66MB mem-reservation=512.00KB thread-res
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-   tuple-ids=3 row-size=8B cardinality=2.88M
+   tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
    in pipelines: 03(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test
index b28e754b9..393d10981 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test
@@ -145,9 +145,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=24B cardinality=2.88M
+   tuple-ids=0 row-size=24B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=46.27MB Threads=11
@@ -325,9 +325,9 @@ Per-Host Resources: mem-estimate=106.09MB mem-reservation=17.75MB thread-reserva
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=24B cardinality=2.88M
+   tuple-ids=0 row-size=24B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=87.77MB Threads=13
@@ -530,8 +530,8 @@ Per-Instance Resources: mem-estimate=50.68MB mem-reservation=8.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=0 row-size=24B cardinality=2.88M
+   tuple-ids=0 row-size=24B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test
index 7aaa78f19..84d4f8312 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test
@@ -103,7 +103,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: inv_quantity_on_hand <= CAST(500 AS INT), inv_quantity_on_hand >= CAST(100 AS INT)
 |     parquet dictionary predicates: inv_quantity_on_hand <= CAST(500 AS INT), inv_quantity_on_hand >= CAST(100 AS INT)
 |     mem-estimate=96.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=1 row-size=16B cardinality=1.17M
+|     tuple-ids=1 row-size=16B cardinality=15.68K(filtered from 1.17M)
 |     in pipelines: 01(GETNEXT)
 |
 03:SCAN HDFS [tpcds_parquet.catalog_sales]
@@ -114,7 +114,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=3 row-size=8B cardinality=1.44M
+   tuple-ids=3 row-size=8B cardinality=125.59K(filtered from 1.44M)
    in pipelines: 03(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=52.31MB Threads=11
@@ -248,7 +248,7 @@ Per-Host Resources: mem-estimate=62.68MB mem-reservation=13.94MB thread-reservat
 |     parquet statistics predicates: inv_quantity_on_hand <= CAST(500 AS INT), inv_quantity_on_hand >= CAST(100 AS INT)
 |     parquet dictionary predicates: inv_quantity_on_hand <= CAST(500 AS INT), inv_quantity_on_hand >= CAST(100 AS INT)
 |     mem-estimate=96.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=1 row-size=16B cardinality=1.17M
+|     tuple-ids=1 row-size=16B cardinality=15.68K(filtered from 1.17M)
 |     in pipelines: 01(GETNEXT)
 |
 03:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
@@ -259,7 +259,7 @@ Per-Host Resources: mem-estimate=62.68MB mem-reservation=13.94MB thread-reservat
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=3 row-size=8B cardinality=1.44M
+   tuple-ids=3 row-size=8B cardinality=125.59K(filtered from 1.44M)
    in pipelines: 03(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=58.19MB Threads=10
@@ -419,7 +419,7 @@ Per-Instance Resources: mem-estimate=59.74MB mem-reservation=11.00MB thread-rese
 |     parquet statistics predicates: inv_quantity_on_hand <= CAST(500 AS INT), inv_quantity_on_hand >= CAST(100 AS INT)
 |     parquet dictionary predicates: inv_quantity_on_hand <= CAST(500 AS INT), inv_quantity_on_hand >= CAST(100 AS INT)
 |     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-|     tuple-ids=1 row-size=16B cardinality=1.17M
+|     tuple-ids=1 row-size=16B cardinality=15.68K(filtered from 1.17M)
 |     in pipelines: 01(GETNEXT)
 |
 03:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
@@ -430,6 +430,6 @@ Per-Instance Resources: mem-estimate=59.74MB mem-reservation=11.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=3 row-size=8B cardinality=1.44M
+   tuple-ids=3 row-size=8B cardinality=125.59K(filtered from 1.44M)
    in pipelines: 03(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test
index 8658fb34f..810e75584 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test
@@ -153,7 +153,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=6.66M
 |     mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=8 row-size=20B cardinality=11.74M
+|     tuple-ids=8 row-size=20B cardinality=4.82M(filtered from 11.74M)
 |     in pipelines: 08(GETNEXT)
 |
 07:AGGREGATE [FINALIZE]
@@ -231,7 +231,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=4.82M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=237.16MB Threads=20
@@ -379,7 +379,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=6.66M
 |     mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=8 row-size=20B cardinality=11.74M
+|     tuple-ids=8 row-size=20B cardinality=4.82M(filtered from 11.74M)
 |     in pipelines: 08(GETNEXT)
 |
 28:EXCHANGE [HASH(i_item_sk,w_warehouse_sk)]
@@ -499,7 +499,7 @@ Per-Host Resources: mem-estimate=266.10MB mem-reservation=60.81MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=4.82M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=254.78MB Threads=19
@@ -680,7 +680,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=6.66M
 |     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-|     tuple-ids=8 row-size=20B cardinality=11.74M
+|     tuple-ids=8 row-size=20B cardinality=4.82M(filtered from 11.74M)
 |     in pipelines: 08(GETNEXT)
 |
 28:EXCHANGE [HASH(i_item_sk,w_warehouse_sk)]
@@ -827,6 +827,6 @@ Per-Instance Resources: mem-estimate=159.29MB mem-reservation=50.00MB thread-res
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=4.82M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test
index 85ed07e3c..b14a39121 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test
@@ -154,7 +154,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=6.66M
 |     mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=8 row-size=20B cardinality=11.74M
+|     tuple-ids=8 row-size=20B cardinality=4.82M(filtered from 11.74M)
 |     in pipelines: 08(GETNEXT)
 |
 07:AGGREGATE [FINALIZE]
@@ -232,7 +232,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=4.82M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=237.16MB Threads=20
@@ -380,7 +380,7 @@ Per-Host Resources: mem-estimate=59.29MB mem-reservation=42.00MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=6.66M
 |     mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=8 row-size=20B cardinality=11.74M
+|     tuple-ids=8 row-size=20B cardinality=4.82M(filtered from 11.74M)
 |     in pipelines: 08(GETNEXT)
 |
 28:EXCHANGE [HASH(i_item_sk,w_warehouse_sk)]
@@ -500,7 +500,7 @@ Per-Host Resources: mem-estimate=266.10MB mem-reservation=60.81MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=128.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=4.82M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=254.78MB Threads=19
@@ -681,7 +681,7 @@ Per-Instance Resources: mem-estimate=23.29MB mem-reservation=6.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=6.66M
 |     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-|     tuple-ids=8 row-size=20B cardinality=11.74M
+|     tuple-ids=8 row-size=20B cardinality=4.82M(filtered from 11.74M)
 |     in pipelines: 08(GETNEXT)
 |
 28:EXCHANGE [HASH(i_item_sk,w_warehouse_sk)]
@@ -828,6 +828,6 @@ Per-Instance Resources: mem-estimate=159.29MB mem-reservation=50.00MB thread-res
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=6.66M
    mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=11.74M
+   tuple-ids=0 row-size=20B cardinality=4.82M(filtered from 11.74M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test
index 72ccbd481..80363dde2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test
@@ -110,7 +110,7 @@ PLAN-ROOT SINK
 |  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=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,1N row-size=48B cardinality=1.44M
+|  tuple-ids=0,1N row-size=48B cardinality=144.35K(filtered from 1.44M)
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds_parquet.catalog_returns]
@@ -132,7 +132,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=28B cardinality=1.44M
+   tuple-ids=0 row-size=28B cardinality=144.35K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=37.77MB Threads=12
@@ -264,7 +264,7 @@ Per-Host Resources: mem-estimate=270.42MB mem-reservation=27.31MB thread-reserva
 |  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=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
-|  tuple-ids=0,1N row-size=48B cardinality=1.44M
+|  tuple-ids=0,1N row-size=48B cardinality=144.35K(filtered from 1.44M)
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--11:EXCHANGE [BROADCAST]
@@ -293,7 +293,7 @@ Per-Host Resources: mem-estimate=270.42MB mem-reservation=27.31MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=28B cardinality=1.44M
+   tuple-ids=0 row-size=28B cardinality=144.35K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=55.08MB Threads=11
@@ -451,7 +451,7 @@ Per-Instance Resources: mem-estimate=58.91MB mem-reservation=10.00MB thread-rese
 |  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=512.00KB thread-reservation=0
-|  tuple-ids=0,1N row-size=48B cardinality=1.44M
+|  tuple-ids=0,1N row-size=48B cardinality=144.35K(filtered from 1.44M)
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -488,6 +488,6 @@ Per-Instance Resources: mem-estimate=58.91MB mem-reservation=10.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=0 row-size=28B cardinality=1.44M
+   tuple-ids=0 row-size=28B cardinality=144.35K(filtered from 1.44M)
    in pipelines: 00(GETNEXT)
 ====
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 69247f590..6d054ec94 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
@@ -95,9 +95,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=15.56MB Threads=8
@@ -207,9 +207,9 @@ Per-Host Resources: mem-estimate=48.41MB mem-reservation=8.88MB thread-reservati
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=26.38MB Threads=9
@@ -336,8 +336,8 @@ Per-Instance Resources: mem-estimate=27.08MB mem-reservation=3.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test
index c448add93..1d25fad87 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test
@@ -103,9 +103,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=1 row-size=12B cardinality=2.88M
+   tuple-ids=1 row-size=12B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=14.84MB Threads=8
@@ -215,9 +215,9 @@ Per-Host Resources: mem-estimate=49.75MB mem-reservation=8.38MB thread-reservati
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=1 row-size=12B cardinality=2.88M
+   tuple-ids=1 row-size=12B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=25.16MB Threads=9
@@ -344,8 +344,8 @@ Per-Instance Resources: mem-estimate=29.76MB mem-reservation=2.50MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-   tuple-ids=1 row-size=12B cardinality=2.88M
+   tuple-ids=1 row-size=12B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
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 86f241995..ba84856f7 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
@@ -246,7 +246,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=18.00K
 |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=26 row-size=38B cardinality=18.00K
+|     tuple-ids=26 row-size=38B cardinality=10(filtered from 18.00K)
 |     in pipelines: 18(GETNEXT)
 |
 19:SCAN HDFS [tpcds_parquet.item i2]
@@ -257,7 +257,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=18.00K
    mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=27 row-size=38B cardinality=18.00K
+   tuple-ids=27 row-size=38B cardinality=10(filtered from 18.00K)
    in pipelines: 19(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=36.44MB Threads=20
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test
index 318eeb488..366aa1234 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test
@@ -173,7 +173,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=719.38K
+   tuple-ids=0 row-size=20B cardinality=73.80K(filtered from 719.38K)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=46.12MB Threads=16
@@ -386,7 +386,7 @@ Per-Host Resources: mem-estimate=138.48MB mem-reservation=17.69MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=719.38K
+   tuple-ids=0 row-size=20B cardinality=73.80K(filtered from 719.38K)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=58.69MB Threads=15
@@ -641,6 +641,6 @@ Per-Instance Resources: mem-estimate=32.41MB mem-reservation=8.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=719.38K
+   tuple-ids=0 row-size=20B cardinality=73.80K(filtered from 719.38K)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test
index f36da6a73..054ca9cf4 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test
@@ -195,9 +195,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=598(filtered from 1824)
    mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=702.92K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=57.14MB Threads=17
@@ -417,9 +417,9 @@ Per-Host Resources: mem-estimate=138.85MB mem-reservation=28.56MB thread-reserva
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=598(filtered from 1824)
    mem-estimate=112.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=702.92K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=84.45MB Threads=20
@@ -689,8 +689,8 @@ Per-Instance Resources: mem-estimate=27.71MB mem-reservation=9.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=598(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=36B cardinality=2.88M
+   tuple-ids=0 row-size=36B cardinality=702.92K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test
index e155a2dce..f1dda2389 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test
@@ -154,9 +154,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=17 row-size=20B cardinality=2.88M
+|     tuple-ids=17 row-size=20B cardinality=685.36K(filtered from 2.88M)
 |     in pipelines: 24(GETNEXT)
 |
 33:HASH JOIN [INNER JOIN]
@@ -269,9 +269,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=1 row-size=20B cardinality=2.88M
+|     tuple-ids=1 row-size=20B cardinality=685.36K(filtered from 2.88M)
 |     in pipelines: 01(GETNEXT)
 |
 22:ANALYTIC
@@ -358,9 +358,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=9 row-size=20B cardinality=2.88M
+   tuple-ids=9 row-size=20B cardinality=685.36K(filtered from 2.88M)
    in pipelines: 14(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=330.51MB Threads=29
@@ -519,9 +519,9 @@ Per-Host Resources: mem-estimate=55.11MB mem-reservation=34.00MB thread-reservat
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=17 row-size=20B cardinality=2.88M
+|     tuple-ids=17 row-size=20B cardinality=685.36K(filtered from 2.88M)
 |     in pipelines: 24(GETNEXT)
 |
 52:EXCHANGE [HASH(rank(),s_store_name,i_category,s_company_name,i_brand)]
@@ -683,9 +683,9 @@ Per-Host Resources: mem-estimate=63.43MB mem-reservation=51.94MB thread-reservat
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=1 row-size=20B cardinality=2.88M
+|     tuple-ids=1 row-size=20B cardinality=685.36K(filtered from 2.88M)
 |     in pipelines: 01(GETNEXT)
 |
 22:ANALYTIC
@@ -807,9 +807,9 @@ Per-Host Resources: mem-estimate=93.03MB mem-reservation=44.75MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=9 row-size=20B cardinality=2.88M
+   tuple-ids=9 row-size=20B cardinality=685.36K(filtered from 2.88M)
    in pipelines: 14(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=416.70MB Threads=36
@@ -1001,9 +1001,9 @@ Per-Instance Resources: mem-estimate=11.53MB mem-reservation=0B thread-reservati
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=17 row-size=20B cardinality=2.88M
+|     tuple-ids=17 row-size=20B cardinality=685.36K(filtered from 2.88M)
 |     in pipelines: 24(GETNEXT)
 |
 52:EXCHANGE [HASH(rank(),s_store_name,i_category,s_company_name,i_brand)]
@@ -1198,9 +1198,9 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=1 row-size=20B cardinality=2.88M
+|     tuple-ids=1 row-size=20B cardinality=685.36K(filtered from 2.88M)
 |     in pipelines: 01(GETNEXT)
 |
 22:ANALYTIC
@@ -1347,8 +1347,8 @@ Per-Instance Resources: mem-estimate=35.56MB mem-reservation=18.00MB thread-rese
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=434(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=9 row-size=20B cardinality=2.88M
+   tuple-ids=9 row-size=20B cardinality=685.36K(filtered from 2.88M)
    in pipelines: 14(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 8cd18ca58..f33a68b11 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
@@ -147,7 +147,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=1.92M
    mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=2 row-size=39B cardinality=181.75K
+   tuple-ids=2 row-size=39B cardinality=10.26K(filtered from 181.75K)
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=31.08MB Threads=12
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test
index 71a75c8a6..45b18dc63 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test
@@ -197,7 +197,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: sts.ss_item_sk = sr.sr_item_sk, sts.ss_ticket_number = sr.sr_ticket_number
 |  |  other predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=16,17N row-size=56B cardinality=288.04K
+|  |  tuple-ids=16,17N row-size=56B cardinality=170.55K(filtered from 288.04K)
 |  |  in pipelines: 23(GETNEXT), 24(OPEN)
 |  |
 |  |--24:SCAN HDFS [tpcds_parquet.store_returns sr]
@@ -221,11 +221,11 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     parquet statistics predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
 |     parquet dictionary predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=16 row-size=32B cardinality=288.04K
+|     tuple-ids=16 row-size=32B cardinality=170.55K(filtered from 288.04K)
 |     in pipelines: 23(GETNEXT)
 |
 |--22:SELECT
@@ -297,7 +297,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs.cs_item_sk = cr.cr_item_sk, cs.cs_order_number = cr.cr_order_number
 |  |  other predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N row-size=56B cardinality=144.16K
+|  |  tuple-ids=8,9N row-size=56B cardinality=85.31K(filtered from 144.16K)
 |  |  in pipelines: 12(GETNEXT), 13(OPEN)
 |  |
 |  |--13:SCAN HDFS [tpcds_parquet.catalog_returns cr]
@@ -324,7 +324,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
 |     parquet dictionary predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
 |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=8 row-size=32B cardinality=144.16K
+|     tuple-ids=8 row-size=32B cardinality=85.31K(filtered from 144.16K)
 |     in pipelines: 12(GETNEXT)
 |
 11:SELECT
@@ -396,7 +396,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ws.ws_item_sk = wr.wr_item_sk, ws.ws_order_number = wr.wr_order_number
 |  other predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N row-size=56B cardinality=71.94K
+|  tuple-ids=0,1N row-size=56B cardinality=42.85K(filtered from 71.94K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.web_returns wr]
@@ -423,7 +423,7 @@ PLAN-ROOT SINK
    parquet statistics predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
    parquet dictionary predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
    mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=32B cardinality=71.94K
+   tuple-ids=0 row-size=32B cardinality=42.85K(filtered from 71.94K)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=108.81MB Threads=22
@@ -549,7 +549,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: sts.ss_item_sk = sr.sr_item_sk, sts.ss_ticket_number = sr.sr_ticket_number
 |  |  other predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
 |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=16,17N row-size=56B cardinality=288.04K
+|  |  tuple-ids=16,17N row-size=56B cardinality=170.55K(filtered from 288.04K)
 |  |  in pipelines: 23(GETNEXT), 24(OPEN)
 |  |
 |  |--46:EXCHANGE [BROADCAST]
@@ -580,11 +580,11 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     parquet statistics predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
 |     parquet dictionary predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
 |     mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=16 row-size=32B cardinality=288.04K
+|     tuple-ids=16 row-size=32B cardinality=170.55K(filtered from 288.04K)
 |     in pipelines: 23(GETNEXT)
 |
 |--22:SELECT
@@ -685,7 +685,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs.cs_item_sk = cr.cr_item_sk, cs.cs_order_number = cr.cr_order_number
 |  |  other predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N row-size=56B cardinality=144.16K
+|  |  tuple-ids=8,9N row-size=56B cardinality=85.31K(filtered from 144.16K)
 |  |  in pipelines: 12(GETNEXT), 13(OPEN)
 |  |
 |  |--41:EXCHANGE [BROADCAST]
@@ -719,7 +719,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
 |     parquet dictionary predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
 |     mem-estimate=288.00MB mem-reservation=16.00MB thread-reservation=1
-|     tuple-ids=8 row-size=32B cardinality=144.16K
+|     tuple-ids=8 row-size=32B cardinality=85.31K(filtered from 144.16K)
 |     in pipelines: 12(GETNEXT)
 |
 11:SELECT
@@ -820,7 +820,7 @@ Per-Host Resources: mem-estimate=207.34MB mem-reservation=22.88MB thread-reserva
 |  fk/pk conjuncts: ws.ws_item_sk = wr.wr_item_sk, ws.ws_order_number = wr.wr_order_number
 |  other predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N row-size=56B cardinality=71.94K
+|  tuple-ids=0,1N row-size=56B cardinality=42.85K(filtered from 71.94K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--36:EXCHANGE [BROADCAST]
@@ -854,7 +854,7 @@ Per-Host Resources: mem-estimate=207.34MB mem-reservation=22.88MB thread-reserva
    parquet statistics predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
    parquet dictionary predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
    mem-estimate=192.00MB mem-reservation=16.00MB thread-reservation=1
-   tuple-ids=0 row-size=32B cardinality=71.94K
+   tuple-ids=0 row-size=32B cardinality=42.85K(filtered from 71.94K)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=138.31MB Threads=21
@@ -990,7 +990,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: sts.ss_item_sk = sr.sr_item_sk, sts.ss_ticket_number = sr.sr_ticket_number
 |  |  other predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
-|  |  tuple-ids=16,17N row-size=56B cardinality=288.04K
+|  |  tuple-ids=16,17N row-size=56B cardinality=170.55K(filtered from 288.04K)
 |  |  in pipelines: 23(GETNEXT), 24(OPEN)
 |  |
 |  |--F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1028,11 +1028,11 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     parquet statistics predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
 |     parquet dictionary predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
 |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=16 row-size=32B cardinality=288.04K
+|     tuple-ids=16 row-size=32B cardinality=170.55K(filtered from 288.04K)
 |     in pipelines: 23(GETNEXT)
 |
 |--22:SELECT
@@ -1143,7 +1143,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: cs.cs_item_sk = cr.cr_item_sk, cs.cs_order_number = cr.cr_order_number
 |  |  other predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9N row-size=56B cardinality=144.16K
+|  |  tuple-ids=8,9N row-size=56B cardinality=85.31K(filtered from 144.16K)
 |  |  in pipelines: 12(GETNEXT), 13(OPEN)
 |  |
 |  |--F19:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1184,7 +1184,7 @@ PLAN-ROOT SINK
 |     parquet statistics predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
 |     parquet dictionary predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
 |     mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
-|     tuple-ids=8 row-size=32B cardinality=144.16K
+|     tuple-ids=8 row-size=32B cardinality=85.31K(filtered from 144.16K)
 |     in pipelines: 12(GETNEXT)
 |
 11:SELECT
@@ -1295,7 +1295,7 @@ Per-Instance Resources: mem-estimate=42.47MB mem-reservation=18.00MB thread-rese
 |  fk/pk conjuncts: ws.ws_item_sk = wr.wr_item_sk, ws.ws_order_number = wr.wr_order_number
 |  other predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1N row-size=56B cardinality=71.94K
+|  tuple-ids=0,1N row-size=56B cardinality=42.85K(filtered from 71.94K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
@@ -1336,6 +1336,6 @@ Per-Instance Resources: mem-estimate=42.47MB mem-reservation=18.00MB thread-rese
    parquet statistics predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
    parquet dictionary predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
    mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
-   tuple-ids=0 row-size=32B cardinality=71.94K
+   tuple-ids=0 row-size=32B cardinality=42.85K(filtered from 71.94K)
    in pipelines: 01(GETNEXT)
 ====
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 a50d4cd14..eef2d8dfd 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
@@ -94,9 +94,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=15.56MB Threads=8
@@ -206,9 +206,9 @@ Per-Host Resources: mem-estimate=48.53MB mem-reservation=8.88MB thread-reservati
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=26.38MB Threads=9
@@ -335,8 +335,8 @@ Per-Instance Resources: mem-estimate=27.32MB mem-reservation=3.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test
index 9785a1ef6..88ad9d223 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test
@@ -116,9 +116,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=362(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=20B cardinality=2.88M
+   tuple-ids=1 row-size=20B cardinality=481(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=18.52MB Threads=10
@@ -251,9 +251,9 @@ Per-Host Resources: mem-estimate=67.14MB mem-reservation=11.81MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=362(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=20B cardinality=2.88M
+   tuple-ids=1 row-size=20B cardinality=481(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=32.27MB Threads=11
@@ -411,8 +411,8 @@ Per-Instance Resources: mem-estimate=26.66MB mem-reservation=3.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=362(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=1 row-size=20B cardinality=2.88M
+   tuple-ids=1 row-size=20B cardinality=481(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
index 3ae861e98..477768a3b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
@@ -180,7 +180,7 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: ca_address_sk = c_current_addr_sk
 |  |  runtime filters: RF006[bloom] <- c_current_addr_sk, RF007[min_max] <- c_current_addr_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=10,9,7 row-size=64B cardinality=3.43K
+|  |  tuple-ids=10,9,7 row-size=64B cardinality=329(filtered from 3.43K)
 |  |  in pipelines: 11(GETNEXT), 10(OPEN)
 |  |
 |  |--20:HASH JOIN [INNER JOIN]
@@ -283,7 +283,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|  |  |     tuple-ids=6 row-size=8B cardinality=100.00K
+|  |  |     tuple-ids=6 row-size=8B cardinality=108(filtered from 100.00K)
 |  |  |     in pipelines: 05(GETNEXT)
 |  |  |
 |  |  10:SCAN HDFS [tpcds_parquet.store_sales]
@@ -295,7 +295,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=9 row-size=12B cardinality=2.88M
+|  |     tuple-ids=9 row-size=12B cardinality=3.43K(filtered from 2.88M)
 |  |     in pipelines: 10(GETNEXT)
 |  |
 |  11:SCAN HDFS [tpcds_parquet.customer_address]
@@ -306,7 +306,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=48.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=10 row-size=44B cardinality=50.00K
+|     tuple-ids=10 row-size=44B cardinality=329(filtered from 50.00K)
 |     in pipelines: 11(GETNEXT)
 |
 13:SCAN HDFS [tpcds_parquet.date_dim]
@@ -317,7 +317,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=73.05K
    mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=12 row-size=8B cardinality=73.05K
+   tuple-ids=12 row-size=8B cardinality=329(filtered from 73.05K)
    in pipelines: 13(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=60.39MB Threads=31
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 bba1f32dd..afaa3acee 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
@@ -91,9 +91,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=15.56MB Threads=8
@@ -203,9 +203,9 @@ Per-Host Resources: mem-estimate=48.49MB mem-reservation=8.88MB thread-reservati
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=26.38MB Threads=9
@@ -332,8 +332,8 @@ Per-Instance Resources: mem-estimate=27.22MB mem-reservation=3.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=1 row-size=16B cardinality=2.88M
+   tuple-ids=1 row-size=16B cardinality=29.00K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
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 fc560e67c..89b8f6e55 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
@@ -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.51K
+|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=1.19K(filtered from 9.51K)
 |  |  in pipelines: 24(GETNEXT), 21(OPEN)
 |  |
 |  |--27:HASH JOIN [INNER JOIN]
@@ -185,7 +185,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=16 row-size=20B cardinality=719.38K
+|  |     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |  |     in pipelines: 21(GETNEXT)
 |  |
 |  24:SCAN HDFS [tpcds_parquet.item]
@@ -196,7 +196,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=18.00K
 |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=19 row-size=36B cardinality=18.00K
+|     tuple-ids=19 row-size=36B cardinality=1.19K(filtered from 18.00K)
 |     in pipelines: 24(GETNEXT)
 |
 |--20:AGGREGATE [FINALIZE]
@@ -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=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=14.35K
+|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=1.19K(filtered from 14.35K)
 |  |  in pipelines: 14(GETNEXT), 11(OPEN)
 |  |
 |  |--17:HASH JOIN [INNER JOIN]
@@ -260,7 +260,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
 |  |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=8,9 row-size=32B cardinality=85.31K
+|  |  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |  |
 |  |  |--12:SCAN HDFS [tpcds_parquet.date_dim]
@@ -284,7 +284,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=8 row-size=20B cardinality=1.44M
+|  |     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |  |     in pipelines: 11(GETNEXT)
 |  |
 |  14:SCAN HDFS [tpcds_parquet.item]
@@ -295,7 +295,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=18.00K
 |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=11 row-size=36B cardinality=18.00K
+|     tuple-ids=11 row-size=36B cardinality=1.19K(filtered from 18.00K)
 |     in pipelines: 14(GETNEXT)
 |
 10:AGGREGATE [FINALIZE]
@@ -370,7 +370,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,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.date_dim]
@@ -393,9 +393,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=75.62MB Threads=36
@@ -592,7 +592,7 @@ Per-Host Resources: mem-estimate=20.73MB mem-reservation=3.94MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=16 row-size=20B cardinality=719.38K
+|     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 21(GETNEXT)
 |
 |--46:AGGREGATE [FINALIZE]
@@ -709,7 +709,7 @@ Per-Host Resources: mem-estimate=20.73MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
 |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K
+|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |
 |  |--40:EXCHANGE [BROADCAST]
@@ -740,7 +740,7 @@ Per-Host Resources: mem-estimate=20.73MB mem-reservation=3.94MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=8 row-size=20B cardinality=1.44M
+|     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 11(GETNEXT)
 |
 39:AGGREGATE [FINALIZE]
@@ -857,7 +857,7 @@ Per-Host Resources: mem-estimate=55.48MB mem-reservation=8.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=0,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--33:EXCHANGE [BROADCAST]
@@ -887,9 +887,9 @@ Per-Host Resources: mem-estimate=55.48MB mem-reservation=8.88MB thread-reservati
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=111.88MB Threads=38
@@ -1120,7 +1120,7 @@ Per-Instance Resources: mem-estimate=21.43MB mem-reservation=3.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=16 row-size=20B cardinality=719.38K
+|     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 21(GETNEXT)
 |
 |--46:AGGREGATE [FINALIZE]
@@ -1263,7 +1263,7 @@ Per-Instance Resources: mem-estimate=21.43MB mem-reservation=3.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,9 row-size=32B cardinality=85.31K
+|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |
 |  |--F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1302,7 +1302,7 @@ Per-Instance Resources: mem-estimate=21.43MB mem-reservation=3.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=8 row-size=20B cardinality=1.44M
+|     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 11(GETNEXT)
 |
 39:AGGREGATE [FINALIZE]
@@ -1445,7 +1445,7 @@ Per-Instance Resources: mem-estimate=17.22MB 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,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F27:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1483,8 +1483,8 @@ Per-Instance Resources: mem-estimate=17.22MB mem-reservation=2.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test
index 8b4f82524..d66b8ad3d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test
@@ -171,7 +171,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=17 row-size=20B cardinality=1.44M
+|     tuple-ids=17 row-size=20B cardinality=342.81K(filtered from 1.44M)
 |     in pipelines: 24(GETNEXT)
 |
 33:HASH JOIN [INNER JOIN]
@@ -285,7 +285,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=1 row-size=20B cardinality=1.44M
+|     tuple-ids=1 row-size=20B cardinality=342.81K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 22:ANALYTIC
@@ -373,7 +373,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=192.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=9 row-size=20B cardinality=1.44M
+   tuple-ids=9 row-size=20B cardinality=342.81K(filtered from 1.44M)
    in pipelines: 14(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=220.48MB Threads=29
@@ -533,7 +533,7 @@ Per-Host Resources: mem-estimate=38.01MB mem-reservation=17.00MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=17 row-size=20B cardinality=1.44M
+|     tuple-ids=17 row-size=20B cardinality=342.81K(filtered from 1.44M)
 |     in pipelines: 24(GETNEXT)
 |
 52:EXCHANGE [HASH(rank(),i_category,cc_name,i_brand)]
@@ -696,7 +696,7 @@ Per-Host Resources: mem-estimate=35.20MB mem-reservation=34.94MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=1 row-size=20B cardinality=1.44M
+|     tuple-ids=1 row-size=20B cardinality=342.81K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 22:ANALYTIC
@@ -819,7 +819,7 @@ Per-Host Resources: mem-estimate=219.90MB mem-reservation=30.75MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=192.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=9 row-size=20B cardinality=1.44M
+   tuple-ids=9 row-size=20B cardinality=342.81K(filtered from 1.44M)
    in pipelines: 14(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=251.67MB Threads=28
@@ -1012,7 +1012,7 @@ Per-Instance Resources: mem-estimate=10.70MB mem-reservation=0B thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=17 row-size=20B cardinality=1.44M
+|     tuple-ids=17 row-size=20B cardinality=342.81K(filtered from 1.44M)
 |     in pipelines: 24(GETNEXT)
 |
 52:EXCHANGE [HASH(rank(),i_category,cc_name,i_brand)]
@@ -1208,7 +1208,7 @@ Per-Instance Resources: mem-estimate=33.00MB mem-reservation=33.00MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-|     tuple-ids=1 row-size=20B cardinality=1.44M
+|     tuple-ids=1 row-size=20B cardinality=342.81K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 22:ANALYTIC
@@ -1356,6 +1356,6 @@ Per-Instance Resources: mem-estimate=66.15MB mem-reservation=21.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=9 row-size=20B cardinality=1.44M
+   tuple-ids=9 row-size=20B cardinality=342.81K(filtered from 1.44M)
    in pipelines: 14(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test
index 8e2c6b51d..379c012ba 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test
@@ -142,7 +142,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=12 row-size=26B cardinality=73.05K
+|  |     tuple-ids=12 row-size=26B cardinality=7(filtered from 73.05K)
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  19:HASH JOIN [INNER JOIN]
@@ -251,7 +251,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=21 row-size=26B cardinality=73.05K
+|  |     tuple-ids=21 row-size=26B cardinality=7(filtered from 73.05K)
 |  |     in pipelines: 25(GETNEXT)
 |  |
 |  30:HASH JOIN [INNER JOIN]
@@ -352,7 +352,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=3 row-size=26B cardinality=73.05K
+|     tuple-ids=3 row-size=26B cardinality=7(filtered from 73.05K)
 |     in pipelines: 03(GETNEXT)
 |
 08:HASH JOIN [INNER JOIN]
@@ -522,7 +522,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=12 row-size=26B cardinality=73.05K
+|  |     tuple-ids=12 row-size=26B cardinality=7(filtered from 73.05K)
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  19:HASH JOIN [INNER JOIN, BROADCAST]
@@ -681,7 +681,7 @@ Per-Host Resources: mem-estimate=34.92MB mem-reservation=10.69MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |  |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=21 row-size=26B cardinality=73.05K
+|  |     tuple-ids=21 row-size=26B cardinality=7(filtered from 73.05K)
 |  |     in pipelines: 25(GETNEXT)
 |  |
 |  30:HASH JOIN [INNER JOIN, BROADCAST]
@@ -832,7 +832,7 @@ Per-Host Resources: mem-estimate=56.14MB mem-reservation=14.62MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=3 row-size=26B cardinality=73.05K
+|     tuple-ids=3 row-size=26B cardinality=7(filtered from 73.05K)
 |     in pipelines: 03(GETNEXT)
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1041,7 +1041,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=12 row-size=26B cardinality=73.05K
+|  |     tuple-ids=12 row-size=26B cardinality=7(filtered from 73.05K)
 |  |     in pipelines: 14(GETNEXT)
 |  |
 |  19:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1243,7 +1243,7 @@ Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|  |     tuple-ids=21 row-size=26B cardinality=73.05K
+|  |     tuple-ids=21 row-size=26B cardinality=7(filtered from 73.05K)
 |  |     in pipelines: 25(GETNEXT)
 |  |
 |  30:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1429,7 +1429,7 @@ Per-Instance Resources: mem-estimate=27.12MB mem-reservation=3.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=73.05K
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=3 row-size=26B cardinality=73.05K
+|     tuple-ids=3 row-size=26B cardinality=7(filtered from 73.05K)
 |     in pipelines: 03(GETNEXT)
 |
 08: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 c2e444e15..2dcdbc57a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
@@ -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.51K
+|  |  tuple-ids=19,16,17,18 row-size=76B cardinality=3.66K(filtered from 9.51K)
 |  |  in pipelines: 24(GETNEXT), 21(OPEN)
 |  |
 |  |--27:HASH JOIN [INNER JOIN]
@@ -186,7 +186,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=16 row-size=20B cardinality=719.38K
+|  |     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |  |     in pipelines: 21(GETNEXT)
 |  |
 |  24:SCAN HDFS [tpcds_parquet.item]
@@ -197,7 +197,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=18.00K
 |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=19 row-size=36B cardinality=18.00K
+|     tuple-ids=19 row-size=36B cardinality=3.66K(filtered from 18.00K)
 |     in pipelines: 24(GETNEXT)
 |
 |--20:AGGREGATE [FINALIZE]
@@ -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=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=14.35K
+|  |  tuple-ids=11,8,9,10 row-size=76B cardinality=3.66K(filtered from 14.35K)
 |  |  in pipelines: 14(GETNEXT), 11(OPEN)
 |  |
 |  |--17:HASH JOIN [INNER JOIN]
@@ -261,7 +261,7 @@ PLAN-ROOT SINK
 |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
 |  |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
 |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  |  tuple-ids=8,9 row-size=32B cardinality=85.31K
+|  |  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |  |
 |  |  |--12:SCAN HDFS [tpcds_parquet.date_dim]
@@ -285,7 +285,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=8 row-size=20B cardinality=1.44M
+|  |     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |  |     in pipelines: 11(GETNEXT)
 |  |
 |  14:SCAN HDFS [tpcds_parquet.item]
@@ -296,7 +296,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=18.00K
 |     mem-estimate=32.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=11 row-size=36B cardinality=18.00K
+|     tuple-ids=11 row-size=36B cardinality=3.66K(filtered from 18.00K)
 |     in pipelines: 14(GETNEXT)
 |
 10:AGGREGATE [FINALIZE]
@@ -371,7 +371,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,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds_parquet.date_dim]
@@ -394,9 +394,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=75.62MB Threads=36
@@ -593,7 +593,7 @@ Per-Host Resources: mem-estimate=20.78MB mem-reservation=3.94MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=16 row-size=20B cardinality=719.38K
+|     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 21(GETNEXT)
 |
 |--46:AGGREGATE [FINALIZE]
@@ -710,7 +710,7 @@ Per-Host Resources: mem-estimate=20.78MB mem-reservation=3.94MB thread-reservati
 |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
 |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K
+|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |
 |  |--40:EXCHANGE [BROADCAST]
@@ -741,7 +741,7 @@ Per-Host Resources: mem-estimate=20.78MB mem-reservation=3.94MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=8 row-size=20B cardinality=1.44M
+|     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 11(GETNEXT)
 |
 39:AGGREGATE [FINALIZE]
@@ -858,7 +858,7 @@ Per-Host Resources: mem-estimate=55.48MB mem-reservation=8.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=0,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--33:EXCHANGE [BROADCAST]
@@ -888,9 +888,9 @@ Per-Host Resources: mem-estimate=55.48MB mem-reservation=8.88MB thread-reservati
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=111.88MB Threads=38
@@ -1121,7 +1121,7 @@ Per-Instance Resources: mem-estimate=21.48MB mem-reservation=3.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=16 row-size=20B cardinality=719.38K
+|     tuple-ids=16 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 21(GETNEXT)
 |
 |--46:AGGREGATE [FINALIZE]
@@ -1264,7 +1264,7 @@ Per-Instance Resources: mem-estimate=21.48MB mem-reservation=3.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,9 row-size=32B cardinality=85.31K
+|  |  tuple-ids=8,9 row-size=32B cardinality=85.31K(filtered from 85.31K)
 |  |  in pipelines: 11(GETNEXT), 12(OPEN)
 |  |
 |  |--F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1303,7 +1303,7 @@ Per-Instance Resources: mem-estimate=21.48MB mem-reservation=3.94MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=8 row-size=20B cardinality=1.44M
+|     tuple-ids=8 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 11(GETNEXT)
 |
 39:AGGREGATE [FINALIZE]
@@ -1446,7 +1446,7 @@ Per-Instance Resources: mem-estimate=17.22MB 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,1 row-size=32B cardinality=170.55K
+|  tuple-ids=0,1 row-size=32B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--F27:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1484,8 +1484,8 @@ Per-Instance Resources: mem-estimate=17.22MB mem-reservation=2.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=0 row-size=20B cardinality=2.88M
+   tuple-ids=0 row-size=20B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
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 eb62c82c0..a57427aa3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
@@ -159,7 +159,7 @@ PLAN-ROOT SINK
 |  |  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=9,11 row-size=36B cardinality=170.55K
+|  |  tuple-ids=9,11 row-size=36B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 14(GETNEXT), 16(OPEN)
 |  |
 |  |--16:SCAN HDFS [tpcds_parquet.date_dim]
@@ -182,9 +182,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=9 row-size=24B cardinality=2.88M
+|     tuple-ids=9 row-size=24B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 14(GETNEXT)
 |
 13:AGGREGATE [FINALIZE]
@@ -299,7 +299,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF010[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=40B cardinality=170.55K
+|  tuple-ids=0,3 row-size=40B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--03:SCAN HDFS [tpcds_parquet.date_dim]
@@ -322,9 +322,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=49.25MB Threads=30
@@ -494,7 +494,7 @@ PLAN-ROOT SINK
 |  |  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=9,11 row-size=36B cardinality=170.55K
+|  |  tuple-ids=9,11 row-size=36B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 14(GETNEXT), 16(OPEN)
 |  |
 |  |--37:EXCHANGE [BROADCAST]
@@ -524,9 +524,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=9 row-size=24B cardinality=2.88M
+|     tuple-ids=9 row-size=24B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 14(GETNEXT)
 |
 36:AGGREGATE [FINALIZE]
@@ -696,7 +696,7 @@ Per-Host Resources: mem-estimate=94.35MB mem-reservation=14.75MB thread-reservat
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF010[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=40B cardinality=170.55K
+|  tuple-ids=0,3 row-size=40B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--28:EXCHANGE [BROADCAST]
@@ -726,9 +726,9 @@ Per-Host Resources: mem-estimate=94.35MB mem-reservation=14.75MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=80.69MB Threads=35
@@ -939,7 +939,7 @@ PLAN-ROOT SINK
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11 row-size=36B cardinality=170.55K
+|  |  tuple-ids=9,11 row-size=36B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 14(GETNEXT), 16(OPEN)
 |  |
 |  |--F22:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -977,9 +977,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-|     tuple-ids=9 row-size=24B cardinality=2.88M
+|     tuple-ids=9 row-size=24B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 14(GETNEXT)
 |
 36:AGGREGATE [FINALIZE]
@@ -1191,7 +1191,7 @@ Per-Instance Resources: mem-estimate=19.21MB 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 row-size=40B cardinality=170.55K
+|  tuple-ids=0,3 row-size=40B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1229,8 +1229,8 @@ Per-Instance Resources: mem-estimate=19.21MB mem-reservation=2.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test
index 3a954b33c..961bfa672 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test
@@ -139,9 +139,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=362(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=20B cardinality=2.88M
+   tuple-ids=1 row-size=20B cardinality=481(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=28.52MB Threads=10
@@ -293,9 +293,9 @@ Per-Host Resources: mem-estimate=67.14MB mem-reservation=11.81MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=362(filtered from 1824)
    mem-estimate=48.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=1 row-size=20B cardinality=2.88M
+   tuple-ids=1 row-size=20B cardinality=481(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=52.27MB Threads=11
@@ -472,8 +472,8 @@ Per-Instance Resources: mem-estimate=26.66MB mem-reservation=3.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=362(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=1 row-size=20B cardinality=2.88M
+   tuple-ids=1 row-size=20B cardinality=481(filtered from 2.88M)
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test
index aad0be77f..1a4040ed1 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test
@@ -197,7 +197,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=144.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=25 row-size=20B cardinality=1.44M
+|  |     tuple-ids=25 row-size=20B cardinality=1.30M(filtered from 1.44M)
 |  |     in pipelines: 41(GETNEXT)
 |  |
 |  75:HASH JOIN [INNER JOIN]
@@ -446,7 +446,7 @@ PLAN-ROOT SINK
 |  |  |  |  |  |  |       columns: all
 |  |  |  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |  |  |  |  |  |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  |  |  |  |  |     tuple-ids=23 row-size=56B cardinality=2.88M
+|  |  |  |  |  |  |     tuple-ids=23 row-size=56B cardinality=59.45K(filtered from 2.88M)
 |  |  |  |  |  |  |     in pipelines: 39(GETNEXT)
 |  |  |  |  |  |  |
 |  |  |  |  |  |  40:SCAN HDFS [tpcds_parquet.store_returns]
@@ -457,7 +457,7 @@ PLAN-ROOT SINK
 |  |  |  |  |  |       columns: all
 |  |  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |  |  |  |  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  |  |  |  |     tuple-ids=24 row-size=16B cardinality=287.51K
+|  |  |  |  |  |     tuple-ids=24 row-size=16B cardinality=19.73K(filtered from 287.51K)
 |  |  |  |  |  |     in pipelines: 40(GETNEXT)
 |  |  |  |  |  |
 |  |  |  |  |  49:SCAN HDFS [tpcds_parquet.customer]
@@ -468,7 +468,7 @@ PLAN-ROOT SINK
 |  |  |  |  |       columns: all
 |  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |  |  |  |     mem-estimate=96.00MB mem-reservation=2.00MB thread-reservation=1
-|  |  |  |  |     tuple-ids=33 row-size=24B cardinality=100.00K
+|  |  |  |  |     tuple-ids=33 row-size=24B cardinality=8.66K(filtered from 100.00K)
 |  |  |  |  |     in pipelines: 49(GETNEXT)
 |  |  |  |  |
 |  |  |  |  50:SCAN HDFS [tpcds_parquet.customer_demographics cd1]
@@ -479,7 +479,7 @@ PLAN-ROOT SINK
 |  |  |  |       columns: all
 |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |  |  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  |  |     tuple-ids=34 row-size=17B cardinality=1.92M
+|  |  |  |     tuple-ids=34 row-size=17B cardinality=8.45K(filtered from 1.92M)
 |  |  |  |     in pipelines: 50(GETNEXT)
 |  |  |  |
 |  |  |  51:SCAN HDFS [tpcds_parquet.customer_demographics cd2]
@@ -490,7 +490,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  |     tuple-ids=35 row-size=17B cardinality=1.92M
+|  |  |     tuple-ids=35 row-size=17B cardinality=8.45K(filtered from 1.92M)
 |  |  |     in pipelines: 51(GETNEXT)
 |  |  |
 |  |  55:SCAN HDFS [tpcds_parquet.customer_address ad1]
@@ -501,7 +501,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |  |     mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=1
-|  |     tuple-ids=39 row-size=78B cardinality=50.00K
+|  |     tuple-ids=39 row-size=78B cardinality=8.67K(filtered from 50.00K)
 |  |     in pipelines: 55(GETNEXT)
 |  |
 |  56:SCAN HDFS [tpcds_parquet.customer_address ad2]
@@ -512,7 +512,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=40 row-size=78B cardinality=50.00K
+|     tuple-ids=40 row-size=78B cardinality=8.67K(filtered from 50.00K)
 |     in pipelines: 56(GETNEXT)
 |
 38:AGGREGATE [FINALIZE]
@@ -565,7 +565,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=144.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=2 row-size=20B cardinality=1.44M
+|     tuple-ids=2 row-size=20B cardinality=1.30M(filtered from 1.44M)
 |     in pipelines: 02(GETNEXT)
 |
 36:HASH JOIN [INNER JOIN]
@@ -815,7 +815,7 @@ PLAN-ROOT SINK
 |  |  |  |  |  |       columns: all
 |  |  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |  |  |  |  |  |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  |  |  |  |     tuple-ids=0 row-size=56B cardinality=2.88M
+|  |  |  |  |  |     tuple-ids=0 row-size=56B cardinality=59.45K(filtered from 2.88M)
 |  |  |  |  |  |     in pipelines: 00(GETNEXT)
 |  |  |  |  |  |
 |  |  |  |  |  01:SCAN HDFS [tpcds_parquet.store_returns]
@@ -826,7 +826,7 @@ PLAN-ROOT SINK
 |  |  |  |  |       columns: all
 |  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
 |  |  |  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=1
-|  |  |  |  |     tuple-ids=1 row-size=16B cardinality=287.51K
+|  |  |  |  |     tuple-ids=1 row-size=16B cardinality=19.73K(filtered from 287.51K)
 |  |  |  |  |     in pipelines: 01(GETNEXT)
 |  |  |  |  |
 |  |  |  |  10:SCAN HDFS [tpcds_parquet.customer]
@@ -837,7 +837,7 @@ PLAN-ROOT SINK
 |  |  |  |       columns: all
 |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
 |  |  |  |     mem-estimate=96.00MB mem-reservation=2.00MB thread-reservation=1
-|  |  |  |     tuple-ids=10 row-size=24B cardinality=100.00K
+|  |  |  |     tuple-ids=10 row-size=24B cardinality=8.66K(filtered from 100.00K)
 |  |  |  |     in pipelines: 10(GETNEXT)
 |  |  |  |
 |  |  |  11:SCAN HDFS [tpcds_parquet.customer_demographics cd1]
@@ -848,7 +848,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  |     tuple-ids=11 row-size=17B cardinality=1.92M
+|  |  |     tuple-ids=11 row-size=17B cardinality=8.45K(filtered from 1.92M)
 |  |  |     in pipelines: 11(GETNEXT)
 |  |  |
 |  |  12:SCAN HDFS [tpcds_parquet.customer_demographics cd2]
@@ -859,7 +859,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=12 row-size=17B cardinality=1.92M
+|  |     tuple-ids=12 row-size=17B cardinality=8.45K(filtered from 1.92M)
 |  |     in pipelines: 12(GETNEXT)
 |  |
 |  16:SCAN HDFS [tpcds_parquet.customer_address ad1]
@@ -870,7 +870,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=50.00K
 |     mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=16 row-size=78B cardinality=50.00K
+|     tuple-ids=16 row-size=78B cardinality=8.67K(filtered from 50.00K)
 |     in pipelines: 16(GETNEXT)
 |
 17:SCAN HDFS [tpcds_parquet.customer_address ad2]
@@ -881,7 +881,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=50.00K
    mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=17 row-size=78B cardinality=50.00K
+   tuple-ids=17 row-size=78B cardinality=8.67K(filtered from 50.00K)
    in pipelines: 17(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=521.64MB Threads=98
@@ -1013,7 +1013,7 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=144.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=25 row-size=20B cardinality=1.44M
+|  |     tuple-ids=25 row-size=20B cardinality=1.30M(filtered from 1.44M)
 |  |     in pipelines: 41(GETNEXT)
 |  |
 |  75:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1491,9 +1491,9 @@ Per-Host Resources: mem-estimate=75.11MB mem-reservation=46.00MB thread-reservat
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=23 row-size=56B cardinality=2.88M
+|     tuple-ids=23 row-size=56B cardinality=59.45K(filtered from 2.88M)
 |     in pipelines: 39(GETNEXT)
 |
 140:EXCHANGE [HASH(i_item_sk,s_store_name,s_zip)]
@@ -1595,7 +1595,7 @@ Per-Host Resources: mem-estimate=48.81MB mem-reservation=42.69MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=144.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=2 row-size=20B cardinality=1.44M
+|     tuple-ids=2 row-size=20B cardinality=1.30M(filtered from 1.44M)
 |     in pipelines: 02(GETNEXT)
 |
 36:HASH JOIN [INNER JOIN, BROADCAST]
@@ -2074,9 +2074,9 @@ Per-Host Resources: mem-estimate=137.46MB mem-reservation=15.88MB thread-reserva
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=0 row-size=56B cardinality=2.88M
+   tuple-ids=0 row-size=56B cardinality=59.45K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=627.27MB Threads=135
@@ -2233,7 +2233,7 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=25 row-size=20B cardinality=1.44M
+|  |     tuple-ids=25 row-size=20B cardinality=1.30M(filtered from 1.44M)
 |  |     in pipelines: 41(GETNEXT)
 |  |
 |  75:HASH JOIN [INNER JOIN, BROADCAST]
@@ -2840,9 +2840,9 @@ Per-Instance Resources: mem-estimate=26.83MB mem-reservation=12.00MB thread-rese
 |       table: rows=2.88M size=200.97MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=23 row-size=56B cardinality=2.88M
+|     tuple-ids=23 row-size=56B cardinality=59.45K(filtered from 2.88M)
 |     in pipelines: 39(GETNEXT)
 |
 140:EXCHANGE [HASH(i_item_sk,s_store_name,s_zip)]
@@ -2961,7 +2961,7 @@ Per-Instance Resources: mem-estimate=27.11MB mem-reservation=17.00MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=2 row-size=20B cardinality=1.44M
+|     tuple-ids=2 row-size=20B cardinality=1.30M(filtered from 1.44M)
 |     in pipelines: 02(GETNEXT)
 |
 36:HASH JOIN [INNER JOIN, BROADCAST]
@@ -3570,8 +3570,8 @@ Per-Instance Resources: mem-estimate=19.17MB mem-reservation=8.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=0 row-size=56B cardinality=2.88M
+   tuple-ids=0 row-size=56B cardinality=59.45K(filtered from 2.88M)
    in pipelines: 00(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 aa33a4c02..32a6e2fd9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
@@ -152,7 +152,7 @@ PLAN-ROOT SINK
 |  |  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=9,11 row-size=36B cardinality=170.55K
+|  |  tuple-ids=9,11 row-size=36B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 14(GETNEXT), 16(OPEN)
 |  |
 |  |--16:SCAN HDFS [tpcds_parquet.date_dim]
@@ -175,9 +175,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=9 row-size=24B cardinality=2.88M
+|     tuple-ids=9 row-size=24B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 14(GETNEXT)
 |
 13:AGGREGATE [FINALIZE]
@@ -292,7 +292,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF010[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=40B cardinality=170.55K
+|  tuple-ids=0,3 row-size=40B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--03:SCAN HDFS [tpcds_parquet.date_dim]
@@ -315,9 +315,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=55.25MB Threads=30
@@ -487,7 +487,7 @@ PLAN-ROOT SINK
 |  |  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=9,11 row-size=36B cardinality=170.55K
+|  |  tuple-ids=9,11 row-size=36B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 14(GETNEXT), 16(OPEN)
 |  |
 |  |--37:EXCHANGE [BROADCAST]
@@ -517,9 +517,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=9 row-size=24B cardinality=2.88M
+|     tuple-ids=9 row-size=24B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 14(GETNEXT)
 |
 36:AGGREGATE [FINALIZE]
@@ -689,7 +689,7 @@ Per-Host Resources: mem-estimate=94.35MB mem-reservation=14.75MB thread-reservat
 |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  runtime filters: RF010[bloom] <- d_date_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,3 row-size=40B cardinality=170.55K
+|  tuple-ids=0,3 row-size=40B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--28:EXCHANGE [BROADCAST]
@@ -719,9 +719,9 @@ Per-Host Resources: mem-estimate=94.35MB mem-reservation=14.75MB thread-reservat
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=80.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=86.69MB Threads=35
@@ -932,7 +932,7 @@ PLAN-ROOT SINK
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
 |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
 |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=9,11 row-size=36B cardinality=170.55K
+|  |  tuple-ids=9,11 row-size=36B cardinality=170.55K(filtered from 170.55K)
 |  |  in pipelines: 14(GETNEXT), 16(OPEN)
 |  |
 |  |--F22:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -970,9 +970,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-|     tuple-ids=9 row-size=24B cardinality=2.88M
+|     tuple-ids=9 row-size=24B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 14(GETNEXT)
 |
 36:AGGREGATE [FINALIZE]
@@ -1184,7 +1184,7 @@ Per-Instance Resources: mem-estimate=19.21MB 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 row-size=40B cardinality=170.55K
+|  tuple-ids=0,3 row-size=40B cardinality=170.55K(filtered from 170.55K)
 |  in pipelines: 00(GETNEXT), 03(OPEN)
 |
 |--F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
@@ -1222,8 +1222,8 @@ Per-Instance Resources: mem-estimate=19.21MB mem-reservation=2.00MB thread-reser
      table: rows=2.88M size=200.96MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=0 row-size=28B cardinality=2.88M
+   tuple-ids=0 row-size=28B cardinality=170.55K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test
index 3adaaa878..cbb44751a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test
@@ -195,7 +195,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=128.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=40B cardinality=2.88M
+   tuple-ids=0 row-size=40B cardinality=702.92K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=111.89MB Threads=16
@@ -410,7 +410,7 @@ Per-Host Resources: mem-estimate=179.04MB mem-reservation=53.56MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=128.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=0 row-size=40B cardinality=2.88M
+   tuple-ids=0 row-size=40B cardinality=702.92K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=143.70MB Threads=17
@@ -675,6 +675,6 @@ Per-Instance Resources: mem-estimate=35.08MB mem-reservation=21.00MB thread-rese
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=0 row-size=40B cardinality=2.88M
+   tuple-ids=0 row-size=40B cardinality=702.92K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test
index f38e661d2..75df60c50 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test
@@ -140,7 +140,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |  |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  |     tuple-ids=2 row-size=75B cardinality=1.92M
+|  |  |     tuple-ids=2 row-size=75B cardinality=100.00K(filtered from 1.92M)
 |  |  |     in pipelines: 02(GETNEXT)
 |  |  |
 |  |  05:HASH JOIN [INNER JOIN]
@@ -171,9 +171,9 @@ PLAN-ROOT SINK
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
 |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|  |     tuple-ids=3 row-size=8B cardinality=2.88M
+|  |     tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
 |  |     in pipelines: 03(GETNEXT)
 |  |
 |  08:HASH JOIN [INNER JOIN]
@@ -205,7 +205,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-|     tuple-ids=6 row-size=8B cardinality=719.38K
+|     tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
 |     in pipelines: 06(GETNEXT)
 |
 11:HASH JOIN [INNER JOIN]
@@ -237,7 +237,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=9 row-size=8B cardinality=1.44M
+   tuple-ids=9 row-size=8B cardinality=93.21K(filtered from 1.44M)
    in pipelines: 09(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=58.25MB Threads=21
@@ -339,7 +339,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=64.00MB mem-reservation=4.00MB thread-reservation=1
-|  |     tuple-ids=6 row-size=8B cardinality=719.38K
+|  |     tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
 |  |     in pipelines: 06(GETNEXT)
 |  |
 |  14:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
@@ -418,7 +418,7 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=2 row-size=75B cardinality=1.92M
+|  |     tuple-ids=2 row-size=75B cardinality=100.00K(filtered from 1.92M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  23:EXCHANGE [HASH(ss_customer_sk)]
@@ -463,9 +463,9 @@ Per-Host Resources: mem-estimate=18.75MB mem-reservation=8.81MB thread-reservati
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=1
-|     tuple-ids=3 row-size=8B cardinality=2.88M
+|     tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
 |     in pipelines: 03(GETNEXT)
 |
 27:EXCHANGE [HASH(cs_ship_customer_sk)]
@@ -511,7 +511,7 @@ Per-Host Resources: mem-estimate=99.27MB mem-reservation=6.94MB thread-reservati
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=96.00MB mem-reservation=4.00MB thread-reservation=1
-   tuple-ids=9 row-size=8B cardinality=1.44M
+   tuple-ids=9 row-size=8B cardinality=93.21K(filtered from 1.44M)
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=79.00MB Threads=21
@@ -638,7 +638,7 @@ Per-Instance Resources: mem-estimate=11.64MB mem-reservation=2.00MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
-|  |     tuple-ids=6 row-size=8B cardinality=719.38K
+|  |     tuple-ids=6 row-size=8B cardinality=46.82K(filtered from 719.38K)
 |  |     in pipelines: 06(GETNEXT)
 |  |
 |  14:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
@@ -743,7 +743,7 @@ Per-Instance Resources: mem-estimate=11.64MB mem-reservation=2.00MB thread-reser
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=1.92M
 |  |     mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
-|  |     tuple-ids=2 row-size=75B cardinality=1.92M
+|  |     tuple-ids=2 row-size=75B cardinality=100.00K(filtered from 1.92M)
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  23:EXCHANGE [HASH(ss_customer_sk)]
@@ -797,9 +797,9 @@ Per-Instance Resources: mem-estimate=11.64MB mem-reservation=2.00MB thread-reser
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=119(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
-|     tuple-ids=3 row-size=8B cardinality=2.88M
+|     tuple-ids=3 row-size=8B cardinality=186.34K(filtered from 2.88M)
 |     in pipelines: 03(GETNEXT)
 |
 27:EXCHANGE [HASH(cs_ship_customer_sk)]
@@ -854,6 +854,6 @@ Per-Instance Resources: mem-estimate=48.33MB mem-reservation=4.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=650.14K
    mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
-   tuple-ids=9 row-size=8B cardinality=1.44M
+   tuple-ids=9 row-size=8B cardinality=93.21K(filtered from 1.44M)
    in pipelines: 09(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 e5f9648aa..e3e4393fe 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
@@ -129,9 +129,9 @@ PLAN-ROOT SINK
 |  |       table: rows=2.88M size=200.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |  |     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-|  |     tuple-ids=5 row-size=20B cardinality=2.88M
+|  |     tuple-ids=5 row-size=20B cardinality=170.55K(filtered from 2.88M)
 |  |     in pipelines: 08(GETNEXT)
 |  |
 |  |--07:HASH JOIN [INNER JOIN]
@@ -163,7 +163,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=3 row-size=20B cardinality=1.44M
+|  |     tuple-ids=3 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |  |     in pipelines: 05(GETNEXT)
 |  |
 |  04:HASH JOIN [INNER JOIN]
@@ -195,7 +195,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=1 row-size=20B cardinality=719.38K
+|     tuple-ids=1 row-size=20B cardinality=42.85K(filtered from 719.38K)
 |     in pipelines: 02(GETNEXT)
 |
 11:SCAN HDFS [tpcds_parquet.time_dim]
@@ -209,7 +209,7 @@ PLAN-ROOT SINK
    parquet statistics predicates: t_meal_time IN ('breakfast', 'dinner')
    parquet dictionary predicates: t_meal_time IN ('breakfast', 'dinner')
    mem-estimate=64.00MB mem-reservation=512.00KB thread-reservation=1
-   tuple-ids=9 row-size=31B cardinality=57.60K
+   tuple-ids=9 row-size=31B cardinality=1.00K(filtered from 57.60K)
    in pipelines: 11(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=42.94MB Threads=15
@@ -358,9 +358,9 @@ Per-Host Resources: mem-estimate=201.69MB mem-reservation=16.88MB thread-reserva
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=48.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=5 row-size=20B cardinality=2.88M
+|     tuple-ids=5 row-size=20B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 08(GETNEXT)
 |
 |--07:HASH JOIN [INNER JOIN, BROADCAST]
@@ -399,7 +399,7 @@ Per-Host Resources: mem-estimate=201.69MB mem-reservation=16.88MB thread-reserva
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=192.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=3 row-size=20B cardinality=1.44M
+|     tuple-ids=3 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 05(GETNEXT)
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -438,7 +438,7 @@ Per-Host Resources: mem-estimate=201.69MB mem-reservation=16.88MB thread-reserva
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=128.00MB mem-reservation=8.00MB thread-reservation=1
-   tuple-ids=1 row-size=20B cardinality=719.38K
+   tuple-ids=1 row-size=20B cardinality=42.85K(filtered from 719.38K)
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=84.50MB Threads=18
@@ -612,9 +612,9 @@ Per-Instance Resources: mem-estimate=49.60MB mem-reservation=8.00MB thread-reser
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=109(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-|     tuple-ids=5 row-size=20B cardinality=2.88M
+|     tuple-ids=5 row-size=20B cardinality=170.55K(filtered from 2.88M)
 |     in pipelines: 08(GETNEXT)
 |
 |--07:HASH JOIN [INNER JOIN, BROADCAST]
@@ -661,7 +661,7 @@ Per-Instance Resources: mem-estimate=49.60MB mem-reservation=8.00MB thread-reser
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=3 row-size=20B cardinality=1.44M
+|     tuple-ids=3 row-size=20B cardinality=85.31K(filtered from 1.44M)
 |     in pipelines: 05(GETNEXT)
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -708,6 +708,6 @@ Per-Instance Resources: mem-estimate=49.60MB mem-reservation=8.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=644.77K
    mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-   tuple-ids=1 row-size=20B cardinality=719.38K
+   tuple-ids=1 row-size=20B cardinality=42.85K(filtered from 719.38K)
    in pipelines: 02(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test
index 4e48cb75c..369a3dbfe 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test
@@ -136,7 +136,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|     tuple-ids=0 row-size=24B cardinality=2.88M
+|     tuple-ids=0 row-size=24B cardinality=162.45K(filtered from 2.88M)
 |     in pipelines: 00(GETNEXT)
 |
 08:SCAN HDFS [tpcds_parquet.customer]
@@ -147,7 +147,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=100.00K
    mem-estimate=80.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=6 row-size=68B cardinality=100.00K
+   tuple-ids=6 row-size=68B cardinality=16.25K(filtered from 100.00K)
    in pipelines: 08(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=29.95MB Threads=13
@@ -317,7 +317,7 @@ Per-Host Resources: mem-estimate=85.08MB mem-reservation=15.75MB thread-reservat
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-   tuple-ids=0 row-size=24B cardinality=2.88M
+   tuple-ids=0 row-size=24B cardinality=162.45K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=45.70MB Threads=16
@@ -520,6 +520,6 @@ Per-Instance Resources: mem-estimate=26.56MB mem-reservation=4.00MB thread-reser
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
    mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
-   tuple-ids=0 row-size=24B cardinality=2.88M
+   tuple-ids=0 row-size=24B cardinality=162.45K(filtered from 2.88M)
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
index 31eb44756..88cb03077 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
@@ -137,7 +137,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=35 row-size=12B cardinality=719.38K
+|  |     tuple-ids=35 row-size=12B cardinality=148.00K(filtered from 719.38K)
 |  |     in pipelines: 23(GETNEXT)
 |  |
 |  22:SCAN HDFS [tpcds_parquet.customer]
@@ -219,9 +219,9 @@ PLAN-ROOT SINK
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=1 row-size=12B cardinality=2.88M
+|     tuple-ids=1 row-size=12B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 02(GETNEXT)
 |
 28:HASH JOIN [INNER JOIN]
@@ -282,7 +282,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=25 row-size=12B cardinality=719.38K
+|  |     tuple-ids=25 row-size=12B cardinality=148.00K(filtered from 719.38K)
 |  |     in pipelines: 16(GETNEXT)
 |  |
 |  15:SCAN HDFS [tpcds_parquet.customer]
@@ -355,9 +355,9 @@ PLAN-ROOT SINK
      table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=11 row-size=12B cardinality=2.88M
+   tuple-ids=11 row-size=12B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 09(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=239.69MB Threads=34
@@ -493,7 +493,7 @@ Per-Host Resources: mem-estimate=11.62MB mem-reservation=4.75MB thread-reservati
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=35 row-size=12B cardinality=719.38K
+|     tuple-ids=35 row-size=12B cardinality=148.00K(filtered from 719.38K)
 |     in pipelines: 23(GETNEXT)
 |
 54:EXCHANGE [HASH(customer_id)]
@@ -613,9 +613,9 @@ Per-Host Resources: mem-estimate=56.27MB mem-reservation=42.69MB thread-reservat
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-|     tuple-ids=1 row-size=12B cardinality=2.88M
+|     tuple-ids=1 row-size=12B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 02(GETNEXT)
 |
 28:HASH JOIN [INNER JOIN, BROADCAST]
@@ -729,7 +729,7 @@ Per-Host Resources: mem-estimate=56.27MB mem-reservation=42.69MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=96.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=25 row-size=12B cardinality=719.38K
+|     tuple-ids=25 row-size=12B cardinality=148.00K(filtered from 719.38K)
 |     in pipelines: 16(GETNEXT)
 |
 07:UNION
@@ -826,9 +826,9 @@ Per-Host Resources: mem-estimate=36.27MB mem-reservation=4.94MB thread-reservati
      table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=32.00MB mem-reservation=1.00MB thread-reservation=1
-   tuple-ids=11 row-size=12B cardinality=2.88M
+   tuple-ids=11 row-size=12B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=267.12MB Threads=43
@@ -989,7 +989,7 @@ Per-Instance Resources: mem-estimate=4.21MB mem-reservation=0B thread-reservatio
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=35 row-size=12B cardinality=719.38K
+|     tuple-ids=35 row-size=12B cardinality=148.00K(filtered from 719.38K)
 |     in pipelines: 23(GETNEXT)
 |
 54:EXCHANGE [HASH(customer_id)]
@@ -1134,9 +1134,9 @@ Per-Instance Resources: mem-estimate=31.73MB mem-reservation=17.00MB thread-rese
 |       table: rows=2.88M size=200.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
-|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-|     tuple-ids=1 row-size=12B cardinality=2.88M
+|     tuple-ids=1 row-size=12B cardinality=589.03K(filtered from 2.88M)
 |     in pipelines: 02(GETNEXT)
 |
 28:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1276,7 +1276,7 @@ Per-Instance Resources: mem-estimate=31.73MB mem-reservation=17.00MB thread-rese
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
-|     tuple-ids=25 row-size=12B cardinality=719.38K
+|     tuple-ids=25 row-size=12B cardinality=148.00K(filtered from 719.38K)
 |     in pipelines: 16(GETNEXT)
 |
 07:UNION
@@ -1391,8 +1391,8 @@ Per-Instance Resources: mem-estimate=16.66MB mem-reservation=1.00MB thread-reser
      table: rows=2.88M size=200.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
-   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
    mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
-   tuple-ids=11 row-size=12B cardinality=2.88M
+   tuple-ids=11 row-size=12B cardinality=589.03K(filtered from 2.88M)
    in pipelines: 09(GETNEXT)
 ====
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 6b8dce164..acb950d06 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
@@ -176,7 +176,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |  |     mem-estimate=160.00MB mem-reservation=8.00MB thread-reservation=1
-|  |  |     tuple-ids=24 row-size=28B cardinality=719.38K
+|  |  |     tuple-ids=24 row-size=28B cardinality=71.88K(filtered from 719.38K)
 |  |  |     in pipelines: 39(GETNEXT)
 |  |  |
 |  |  42:SCAN HDFS [tpcds_parquet.web_returns]
@@ -248,7 +248,7 @@ PLAN-ROOT SINK
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |  |  |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|  |  |     tuple-ids=20 row-size=28B cardinality=2.88M
+|  |  |     tuple-ids=20 row-size=28B cardinality=287.80K(filtered from 2.88M)
 |  |  |     in pipelines: 32(GETNEXT)
 |  |  |
 |  |  35:SCAN HDFS [tpcds_parquet.store_returns]
@@ -320,7 +320,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |  |     mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=16 row-size=28B cardinality=1.44M
+|  |     tuple-ids=16 row-size=28B cardinality=144.03K(filtered from 1.44M)
 |  |     in pipelines: 25(GETNEXT)
 |  |
 |  28:SCAN HDFS [tpcds_parquet.catalog_returns]
@@ -412,7 +412,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=160.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=8 row-size=28B cardinality=719.38K
+|  |     tuple-ids=8 row-size=28B cardinality=71.88K(filtered from 719.38K)
 |  |     in pipelines: 15(GETNEXT)
 |  |
 |  18:SCAN HDFS [tpcds_parquet.web_returns]
@@ -485,7 +485,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
 |  |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|  |     tuple-ids=4 row-size=28B cardinality=2.88M
+|  |     tuple-ids=4 row-size=28B cardinality=287.80K(filtered from 2.88M)
 |  |     in pipelines: 08(GETNEXT)
 |  |
 |  11:SCAN HDFS [tpcds_parquet.store_returns]
@@ -558,7 +558,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=0 row-size=28B cardinality=1.44M
+|     tuple-ids=0 row-size=28B cardinality=144.03K(filtered from 1.44M)
 |     in pipelines: 01(GETNEXT)
 |
 04:SCAN HDFS [tpcds_parquet.catalog_returns]
@@ -752,7 +752,7 @@ Per-Host Resources: mem-estimate=2.10MB mem-reservation=1.94MB thread-reservatio
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |  |     mem-estimate=160.00MB mem-reservation=8.00MB thread-reservation=1
-|  |     tuple-ids=24 row-size=28B cardinality=719.38K
+|  |     tuple-ids=24 row-size=28B cardinality=71.88K(filtered from 719.38K)
 |  |     in pipelines: 39(GETNEXT)
 |  |
 |  |--38:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
@@ -849,9 +849,9 @@ Per-Host Resources: mem-estimate=2.10MB mem-reservation=1.94MB thread-reservatio
 |  |       table: rows=2.88M size=200.96MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
-|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K est-scan-range=374(filtered from 1824)
 |  |     mem-estimate=64.00MB mem-reservation=2.00MB thread-reservation=1
-|  |     tuple-ids=20 row-size=28B cardinality=2.88M
+|  |     tuple-ids=20 row-size=28B cardinality=287.80K(filtered from 2.88M)
 |  |     in pipelines: 32(GETNEXT)
 |  |
 |  31:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
@@ -949,7 +949,7 @@ Per-Host Resources: mem-estimate=2.10MB mem-reservation=1.94MB thread-reservatio
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=650.14K
 |     mem-estimate=240.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=16 row-size=28B cardinality=1.44M
+|     tuple-ids=16 row-size=28B cardinality=144.03K(filtered from 1.44M)
 |     in pipelines: 25(GETNEXT)
 |
 82:EXCHANGE [HASH(i_brand_id,i_category_id,i_class_id,i_manufact_id)]
@@ -1101,7 +1101,7 @@ Per-Host Resources: mem-estimate=27.25MB mem-reservation=11.50MB thread-reservat
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=644.77K
 |     mem-estimate=160.00MB mem-reservation=8.00MB thread-reservation=1
-|     tuple-ids=8 row-size=28B cardinality=719.38K
+|     tuple-ids=8 row-size=28B cardinality=71.88K(filtered from 719.38K)
 |     in pipelines: 15(GETNEXT)
 |
 |--14:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
@@ -1199,9 +1199,9 @@ Per-Host Resources: mem-estimate=27.25MB mem-reservation=11.50MB thread-reservat
 |       table: rows=2.88M size=200.96MB
 |       partitions: 1824/1824 rows=2.88M
... 1762 lines suppressed ...