You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by tm...@apache.org on 2021/03/03 00:38:46 UTC

[impala] 04/06: IMPALA-9856: Enable result spooling by default.

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

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

commit 49ac55fb691e280ccf061a3fa067aff938614af9
Author: Riza Suminto <ri...@cloudera.com>
AuthorDate: Wed Oct 21 11:23:35 2020 -0700

    IMPALA-9856: Enable result spooling by default.
    
    Result spooling has been relatively stable since it was introduced, and
    it has several benefits described in IMPALA-8656. This patch enable
    result spooling (SPOOL_QUERY_RESULTS) query options by default.
    
    Furthermore, some tests need to be adjusted to account for result
    spooling by default. The following are the adjustment categories and
    list of tests that fall under such category.
    
    Change in assertions:
    PlannerTest#testAcidTableScans
    PlannerTest#testBloomFilterAssignment
    PlannerTest#testConstantFolding
    PlannerTest#testFkPkJoinDetection
    PlannerTest#testFkPkJoinDetectionWithHDFSNumRowsEstDisabled
    PlannerTest#testKuduSelectivity
    PlannerTest#testMaxRowSize
    PlannerTest#testMinMaxRuntimeFilters
    PlannerTest#testMinMaxRuntimeFiltersWithHDFSNumRowsEstDisabled
    PlannerTest#testMtDopValidation
    PlannerTest#testParquetFiltering
    PlannerTest#testParquetFilteringDisabled
    PlannerTest#testPartitionPruning
    PlannerTest#testPreaggBytesLimit
    PlannerTest#testResourceRequirements
    PlannerTest#testRuntimeFilterQueryOptions
    PlannerTest#testSortExprMaterialization
    PlannerTest#testSpillableBufferSizing
    PlannerTest#testTableSample
    PlannerTest#testTpch
    PlannerTest#testKuduTpch
    PlannerTest#testTpchNested
    PlannerTest#testUnion
    TpcdsPlannerTest
    custom_cluster/test_admission_controller.py::TestAdmissionController::test_dedicated_coordinator_planner_estimates
    custom_cluster/test_admission_controller.py::TestAdmissionController::test_memory_rejection
    custom_cluster/test_admission_controller.py::TestAdmissionController::test_pool_mem_limit_configs
    metadata/test_explain.py::TestExplain::test_explain_level2
    metadata/test_explain.py::TestExplain::test_explain_level3
    metadata/test_stats_extrapolation.py::TestStatsExtrapolation::test_stats_extrapolation
    
    Increase BUFFER_POOL_LIMIT:
    query_test/test_queries.py::TestQueries::test_analytic_fns
    query_test/test_runtime_filters.py::TestRuntimeRowFilters::test_row_filter_reservation
    query_test/test_sort.py::TestQueryFullSort::test_multiple_mem_limits_full_output
    query_test/test_spilling.py::TestSpillingBroadcastJoins::test_spilling_broadcast_joins
    query_test/test_spilling.py::TestSpillingDebugActionDimensions::test_spilling_aggs
    query_test/test_spilling.py::TestSpillingDebugActionDimensions::test_spilling_regression_exhaustive
    query_test/test_udfs.py::TestUdfExecution::test_mem_limits
    
    Increase MEM_LIMIT:
    query_test/test_mem_usage_scaling.py::TestExchangeMemUsage::test_exchange_mem_usage_scaling
    query_test/test_mem_usage_scaling.py::TestScanMemLimit::test_hdfs_scanner_thread_mem_scaling
    
    Increase MAX_ROW_SIZE:
    custom_cluster/test_parquet_max_page_header.py::TestParquetMaxPageHeader::test_large_page_header_config
    query_test/test_insert.py::TestInsertQueries::test_insert_large_string
    query_test/test_query_mem_limit.py::TestQueryMemLimit::test_mem_limit
    query_test/test_scanners.py::TestTextSplitDelimiters::test_text_split_across_buffers_delimiter
    query_test/test_scanners.py::TestWideRow::test_wide_row
    
    Disable result spooling to maintain assertion:
    custom_cluster/test_admission_controller.py::TestAdmissionController::test_set_request_pool
    custom_cluster/test_admission_controller.py::TestAdmissionController::test_timeout_reason_host_memory
    custom_cluster/test_admission_controller.py::TestAdmissionController::test_timeout_reason_pool_memory
    custom_cluster/test_admission_controller.py::TestAdmissionController::test_queue_reasons_memory
    custom_cluster/test_admission_controller.py::TestAdmissionController::test_pool_config_change_while_queued
    custom_cluster/test_query_retries.py::TestQueryRetries::test_retry_fetched_rows
    custom_cluster/test_query_retries.py::TestQueryRetries::test_retry_finished_query
    custom_cluster/test_scratch_disk.py::TestScratchDir::test_no_dirs
    custom_cluster/test_scratch_disk.py::TestScratchDir::test_non_existing_dirs
    custom_cluster/test_scratch_disk.py::TestScratchDir::test_non_writable_dirs
    query_test/test_insert.py::TestInsertQueries::test_insert_large_string (the last query only)
    query_test/test_kudu.py::TestKuduMemLimits::test_low_mem_limit_low_selectivity_scan
    query_test/test_mem_usage_scaling.py::TestScanMemLimit::test_kudu_scan_mem_usage
    query_test/test_queries.py::TestQueriesParquetTables::test_very_large_strings
    query_test/test_query_mem_limit.py::TestCodegenMemLimit::test_codegen_mem_limit
    shell/test_shell_client.py::TestShellClient::test_fetch_size
    
    Testing:
    - Pass exhaustive tests.
    
    Change-Id: I9e360c1428676d8f3fab5d95efee18aca085eba4
    Reviewed-on: http://gerrit.cloudera.org:8080/16755
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 common/thrift/ImpalaInternalService.thrift         |   2 +-
 .../queries/PlannerTest/acid-scans.test            |   8 +-
 .../PlannerTest/bloom-filter-assignment.test       |  20 +-
 .../queries/PlannerTest/constant-folding.test      |  44 +-
 ...k-join-detection-hdfs-num-rows-est-enabled.test |   4 +-
 .../queries/PlannerTest/fk-pk-join-detection.test  |  22 +-
 .../queries/PlannerTest/kudu-selectivity.test      |  26 +-
 .../queries/PlannerTest/max-row-size.test          |  56 +-
 ...-runtime-filters-hdfs-num-rows-est-enabled.test |   2 +-
 .../PlannerTest/min-max-runtime-filters.test       |  12 +-
 .../queries/PlannerTest/mt-dop-validation.test     |  24 +-
 .../PlannerTest/parquet-filtering-disabled.test    |  14 +-
 .../queries/PlannerTest/parquet-filtering.test     |  26 +-
 .../queries/PlannerTest/partition-pruning.test     |  12 +-
 .../queries/PlannerTest/preagg-bytes-limit.test    |   6 +-
 .../queries/PlannerTest/resource-requirements.test | 924 ++++++++++-----------
 .../PlannerTest/runtime-filter-query-options.test  |  12 +-
 .../PlannerTest/sort-expr-materialization.test     |  40 +-
 .../PlannerTest/spillable-buffer-sizing.test       | 144 ++--
 .../queries/PlannerTest/tablesample.test           |  44 +-
 .../queries/PlannerTest/tpcds/tpcds-q01.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q02.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q03.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q04.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q05.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q06.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q07.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q08.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q09.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q10a.test      |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q11.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q12.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q13.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q14a.test      |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q14b.test      |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q15.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q16.test       |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q17.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q18.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q19.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q20.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q21.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q22.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q23a.test      |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q23b.test      |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q24a.test      |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q24b.test      |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q25.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q26.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q27.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q28.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q29.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q30.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q31.test       |  22 +-
 .../queries/PlannerTest/tpcds/tpcds-q32.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q33.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q34.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q35a.test      |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q36.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q37.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q38.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q39a.test      |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q39b.test      |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q40.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q41.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q42.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q43.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q44.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q45.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q46.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q47.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q48.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q49.test       |   6 +-
 .../queries/PlannerTest/tpcds/tpcds-q50.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q51.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q52.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q53.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q54.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q55.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q56.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q57.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q58.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q59.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q60.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q61.test       |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q62.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q63.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q64.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q65.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q66.test       |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q67.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q68.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q69.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q70.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q71.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q72.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q73.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q74.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q75.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q76.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q77.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q78.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q79.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q80.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q81.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q82.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q83.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q84.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q85.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q86.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q87.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q88.test       |  16 +-
 .../queries/PlannerTest/tpcds/tpcds-q89.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q90.test       |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q91.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q92.test       |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q93.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q94.test       |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q95.test       |  14 +-
 .../queries/PlannerTest/tpcds/tpcds-q96.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q97.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q98.test       |  18 +-
 .../queries/PlannerTest/tpcds/tpcds-q99.test       |  18 +-
 .../queries/PlannerTest/tpch-all.test              | 176 ++--
 .../queries/PlannerTest/tpch-kudu.test             |  20 +-
 .../queries/PlannerTest/tpch-nested.test           |  90 +-
 .../queries/PlannerTest/union.test                 |  48 +-
 .../QueryTest/admission-max-min-mem-limits.test    |  18 +-
 .../admission-reject-min-reservation.test          |   4 +-
 .../queries/QueryTest/analytic-fns.test            |   2 +-
 .../queries/QueryTest/codegen-mem-limit.test       |   3 +
 .../QueryTest/dedicated-coord-mem-estimates.test   |  18 +-
 .../queries/QueryTest/exchange-mem-scaling.test    |   2 +-
 .../queries/QueryTest/explain-level2.test          |   4 +-
 .../queries/QueryTest/explain-level3.test          |   2 +-
 .../QueryTest/hdfs-scanner-thread-mem-scaling.test |   2 +-
 .../QueryTest/runtime_row_filter_reservations.test |   4 +-
 .../queries/QueryTest/spilling-aggs.test           |   2 +-
 .../QueryTest/spilling-broadcast-joins.test        |   2 +-
 ...gression-exhaustive-no-default-buffer-size.test |   2 +-
 .../queries/QueryTest/stats-extrapolation.test     |   4 +-
 tests/custom_cluster/test_admission_controller.py  |  32 +-
 .../custom_cluster/test_parquet_max_page_header.py |   3 +
 tests/custom_cluster/test_query_retries.py         |  16 +-
 tests/custom_cluster/test_scratch_disk.py          |   9 +
 tests/query_test/test_insert.py                    |   4 +
 tests/query_test/test_kudu.py                      |   3 +
 tests/query_test/test_mem_usage_scaling.py         |   9 +-
 tests/query_test/test_queries.py                   |   6 +-
 tests/query_test/test_query_mem_limit.py           |   4 +
 tests/query_test/test_scanners.py                  |   7 +-
 tests/query_test/test_sort.py                      |   4 +-
 tests/query_test/test_udfs.py                      |   2 +-
 tests/shell/test_shell_client.py                   |   2 +-
 154 files changed, 1896 insertions(+), 1828 deletions(-)

diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index 2983d12..668d183 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -365,7 +365,7 @@ struct TQueryOptions {
   85: optional string default_hints_insert_statement;
 
   // See comment in ImpalaService.thrift
-  86: optional bool spool_query_results = false;
+  86: optional bool spool_query_results = true;
 
   // See comment in ImpalaService.thrift
   87: optional TTransactionalType default_transactional_type = TTransactionalType.NONE;
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test b/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
index ebd8cde..4ed5b97 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/acid-scans.test
@@ -266,7 +266,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=228.88MB mem-reservation=6.98MB thread-reservation=5 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: dd.id, dd.bool_col, dd.tinyint_col, dd.smallint_col, dd.int_col, dd.bigint_col, dd.float_col, dd.double_col, dd.date_string_col, dd.string_col, dd.timestamp_col, dd.year, dd.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 07:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = max(id)
@@ -343,7 +343,7 @@ F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=424.01KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: dd.id, dd.bool_col, dd.tinyint_col, dd.smallint_col, dd.int_col, dd.bigint_col, dd.float_col, dd.double_col, dd.date_string_col, dd.string_col, dd.timestamp_col, dd.year, dd.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=424.01KB mem-reservation=0B thread-reservation=0
@@ -470,7 +470,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=212.88MB mem-reservation=6.89MB thread-reservation=5 runtime-filters-memory=1.00MB
   PLAN-ROOT SINK
   |  output exprs: t1.id, t2.`month`
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   06:HASH JOIN [INNER JOIN]
   |  hash predicates: t1.id % 12 = t2.`month`
@@ -545,7 +545,7 @@ F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=271.21KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
   |  output exprs: t1.id, t2.`month`
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   11:EXCHANGE [UNPARTITIONED]
      mem-estimate=271.21KB mem-reservation=0B thread-reservation=0
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 9e23d9f..be01aa7 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/bloom-filter-assignment.test
@@ -10,7 +10,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=34.94MB mem-reservation=2.97MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -61,7 +61,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=13.69MB mem-reservation=2.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -100,7 +100,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=28.94MB mem-reservation=2.95MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -145,7 +145,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=19.69MB mem-reservation=2.95MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -190,7 +190,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=28.94MB mem-reservation=2.95MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -235,7 +235,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=18.69MB mem-reservation=1.95MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -280,7 +280,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=54.62MB mem-reservation=5.91MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -351,7 +351,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=54.62MB mem-reservation=5.91MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -421,7 +421,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=55.62MB mem-reservation=6.91MB thread-reservation=4 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -492,7 +492,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.88MB mem-reservation=6.89MB thread-reservation=4 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
index c942508..173d5aa 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
@@ -9,10 +9,10 @@ c.c_orders o, o.o_lineitems WHERE c_custkey > CAST(10 AS BIGINT) AND o_orderkey
 = CAST(4 AS BIGINT) AND CAST(20 AS BIGINT) + CAST(l_linenumber AS BIGINT) <
 CAST(0 AS BIGINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=264.00MB mem-reservation=16.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=326.94MB mem-reservation=20.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=62.94MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -84,10 +84,10 @@ where string_col = cast(4 as string) and 2 + 3 = tinyint_col
 Analyzed query: SELECT * FROM functional_hbase.stringids WHERE string_col = '4'
 AND tinyint_col = CAST(5 AS TINYINT) AND id >= '10' AND id <= '20'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_hbase.stringids.id, functional_hbase.stringids.bigint_col, functional_hbase.stringids.bool_col, functional_hbase.stringids.date_string_col, functional_hbase.stringids.day, functional_hbase.stringids.double_col, functional_hbase.stringids.float_col, functional_hbase.stringids.int_col, functional_hbase.stringids.month, functional_hbase.stringids.smallint_col, functional_hbase.stringids.string_col, functional_hbase.stringids.timestamp_col, functional_hbase.string [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HBASE [functional_hbase.stringids]
    key predicates: id >= '10', id <= '20'
@@ -110,10 +110,10 @@ Analyzed query: SELECT * FROM functional.alltypes_datasource WHERE
 CAST(tinyint_col AS DOUBLE) < CAST(256 AS DOUBLE) AND float_col != CAST(0 AS
 FLOAT) AND int_col < CAST(2 AS INT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=1.00GB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes_datasource.id, functional.alltypes_datasource.bool_col, functional.alltypes_datasource.tinyint_col, functional.alltypes_datasource.smallint_col, functional.alltypes_datasource.int_col, functional.alltypes_datasource.bigint_col, functional.alltypes_datasource.float_col, functional.alltypes_datasource.double_col, functional.alltypes_datasource.timestamp_col, functional.alltypes_datasource.string_col, functional.alltypes_datasource.dec_col1, functional.a [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
 data source predicates: CAST(tinyint_col AS DOUBLE) < CAST(256 AS DOUBLE), int_col < CAST(2 AS INT)
@@ -137,10 +137,10 @@ BIGINT) AND sum(CAST(2 AS BIGINT) + CAST(id AS BIGINT)) > CAST(1 AS BIGINT) AND
 sum(CAST(2 AS BIGINT) + CAST(id AS BIGINT)) >= CAST(5 AS BIGINT) AND sum(CAST(2
 AS BIGINT) + CAST(id AS BIGINT)) <= CAST(10 AS BIGINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=1.97MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=5.94MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: sum(2 + id)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: sum(CAST(2 AS BIGINT) + CAST(id AS BIGINT)), count(*)
@@ -175,10 +175,10 @@ AS BIGINT) + b.bigint_col AND CAST(a.int_col AS BIGINT) <= b.bigint_col +
 CAST(97 AS BIGINT)) WHERE CAST(b.double_col AS DECIMAL(3,2)) > CAST(11.1 AS
 DECIMAL(6,1))
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=257.94MB mem-reservation=2.00MB thread-reservation=3
+|  Per-Host Resources: mem-estimate=257.94MB mem-reservation=5.97MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: 2 + a.id = b.id - 2
@@ -228,10 +228,10 @@ CAST(b.id AS BIGINT) - CAST(2 AS BIGINT) OR CAST(a.int_col AS BIGINT) >= CAST(0
 AS BIGINT) + b.bigint_col WHERE CAST(b.double_col AS DECIMAL(3,2)) > CAST(11.1
 AS DECIMAL(6,1))
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=256.01MB mem-reservation=64.00KB thread-reservation=3
+|  Per-Host Resources: mem-estimate=260.01MB mem-reservation=4.06MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:NESTED LOOP JOIN [LEFT OUTER JOIN]
 |  join predicates: CAST(2 AS BIGINT) + CAST(a.id AS BIGINT) = CAST(b.id AS BIGINT) - CAST(2 AS BIGINT) OR CAST(a.int_col AS BIGINT) <= b.bigint_col + CAST(97 AS BIGINT), CAST(2 AS BIGINT) + CAST(a.id AS BIGINT) = CAST(b.id AS BIGINT) - CAST(2 AS BIGINT) OR CAST(a.int_col AS BIGINT) >= CAST(0 AS BIGINT) + b.bigint_col
@@ -274,10 +274,10 @@ functional.alltypes GROUP BY timestamp_col = TIMESTAMP '2016-11-15 00:00:00'
 HAVING CAST(1048576 AS BIGINT) * count(*) % CAST(2 AS BIGINT) = CAST(0 AS
 BIGINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=3.88MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=5.94MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: sum(2 + id)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:AGGREGATE [FINALIZE]
 |  output: sum(2 + id), count:merge(*)
@@ -314,10 +314,10 @@ Analyzed query: SELECT sum(DISTINCT CAST(2 AS BIGINT) + CAST(id AS BIGINT)) FROM
 functional.alltypes HAVING CAST(1048576 AS BIGINT) * count(*) % CAST(2 AS
 BIGINT) = CAST(0 AS BIGINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=1.97MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: sum(2 + id)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:AGGREGATE [FINALIZE]
 |  output: sum(2 + id), count:merge(*)
@@ -354,10 +354,10 @@ Analyzed query: SELECT first_value(1 + 1 + int_col - (1 - 1)) OVER (PARTITION BY
 concat(concat('a', 'b'), string_col) ORDER BY greatest(greatest(10, 20),
 bigint_col) ASC) FROM functional.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=16.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=20.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: first_value(2 + int_col - 0)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:ANALYTIC
 |  functions: first_value(2 + int_col - 0)
@@ -393,10 +393,10 @@ order by id * abs((factorial(5) / power(2, 4)))
 Analyzed query: SELECT int_col FROM functional.alltypes ORDER BY id * CAST(7.5
 AS DOUBLE) ASC
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=134.00MB mem-reservation=6.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=134.00MB mem-reservation=10.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: int_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SORT
 |  order by: id * 7.5 ASC
@@ -460,10 +460,10 @@ id, CAST(10 AS INT) + CAST(c2 AS INT) c3 FROM (SELECT id, CAST(20 AS SMALLINT) +
 CAST(c1 AS SMALLINT) c2 FROM (SELECT id, CAST(30 AS TINYINT) c1 FROM
 functional.alltypes LIMIT CAST(2 AS TINYINT)) v1) v2) v3
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: sum(id + c3)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: sum(CAST(id AS BIGINT) + CAST(CAST(10 AS INT) + CAST(CAST(20 AS SMALLINT) + CAST(30 AS SMALLINT) AS INT) AS BIGINT))
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
index 0b8eabb..c5dcf6e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
@@ -13,7 +13,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=178.94MB mem-reservation=18.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=21.98MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_customer_sk = c_customer_sk
@@ -54,7 +54,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=178.94MB mem-reservation=18.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=61.64MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_customer_sk = c_customer_sk
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 86c0d32..ad7077e 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
@@ -8,7 +8,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=181.75MB mem-reservation=21.75MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_customer_sk = c_customer_sk
@@ -52,7 +52,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=180.75MB mem-reservation=20.75MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: ss_customer_sk = c_customer_sk
@@ -95,7 +95,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=181.75MB mem-reservation=21.75MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: ss_customer_sk = c_customer_sk
@@ -138,7 +138,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=214.75MB mem-reservation=22.75MB thread-reservation=3 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=37.98MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
@@ -180,7 +180,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=397.67MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_sold_time_sk = ws_sold_time_sk
@@ -222,7 +222,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=114.00MB mem-reservation=34.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: a.d_date_sk, a.d_date_id, a.d_date, a.d_month_seq, a.d_week_seq, a.d_quarter_seq, a.d_year, a.d_dow, a.d_moy, a.d_dom, a.d_qoy, a.d_fy_year, a.d_fy_quarter_seq, a.d_fy_week_seq, a.d_day_name, a.d_quarter_name, a.d_holiday, a.d_weekend, a.d_following_holiday, a.d_first_dom, a.d_last_dom, a.d_same_day_ly, a.d_same_day_lq, a.d_current_day, a.d_current_week, a.d_current_month, a.d_current_quarter, a.d_current_year, b.d_date_sk, b.d_date_id, b.d_date, b.d_month_seq, b.d_week_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=17.77MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: b.d_date_sk = a.d_date_sk
@@ -268,7 +268,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=362.81MB mem-reservation=50.81MB thread-reservation=6 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_addr_sk = c_current_addr_sk
@@ -365,7 +365,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=211.00MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_customer_sk % 10 = c_customer_sk / 100
@@ -407,7 +407,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=2.17GB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=21.98MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_customer_sk = c_customer_sk
@@ -448,7 +448,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=178.94MB mem-reservation=18.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: CAST(1 AS TINYINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_customer_sk = c_customer_sk
@@ -491,7 +491,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=299.00MB mem-reservation=18.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: ss_sold_time_sk = ws_sold_time_sk
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
index fde05a0..b1a4538 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
@@ -4,7 +4,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=1.88MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      kudu predicates: id = '8600000US00601'
@@ -16,7 +16,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   01:EXCHANGE [UNPARTITIONED]
      mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -40,7 +40,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      predicates: id != '1'
@@ -53,7 +53,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   01:EXCHANGE [UNPARTITIONED]
      mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -77,7 +77,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      kudu predicates: zip > '2', id > '1'
@@ -89,7 +89,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=517.93KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   01:EXCHANGE [UNPARTITIONED]
      mem-estimate=517.93KB mem-reservation=0B thread-reservation=0
@@ -112,7 +112,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      predicates: id IN ('1', '2') OR zip = '3'
@@ -124,7 +124,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   01:EXCHANGE [UNPARTITIONED]
      mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -170,7 +170,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.alltypes]
      predicates: id IN (int_col), CAST(int_col AS BIGINT) IN (CAST(9999999999 AS BIGINT)), CAST(bigint_col AS DECIMAL(19,0)) IN (CAST(9999999999999999999 AS DECIMAL(19,0))), double_col IN (CAST('inf' AS DOUBLE)), float_col IN (CAST('NaN' AS FLOAT)), CAST(smallint_col AS INT) IN (CAST(99999 AS INT), CAST(2 AS INT)), CAST(tinyint_col AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(999 AS SMALLINT)), CAST(bool_col AS TINYINT) IN (CAST(1 AS TINYINT)), string_col NOT IN ('bar')
@@ -186,7 +186,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=4.50MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.decimal_tbl.d1, functional_kudu.decimal_tbl.d2, functional_kudu.decimal_tbl.d3, functional_kudu.decimal_tbl.d4, functional_kudu.decimal_tbl.d5, functional_kudu.decimal_tbl.d6
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.decimal_tbl]
      kudu predicates: d1 IN (CAST(1234 AS DECIMAL(9,0)), CAST(12345 AS DECIMAL(9,0)))
@@ -205,7 +205,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.alltypes]
      predicates: CAST(date_string_col AS TINYINT) IS NULL, timestamp_col < NULL
@@ -222,7 +222,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.alltypes]
      kudu predicates: timestamp_col IN (TIMESTAMP '2010-03-01 00:00:00', TIMESTAMP '2010-03-01 00:01:00')
@@ -239,7 +239,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
   |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
-  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.alltypes]
      predicates: timestamp_col IN (TIMESTAMP '2010-03-01 00:00:00', NULL, TIMESTAMP '2010-03-01 00:01:00')
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 ab405df..36e2de6 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
@@ -4,16 +4,16 @@ select straight_join *
 from tpch_parquet.customer
     inner join tpch_parquet.nation on c_nationkey = n_nationkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=33.97MB Threads=5
-Per-Host Resource Estimates: Memory=68MB
+Max Per-Host Resource Reservation: Memory=49.97MB Threads=5
+Per-Host Resource Estimates: Memory=115MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.customer INNER
 JOIN tpch_parquet.nation ON c_nationkey = n_nationkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.33MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=57.09MB mem-reservation=16.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.customer.c_acctbal, tpch_parquet.customer.c_mktsegment, tpch_parquet.customer.c_comment, tpch_parquet.nation.n_nationkey, tpch_parquet.nation.n_name, tpch_parquet.nation.n_regionkey, tpch_parquet.nation.n_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=46.77MB mem-reservation=16.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.33MB mem-reservation=0B thread-reservation=0
@@ -64,16 +64,16 @@ select straight_join *
 from tpch_parquet.lineitem
     left join tpch_parquet.orders on l_orderkey = o_orderkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=110.00MB Threads=5
-Per-Host Resource Estimates: Memory=410MB
+Max Per-Host Resource Reservation: Memory=126.00MB Threads=5
+Per-Host Resource Estimates: Memory=510MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.lineitem LEFT
 OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=111.20MB mem-reservation=16.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=16.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.20MB mem-reservation=0B thread-reservation=0
@@ -121,17 +121,17 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=86.00MB thread-reserva
 select * from tpch_parquet.lineitem
 where l_orderkey not in (select o_orderkey from tpch_parquet.orders)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=78.00MB Threads=5
-Per-Host Resource Estimates: Memory=175MB
+Max Per-Host Resource Reservation: Memory=94.00MB Threads=5
+Per-Host Resource Estimates: Memory=275MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem NULL AWARE LEFT ANTI JOIN
 (SELECT o_orderkey FROM tpch_parquet.orders) `$a$1` (`$c$1`) ON l_orderkey =
 `$a$1`.`$c$1`
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.69MB mem-reservation=16.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=16.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
@@ -181,18 +181,18 @@ from tpch_parquet.lineitem
 group by 1, 2
 having count(*) = 1
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=110.00MB Threads=7
-Per-Host Resource Estimates: Memory=272MB
+Max Per-Host Resource Reservation: Memory=126.00MB Threads=7
+Per-Host Resource Estimates: Memory=372MB
 Analyzed query: SELECT /* +straight_join */ l_orderkey, o_orderstatus, count(*)
 FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON o_orderkey =
 l_orderkey GROUP BY l_orderkey, o_orderstatus HAVING count(*) = CAST(1 AS
 BIGINT)
 
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.10MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.10MB mem-reservation=16.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, o_orderstatus, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=16.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.10MB mem-reservation=0B thread-reservation=0
@@ -271,15 +271,15 @@ Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservati
 select distinct *
 from tpch_parquet.lineitem
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=120.00MB Threads=4
-Per-Host Resource Estimates: Memory=1.52GB
+Max Per-Host Resource Reservation: Memory=136.00MB Threads=4
+Per-Host Resource Estimates: Memory=1.62GB
 Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.69MB mem-reservation=16.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=16.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
@@ -323,16 +323,16 @@ select l_orderkey, l_partkey, group_concat(l_linestatus, ",")
 from tpch_parquet.lineitem
 group by 1, 2
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=98.00MB Threads=4
-Per-Host Resource Estimates: Memory=276MB
+Max Per-Host Resource Reservation: Memory=114.00MB Threads=4
+Per-Host Resource Estimates: Memory=376MB
 Analyzed query: SELECT l_orderkey, l_partkey, group_concat(l_linestatus, ',')
 FROM tpch_parquet.lineitem GROUP BY l_orderkey, l_partkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.09MB mem-reservation=16.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, group_concat(l_linestatus, ',')
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=16.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.09MB mem-reservation=0B thread-reservation=0
@@ -377,17 +377,17 @@ Per-Host Resources: mem-estimate=168.14MB mem-reservation=50.00MB thread-reserva
 select max(tinyint_col) over(partition by int_col)
 from functional.alltypes
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=40.03MB Threads=4
-Per-Host Resource Estimates: Memory=56MB
+Max Per-Host Resource Reservation: Memory=56.03MB Threads=4
+Per-Host Resource Estimates: Memory=72MB
 Codegen disabled by planner
 Analyzed query: SELECT max(tinyint_col) OVER (PARTITION BY int_col) FROM
 functional.alltypes
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=56.26KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=16.05MB mem-reservation=16.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: max(tinyint_col)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=16.00MB mem-reservation=16.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=56.26KB mem-reservation=0B thread-reservation=0
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
index c4b691d..4d4821a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
@@ -11,7 +11,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=31.88MB mem-reservation=5.89MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
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 6ec2e16..685ac15 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
@@ -6,7 +6,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=13.44MB mem-reservation=1.94MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -44,7 +44,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=14.19MB mem-reservation=1.94MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -81,7 +81,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=14.94MB mem-reservation=1.94MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -117,7 +117,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=2.02GB mem-reservation=36.95MB thread-reservation=4 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -174,7 +174,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=14.62MB mem-reservation=3.88MB thread-reservation=4
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -214,4 +214,4 @@ PLAN-ROOT SINK
    mem-estimate=768.00KB mem-reservation=0B thread-reservation=1
    tuple-ids=0 row-size=4B cardinality=22
    in pipelines: 00(GETNEXT)
-====
\ No newline at end of file
+====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
index e02c5f1..96197c9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
@@ -7,7 +7,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=2.04GB mem-reservation=16.00KB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -46,7 +46,7 @@ F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
@@ -118,7 +118,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=2.03GB mem-reservation=35.02MB thread-reservation=1 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -161,7 +161,7 @@ F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
@@ -331,7 +331,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=144.00MB mem-reservation=34.02MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(int_col)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:TOP-N [LIMIT=10]
 |  order by: count(int_col) ASC, bigint_col ASC
@@ -364,7 +364,7 @@ F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(int_col)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(int_col) ASC, bigint_col ASC
@@ -425,7 +425,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=26.00MB mem-reservation=10.02MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:ANALYTIC
 |  functions: row_number()
@@ -460,7 +460,7 @@ F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=216.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=216.00KB mem-reservation=0B thread-reservation=0
@@ -514,7 +514,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=104.00MB mem-reservation=104.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, o.item.o_orderkey, o.item.o_orderstatus, o.item.o_totalprice, o.item.o_orderdate, o.item.o_orderpriority, o.item.o_clerk, o.item.o_shippriority, o.item.o_comment, o_lineitems.item.l_partkey, o_lineitems.item.l_suppkey, o_lineitems.item.l_linenumber, o_lineitems.item.l_quantity, o_lineitems.item.l_extendedprice, o_lineitems.item.l_discount, o_lineitems.item.l_tax, o_lin [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -583,7 +583,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=11.93MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, o.item.o_orderkey, o.item.o_orderstatus, o.item.o_totalprice, o.item.o_orderdate, o.item.o_orderpriority, o.item.o_clerk, o.item.o_shippriority, o.item.o_comment, o_lineitems.item.l_partkey, o_lineitems.item.l_suppkey, o_lineitems.item.l_linenumber, o_lineitems.item.l_quantity, o_lineitems.item.l_extendedprice, o_lineitems.item.l_discount, o_lineitems.item.l_tax, o_lin [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 09:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.93MB mem-reservation=0B thread-reservation=0
@@ -664,7 +664,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=89.94MB mem-reservation=17.94MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -719,7 +719,7 @@ F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=11.05MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 07:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.05MB mem-reservation=0B thread-reservation=0
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
index 6b53ede..54a1e8b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
@@ -12,7 +12,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=42.00MB mem-reservation=16.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -50,7 +50,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -82,7 +82,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=58.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -112,7 +112,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=15.11MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -181,7 +181,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=264.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_custkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=68.66MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -255,7 +255,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=616.00MB mem-reservation=32.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_name, o.o_clerk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -332,7 +332,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
index 90792b8..95923d9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
@@ -11,7 +11,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=42.00MB mem-reservation=16.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -42,7 +42,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -75,7 +75,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -111,7 +111,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -146,7 +146,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=58.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -177,7 +177,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=15.11MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -250,7 +250,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -319,7 +319,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -384,7 +384,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=264.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_custkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=68.66MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -458,7 +458,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=26.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -511,7 +511,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=616.00MB mem-reservation=32.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_name, o.o_clerk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -586,7 +586,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -623,7 +623,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.00MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test b/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
index 67b0e75..dfc4533 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
@@ -7,7 +7,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.stringpartitionkey.id, functional.stringpartitionkey.string_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.stringpartitionkey]
    partition predicates: CAST(string_col AS TIMESTAMP) = TIMESTAMP '2009-01-01 00:00:00'
@@ -28,7 +28,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.part_strings_with_quotes]
    partition predicates: p = '"'
@@ -49,7 +49,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.part_strings_with_quotes]
    partition predicates: p = '\''
@@ -70,7 +70,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.part_strings_with_quotes]
    partition predicates: p = '\\"'
@@ -91,7 +91,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.part_strings_with_quotes]
    partition predicates: p = '\\\''
@@ -112,7 +112,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.part_strings_with_quotes]
    partition predicates: p IN ('"', '\\"')
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/preagg-bytes-limit.test b/testdata/workloads/functional-planner/queries/PlannerTest/preagg-bytes-limit.test
index 5e923cf..c9a2c7a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/preagg-bytes-limit.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/preagg-bytes-limit.test
@@ -5,7 +5,7 @@ F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.08MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.08MB mem-reservation=0B thread-reservation=0
@@ -50,7 +50,7 @@ F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=61.29KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_suppkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=61.29KB mem-reservation=0B thread-reservation=0
@@ -97,7 +97,7 @@ F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.08MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=16.00MB spill-buffer=8.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.08MB mem-reservation=0B thread-reservation=0
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
index 0bb9b08..cae299a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -1,15 +1,15 @@
 # Parquet scan
 select * from tpch_parquet.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=40.00MB Threads=2
-Per-Host Resource Estimates: Memory=80MB
+Max Per-Host Resource Reservation: Memory=44.00MB Threads=2
+Per-Host Resource Estimates: Memory=180MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=180.00MB mem-reservation=44.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
    HDFS partitions=1/1 files=3 size=193.99MB
@@ -21,15 +21,15 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=40.00MB Threads=3
-Per-Host Resource Estimates: Memory=91MB
+Max Per-Host Resource Reservation: Memory=44.00MB Threads=3
+Per-Host Resource Estimates: Memory=191MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
@@ -48,15 +48,15 @@ Per-Host Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservat
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=40.00MB Threads=2
-Per-Host Resource Estimates: Memory=91MB
+Max Per-Host Resource Reservation: Memory=44.00MB Threads=2
+Per-Host Resource Estimates: Memory=191MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
@@ -79,15 +79,15 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-rese
 # scan.
 select l_comment from tpch_parquet.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
-Per-Host Resource Estimates: Memory=80MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
+Per-Host Resource Estimates: Memory=180MB
 Analyzed query: SELECT l_comment FROM tpch_parquet.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=180.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
    HDFS partitions=1/1 files=3 size=193.99MB
@@ -99,15 +99,15 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=38B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=4.00MB Threads=3
-Per-Host Resource Estimates: Memory=90MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
+Per-Host Resource Estimates: Memory=190MB
 Analyzed query: SELECT l_comment FROM tpch_parquet.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.12MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.12MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.12MB mem-reservation=0B thread-reservation=0
@@ -126,15 +126,15 @@ Per-Host Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reservati
    tuple-ids=0 row-size=38B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
-Per-Host Resource Estimates: Memory=90MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
+Per-Host Resource Estimates: Memory=190MB
 Analyzed query: SELECT l_comment FROM tpch_parquet.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.12MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.12MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.12MB mem-reservation=0B thread-reservation=0
@@ -156,17 +156,17 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reser
 # Small parquet files - memory reservation is reduced because of small file size.
 select string_col from functional_parquet.alltypes;
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=16.00KB Threads=2
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.02MB Threads=2
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT string_col FROM functional_parquet.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=4.02MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: string_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
    HDFS partitions=24/24 files=24 size=201.80KB
@@ -179,17 +179,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=16.00KB Threads=3
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.02MB Threads=3
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT string_col FROM functional_parquet.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=48.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: string_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=48.00KB mem-reservation=0B thread-reservation=0
@@ -209,17 +209,17 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservat
    tuple-ids=0 row-size=12B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=32.00KB Threads=3
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=4.03MB Threads=3
+Per-Host Resource Estimates: Memory=42MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT string_col FROM functional_parquet.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.09MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: string_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=96.00KB mem-reservation=0B thread-reservation=0
@@ -243,18 +243,18 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-rese
 # small file size but a minimum amount is reserved per column.
 select int_col, float_col, string_col from functional_parquet.alltypes;
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=24.00KB Threads=2
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.02MB Threads=2
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT int_col, float_col, string_col FROM
 functional_parquet.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=4.02MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: int_col, float_col, string_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
    HDFS partitions=24/24 files=24 size=201.80KB
@@ -267,18 +267,18 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=24.00KB Threads=3
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.02MB Threads=3
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT int_col, float_col, string_col FROM
 functional_parquet.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=71.99KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: int_col, float_col, string_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=71.99KB mem-reservation=0B thread-reservation=0
@@ -298,18 +298,18 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservat
    tuple-ids=0 row-size=20B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=48.00KB Threads=3
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=4.05MB Threads=3
+Per-Host Resource Estimates: Memory=42MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT int_col, float_col, string_col FROM
 functional_parquet.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=143.99KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.14MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: int_col, float_col, string_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=143.99KB mem-reservation=0B thread-reservation=0
@@ -333,17 +333,17 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-rese
 # row count.
 select 'foo' from functional_parquet.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=16.00KB Threads=2
-Per-Host Resource Estimates: Memory=10MB
+Max Per-Host Resource Reservation: Memory=4.02MB Threads=2
+Per-Host Resource Estimates: Memory=11MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT 'foo' FROM functional_parquet.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=1.00MB mem-reservation=16.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=11.00MB mem-reservation=4.02MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: 'foo'
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
    HDFS partitions=24/24 files=24 size=201.80KB
@@ -356,17 +356,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=0B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=16.00KB Threads=3
-Per-Host Resource Estimates: Memory=10MB
+Max Per-Host Resource Reservation: Memory=4.02MB Threads=3
+Per-Host Resource Estimates: Memory=11MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT 'foo' FROM functional_parquet.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: 'foo'
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -386,17 +386,17 @@ Per-Host Resources: mem-estimate=1.00MB mem-reservation=16.00KB thread-reservati
    tuple-ids=0 row-size=0B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=32.00KB Threads=3
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=4.03MB Threads=3
+Per-Host Resource Estimates: Memory=42MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
 Analyzed query: SELECT 'foo' FROM functional_parquet.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=24.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: 'foo'
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=24.00KB mem-reservation=0B thread-reservation=0
@@ -420,15 +420,15 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-rese
 # reservation for column with low NDV.
 select c_nationkey from tpch_parquet.customer
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=128.00KB Threads=2
-Per-Host Resource Estimates: Memory=24MB
+Max Per-Host Resource Reservation: Memory=4.12MB Threads=2
+Per-Host Resource Estimates: Memory=28MB
 Analyzed query: SELECT c_nationkey FROM tpch_parquet.customer
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=24.00MB mem-reservation=128.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=28.00MB mem-reservation=4.12MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_nationkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_parquet.customer]
    HDFS partitions=1/1 files=1 size=12.34MB
@@ -444,15 +444,15 @@ PLAN-ROOT SINK
 # for fixed-width column with high NDV.
 select c_custkey from tpch_parquet.customer
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=2.00MB Threads=2
-Per-Host Resource Estimates: Memory=24MB
+Max Per-Host Resource Reservation: Memory=6.00MB Threads=2
+Per-Host Resource Estimates: Memory=28MB
 Analyzed query: SELECT c_custkey FROM tpch_parquet.customer
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=28.00MB mem-reservation=6.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_custkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_parquet.customer]
    HDFS partitions=1/1 files=1 size=12.34MB
@@ -469,17 +469,17 @@ PLAN-ROOT SINK
 # the column size to significantly smaller than the file size.
 select c_mktsegment from functional_parquet.customer_multiblock
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=512.00KB Threads=2
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.50MB Threads=2
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.customer_multiblock
 Analyzed query: SELECT c_mktsegment FROM functional_parquet.customer_multiblock
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=4.50MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_mktsegment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.customer_multiblock]
    HDFS partitions=1/1 files=1 size=482.93KB
@@ -495,16 +495,16 @@ PLAN-ROOT SINK
 select o_orderkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk
 from tpch_nested_parquet.customer.c_orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=24.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=28.00MB Threads=2
+Per-Host Resource Estimates: Memory=180MB
 Analyzed query: SELECT o_orderkey, o_orderstatus, o_totalprice, o_orderdate,
 o_orderpriority, o_clerk FROM tpch_nested_parquet.customer.c_orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=24.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=179.55MB mem-reservation=28.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: o_orderkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=91.55MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
    HDFS partitions=1/1 files=4 size=288.99MB
@@ -521,16 +521,16 @@ PLAN-ROOT SINK
 select o_orderkey, pos
 from tpch_nested_parquet.customer.c_orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
+Per-Host Resource Estimates: Memory=111MB
 Analyzed query: SELECT o_orderkey, pos FROM
 tpch_nested_parquet.customer.c_orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=4.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=110.89MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: o_orderkey, pos
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=22.89MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
    HDFS partitions=1/1 files=4 size=288.99MB
@@ -547,15 +547,15 @@ PLAN-ROOT SINK
 select pos
 from tpch_nested_parquet.customer.c_orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
+Per-Host Resource Estimates: Memory=99MB
 Analyzed query: SELECT pos FROM tpch_nested_parquet.customer.c_orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=4.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=99.44MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: pos
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=11.44MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
    HDFS partitions=1/1 files=4 size=288.99MB
@@ -573,17 +573,17 @@ select c_custkey, o_orderkey, o_orderstatus, o_totalprice, o_orderdate, o_orderp
     o_clerk
 from tpch_nested_parquet.customer c, c.c_orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=32.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=36.00MB Threads=2
+Per-Host Resource Estimates: Memory=188MB
 Analyzed query: SELECT c_custkey, o_orderkey, o_orderstatus, o_totalprice,
 o_orderdate, o_orderpriority, o_clerk FROM tpch_nested_parquet.customer c,
 c.c_orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=32.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=188.00MB mem-reservation=36.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_custkey, o_orderkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -623,16 +623,16 @@ PLAN-ROOT SINK
 select c_custkey, o_orderkey, pos
 from tpch_nested_parquet.customer c, c.c_orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=139MB
 Analyzed query: SELECT c_custkey, o_orderkey, pos FROM
 tpch_nested_parquet.customer c, c.c_orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=139.50MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_custkey, o_orderkey, pos
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=51.50MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -673,16 +673,16 @@ PLAN-ROOT SINK
 select c_custkey, pos
 from tpch_nested_parquet.customer c, c.c_orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=128MB
 Analyzed query: SELECT c_custkey, pos FROM tpch_nested_parquet.customer c,
 c.c_orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=128.05MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_custkey, pos
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=40.05MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -723,15 +723,15 @@ PLAN-ROOT SINK
 select c_custkey
 from tpch_nested_parquet.customer c, c.c_orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=117MB
 Analyzed query: SELECT c_custkey FROM tpch_nested_parquet.customer c, c.c_orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=116.61MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_custkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=28.61MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -771,16 +771,16 @@ PLAN-ROOT SINK
 select o_orderkey
 from tpch_nested_parquet.customer c, c.c_orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
+Per-Host Resource Estimates: Memory=117MB
 Analyzed query: SELECT o_orderkey FROM tpch_nested_parquet.customer c,
 c.c_orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=4.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=116.61MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: o_orderkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=28.61MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -820,16 +820,16 @@ PLAN-ROOT SINK
 select c_custkey, o_orderkey, l_comment
 from tpch_nested_parquet.customer c, c.c_orders o, o.o_lineitems
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=16.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=20.00MB Threads=2
+Per-Host Resource Estimates: Memory=188MB
 Analyzed query: SELECT c_custkey, o_orderkey, l_comment FROM
 tpch_nested_parquet.customer c, c.c_orders o, o.o_lineitems
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=16.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=188.00MB mem-reservation=20.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_custkey, o_orderkey, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -890,15 +890,15 @@ PLAN-ROOT SINK
 # Text scan
 select * from tpch.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=188MB
 Analyzed query: SELECT * FROM tpch.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=188.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch.lineitem]
    HDFS partitions=1/1 files=1 size=718.94MB
@@ -910,15 +910,15 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=99MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=3
+Per-Host Resource Estimates: Memory=199MB
 Analyzed query: SELECT * FROM tpch.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
@@ -937,15 +937,15 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=16.00MB Threads=3
-Per-Host Resource Estimates: Memory=187MB
+Max Per-Host Resource Reservation: Memory=20.00MB Threads=3
+Per-Host Resource Estimates: Memory=287MB
 Analyzed query: SELECT * FROM tpch.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=111.38MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.38MB mem-reservation=0B thread-reservation=0
@@ -967,15 +967,15 @@ Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reser
 # Single column text scan - memory reservation is same as multi-column scan.
 select l_comment from tpch.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=188MB
 Analyzed query: SELECT l_comment FROM tpch.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=188.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch.lineitem]
    HDFS partitions=1/1 files=1 size=718.94MB
@@ -987,15 +987,15 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=38B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=98MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=3
+Per-Host Resource Estimates: Memory=198MB
 Analyzed query: SELECT l_comment FROM tpch.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.12MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.12MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.12MB mem-reservation=0B thread-reservation=0
@@ -1014,15 +1014,15 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati
    tuple-ids=0 row-size=38B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=16.00MB Threads=3
-Per-Host Resource Estimates: Memory=186MB
+Max Per-Host Resource Reservation: Memory=20.00MB Threads=3
+Per-Host Resource Estimates: Memory=286MB
 Analyzed query: SELECT l_comment FROM tpch.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.25MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.25MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.25MB mem-reservation=0B thread-reservation=0
@@ -1044,16 +1044,16 @@ Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reser
 # Text scan on small files - memory reservation is reduced.
 select * from functional.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=32.00KB Threads=2
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.03MB Threads=2
+Per-Host Resource Estimates: Memory=20MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=20.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    HDFS partitions=24/24 files=24 size=478.45KB
@@ -1066,16 +1066,16 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=89B cardinality=7.30K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=32.00KB Threads=3
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.03MB Threads=3
+Per-Host Resource Estimates: Memory=20MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=490.49KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.48MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=490.49KB mem-reservation=0B thread-reservation=0
@@ -1095,16 +1095,16 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservat
    tuple-ids=0 row-size=89B cardinality=7.30K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=64.00KB Threads=3
-Per-Host Resource Estimates: Memory=33MB
+Max Per-Host Resource Reservation: Memory=4.06MB Threads=3
+Per-Host Resource Estimates: Memory=37MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=769.49KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.75MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=769.49KB mem-reservation=0B thread-reservation=0
@@ -1130,17 +1130,17 @@ select * from tpch_avro.orders
 # Hive 3 creates different number of files for this table than Hive 2.
 2
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=74MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpch_avro.orders
 Analyzed query: SELECT * FROM tpch_avro.orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=74.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_avro.orders]
    HDFS partitions=1/1 files=2 size=156.92MB
@@ -1152,17 +1152,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=88B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=3
+Per-Host Resource Estimates: Memory=74MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpch_avro.orders
 Analyzed query: SELECT * FROM tpch_avro.orders
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=183.96KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.27MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=183.96KB mem-reservation=0B thread-reservation=0
@@ -1181,17 +1181,17 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati
    tuple-ids=0 row-size=88B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=74MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpch_avro.orders
 Analyzed query: SELECT * FROM tpch_avro.orders
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=183.96KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.27MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=183.96KB mem-reservation=0B thread-reservation=0
@@ -1215,17 +1215,17 @@ select * from tpch_avro.orders
 # Hive 3 creates different number of files for this table than Hive 2.
 3
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=64MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=74MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpch_avro.orders
 Analyzed query: SELECT * FROM tpch_avro.orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=64.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=74.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_avro.orders]
    HDFS partitions=1/1 files=3 size=156.92MB
@@ -1237,17 +1237,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=88B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=64MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=3
+Per-Host Resource Estimates: Memory=74MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpch_avro.orders
 Analyzed query: SELECT * FROM tpch_avro.orders
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=275.97KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.27MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=275.97KB mem-reservation=0B thread-reservation=0
@@ -1269,17 +1269,17 @@ Per-Host Resources: mem-estimate=64.00MB mem-reservation=8.00MB thread-reservati
 # RC scan.
 select * from tpch_rc.customer
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=42MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpch_rc.customer
 Analyzed query: SELECT * FROM tpch_rc.customer
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=42.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch_rc.customer.c_custkey, tpch_rc.customer.c_name, tpch_rc.customer.c_address, tpch_rc.customer.c_nationkey, tpch_rc.customer.c_phone, tpch_rc.customer.c_acctbal, tpch_rc.customer.c_mktsegment, tpch_rc.customer.c_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_rc.customer]
    HDFS partitions=1/1 files=1 size=22.47MB
@@ -1291,17 +1291,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=78B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=3
+Per-Host Resource Estimates: Memory=42MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpch_rc.customer
 Analyzed query: SELECT * FROM tpch_rc.customer
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=81.92KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.08MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_rc.customer.c_custkey, tpch_rc.customer.c_name, tpch_rc.customer.c_address, tpch_rc.customer.c_nationkey, tpch_rc.customer.c_phone, tpch_rc.customer.c_acctbal, tpch_rc.customer.c_mktsegment, tpch_rc.customer.c_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=81.92KB mem-reservation=0B thread-reservation=0
@@ -1320,17 +1320,17 @@ Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservati
    tuple-ids=0 row-size=78B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=42MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpch_rc.customer
 Analyzed query: SELECT * FROM tpch_rc.customer
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=81.92KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_rc.customer.c_custkey, tpch_rc.customer.c_name, tpch_rc.customer.c_address, tpch_rc.customer.c_nationkey, tpch_rc.customer.c_phone, tpch_rc.customer.c_acctbal, tpch_rc.customer.c_mktsegment, tpch_rc.customer.c_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=81.92KB mem-reservation=0B thread-reservation=0
@@ -1352,17 +1352,17 @@ Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reser
 # Seq scan.
 select * from tpcds_seq_snap.web_returns
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpcds_seq_snap.web_returns
 Analyzed query: SELECT * FROM tpcds_seq_snap.web_returns
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpcds_seq_snap.web_returns.wr_returned_date_sk, tpcds_seq_snap.web_returns.wr_returned_time_sk, tpcds_seq_snap.web_returns.wr_item_sk, tpcds_seq_snap.web_returns.wr_refunded_customer_sk, tpcds_seq_snap.web_returns.wr_refunded_cdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_hdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_addr_sk, tpcds_seq_snap.web_returns.wr_returning_customer_sk, tpcds_seq_snap.web_returns.wr_returning_cdemo_sk, tpcds_seq_snap.web_returns.wr_returning [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpcds_seq_snap.web_returns]
    HDFS partitions=1/1 files=1 size=6.61MB
@@ -1374,17 +1374,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=104B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=3
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpcds_seq_snap.web_returns
 Analyzed query: SELECT * FROM tpcds_seq_snap.web_returns
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=107.90KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.11MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpcds_seq_snap.web_returns.wr_returned_date_sk, tpcds_seq_snap.web_returns.wr_returned_time_sk, tpcds_seq_snap.web_returns.wr_item_sk, tpcds_seq_snap.web_returns.wr_refunded_customer_sk, tpcds_seq_snap.web_returns.wr_refunded_cdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_hdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_addr_sk, tpcds_seq_snap.web_returns.wr_returning_customer_sk, tpcds_seq_snap.web_returns.wr_returning_cdemo_sk, tpcds_seq_snap.web_returns.wr_returning [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=107.90KB mem-reservation=0B thread-reservation=0
@@ -1403,17 +1403,17 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservati
    tuple-ids=0 row-size=104B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 tpcds_seq_snap.web_returns
 Analyzed query: SELECT * FROM tpcds_seq_snap.web_returns
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=107.90KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.11MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpcds_seq_snap.web_returns.wr_returned_date_sk, tpcds_seq_snap.web_returns.wr_returned_time_sk, tpcds_seq_snap.web_returns.wr_item_sk, tpcds_seq_snap.web_returns.wr_refunded_customer_sk, tpcds_seq_snap.web_returns.wr_refunded_cdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_hdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_addr_sk, tpcds_seq_snap.web_returns.wr_returning_customer_sk, tpcds_seq_snap.web_returns.wr_returning_cdemo_sk, tpcds_seq_snap.web_returns.wr_returning [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=107.90KB mem-reservation=0B thread-reservation=0
@@ -1438,15 +1438,15 @@ select * from tpch_orc_def.lineitem
 # Hive 3 creates different number of files for this table than Hive 2.
 2
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=40MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=124MB
 Analyzed query: SELECT * FROM tpch_orc_def.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=124.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch_orc_def.lineitem.l_orderkey, tpch_orc_def.lineitem.l_partkey, tpch_orc_def.lineitem.l_suppkey, tpch_orc_def.lineitem.l_linenumber, tpch_orc_def.lineitem.l_quantity, tpch_orc_def.lineitem.l_extendedprice, tpch_orc_def.lineitem.l_discount, tpch_orc_def.lineitem.l_tax, tpch_orc_def.lineitem.l_returnflag, tpch_orc_def.lineitem.l_linestatus, tpch_orc_def.lineitem.l_shipdate, tpch_orc_def.lineitem.l_commitdate, tpch_orc_def.lineitem.l_receiptdate, tpch_orc_def.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_orc_def.lineitem]
    HDFS partitions=1/1 files=6 size=144.66MB
@@ -1464,15 +1464,15 @@ select * from tpch_orc_def.lineitem
 # Hive 3 creates different number of files for this table than Hive 2.
 3
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=188MB
 Analyzed query: SELECT * FROM tpch_orc_def.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=188.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch_orc_def.lineitem.l_orderkey, tpch_orc_def.lineitem.l_partkey, tpch_orc_def.lineitem.l_suppkey, tpch_orc_def.lineitem.l_linenumber, tpch_orc_def.lineitem.l_quantity, tpch_orc_def.lineitem.l_extendedprice, tpch_orc_def.lineitem.l_discount, tpch_orc_def.lineitem.l_tax, tpch_orc_def.lineitem.l_returnflag, tpch_orc_def.lineitem.l_linestatus, tpch_orc_def.lineitem.l_shipdate, tpch_orc_def.lineitem.l_commitdate, tpch_orc_def.lineitem.l_receiptdate, tpch_orc_def.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_orc_def.lineitem]
    HDFS partitions=1/1 files=1 size=142.84MB
@@ -1490,15 +1490,15 @@ select l_comment from tpch_orc_def.lineitem
 # Hive 3 creates different number of files for this table than Hive 2.
 2
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=40MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=124MB
 Analyzed query: SELECT l_comment FROM tpch_orc_def.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=124.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_orc_def.lineitem]
    HDFS partitions=1/1 files=6 size=144.66MB
@@ -1515,15 +1515,15 @@ select l_comment from tpch_orc_def.lineitem
 # Hive 3 creates different number of files for this table than Hive 2.
 3
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=188MB
 Analyzed query: SELECT l_comment FROM tpch_orc_def.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=188.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [tpch_orc_def.lineitem]
    HDFS partitions=1/1 files=1 size=142.84MB
@@ -1538,17 +1538,17 @@ PLAN-ROOT SINK
 # ORC scan on small files - memory reservation is reduced.
 select * from functional_orc_def.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=88.00KB Threads=2
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.09MB Threads=2
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_orc_def.alltypes
 Analyzed query: SELECT * FROM functional_orc_def.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=4.09MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional_orc_def.alltypes.id, functional_orc_def.alltypes.bool_col, functional_orc_def.alltypes.tinyint_col, functional_orc_def.alltypes.smallint_col, functional_orc_def.alltypes.int_col, functional_orc_def.alltypes.bigint_col, functional_orc_def.alltypes.float_col, functional_orc_def.alltypes.double_col, functional_orc_def.alltypes.date_string_col, functional_orc_def.alltypes.string_col, functional_orc_def.alltypes.timestamp_col, functional_orc_def.alltypes.year, func [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional_orc_def.alltypes]
    HDFS partitions=24/24 files=24 size=42.27KB
@@ -1564,17 +1564,17 @@ PLAN-ROOT SINK
 # Mixed table format scan
 select * from functional.alltypesmixedformat
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=88.00KB Threads=2
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.09MB Threads=2
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional.alltypesmixedformat
 Analyzed query: SELECT * FROM functional.alltypesmixedformat
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=4.09MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypesmixedformat.id, functional.alltypesmixedformat.bool_col, functional.alltypesmixedformat.tinyint_col, functional.alltypesmixedformat.smallint_col, functional.alltypesmixedformat.int_col, functional.alltypesmixedformat.bigint_col, functional.alltypesmixedformat.float_col, functional.alltypesmixedformat.double_col, functional.alltypesmixedformat.date_string_col, functional.alltypesmixedformat.string_col, functional.alltypesmixedformat.timestamp_col, funct [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypesmixedformat]
    HDFS partitions=4/4 files=4 size=66.33KB
@@ -1587,17 +1587,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=80B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=88.00KB Threads=3
-Per-Host Resource Estimates: Memory=16MB
+Max Per-Host Resource Reservation: Memory=4.09MB Threads=3
+Per-Host Resource Estimates: Memory=26MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional.alltypesmixedformat
 Analyzed query: SELECT * FROM functional.alltypesmixedformat
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=251.97KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.25MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypesmixedformat.id, functional.alltypesmixedformat.bool_col, functional.alltypesmixedformat.tinyint_col, functional.alltypesmixedformat.smallint_col, functional.alltypesmixedformat.int_col, functional.alltypesmixedformat.bigint_col, functional.alltypesmixedformat.float_col, functional.alltypesmixedformat.double_col, functional.alltypesmixedformat.date_string_col, functional.alltypesmixedformat.string_col, functional.alltypesmixedformat.timestamp_col, funct [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=251.97KB mem-reservation=0B thread-reservation=0
@@ -1617,17 +1617,17 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservat
    tuple-ids=0 row-size=80B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=176.00KB Threads=3
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=4.17MB Threads=3
+Per-Host Resource Estimates: Memory=42MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional.alltypesmixedformat
 Analyzed query: SELECT * FROM functional.alltypesmixedformat
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=335.97KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.33MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypesmixedformat.id, functional.alltypesmixedformat.bool_col, functional.alltypesmixedformat.tinyint_col, functional.alltypesmixedformat.smallint_col, functional.alltypesmixedformat.int_col, functional.alltypesmixedformat.bigint_col, functional.alltypesmixedformat.float_col, functional.alltypesmixedformat.double_col, functional.alltypesmixedformat.date_string_col, functional.alltypesmixedformat.string_col, functional.alltypesmixedformat.timestamp_col, funct [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=335.97KB mem-reservation=0B thread-reservation=0
@@ -1650,17 +1650,17 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-rese
 # HBase scan on table without stats.
 select * from functional_hbase.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=1
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=1
 Per-Host Resource Estimates: Memory=10MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_hbase.alltypes
 Analyzed query: SELECT * FROM functional_hbase.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=256.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.25MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_hbase.alltypes.id, functional_hbase.alltypes.bigint_col, functional_hbase.alltypes.bool_col, functional_hbase.alltypes.date_string_col, functional_hbase.alltypes.double_col, functional_hbase.alltypes.float_col, functional_hbase.alltypes.int_col, functional_hbase.alltypes.month, functional_hbase.alltypes.smallint_col, functional_hbase.alltypes.string_col, functional_hbase.alltypes.timestamp_col, functional_hbase.alltypes.tinyint_col, functional_hbase.alltypes.year
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HBASE [functional_hbase.alltypes]
    stored statistics:
@@ -1670,17 +1670,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=80B cardinality=14.23K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=10MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_hbase.alltypes
 Analyzed query: SELECT * FROM functional_hbase.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=1.17MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=5.17MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_hbase.alltypes.id, functional_hbase.alltypes.bigint_col, functional_hbase.alltypes.bool_col, functional_hbase.alltypes.date_string_col, functional_hbase.alltypes.double_col, functional_hbase.alltypes.float_col, functional_hbase.alltypes.int_col, functional_hbase.alltypes.month, functional_hbase.alltypes.smallint_col, functional_hbase.alltypes.string_col, functional_hbase.alltypes.timestamp_col, functional_hbase.alltypes.tinyint_col, functional_hbase.alltypes.year
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=1.17MB mem-reservation=0B thread-reservation=0
@@ -1697,17 +1697,17 @@ Per-Host Resources: mem-estimate=256.00KB mem-reservation=0B thread-reservation=
    tuple-ids=0 row-size=80B cardinality=14.23K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=10MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_hbase.alltypes
 Analyzed query: SELECT * FROM functional_hbase.alltypes
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=1.17MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=5.17MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_hbase.alltypes.id, functional_hbase.alltypes.bigint_col, functional_hbase.alltypes.bool_col, functional_hbase.alltypes.date_string_col, functional_hbase.alltypes.double_col, functional_hbase.alltypes.float_col, functional_hbase.alltypes.int_col, functional_hbase.alltypes.month, functional_hbase.alltypes.smallint_col, functional_hbase.alltypes.string_col, functional_hbase.alltypes.timestamp_col, functional_hbase.alltypes.tinyint_col, functional_hbase.alltypes.year
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=1.17MB mem-reservation=0B thread-reservation=0
@@ -1727,16 +1727,16 @@ Per-Instance Resources: mem-estimate=256.00KB mem-reservation=0B thread-reservat
 # HBase scan on table with stats.
 select * from functional_hbase.alltypessmall
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=1
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=1
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional_hbase.alltypessmall
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_hbase.alltypessmall.id, functional_hbase.alltypessmall.bigint_col, functional_hbase.alltypessmall.bool_col, functional_hbase.alltypessmall.date_string_col, functional_hbase.alltypessmall.double_col, functional_hbase.alltypessmall.float_col, functional_hbase.alltypessmall.int_col, functional_hbase.alltypessmall.month, functional_hbase.alltypessmall.smallint_col, functional_hbase.alltypessmall.string_col, functional_hbase.alltypessmall.timestamp_col, functional_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
    stored statistics:
@@ -1746,16 +1746,16 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=89B cardinality=50
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional_hbase.alltypessmall
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_hbase.alltypessmall.id, functional_hbase.alltypessmall.bigint_col, functional_hbase.alltypessmall.bool_col, functional_hbase.alltypessmall.date_string_col, functional_hbase.alltypessmall.double_col, functional_hbase.alltypessmall.float_col, functional_hbase.alltypessmall.int_col, functional_hbase.alltypessmall.month, functional_hbase.alltypessmall.smallint_col, functional_hbase.alltypessmall.string_col, functional_hbase.alltypessmall.timestamp_col, functional_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -1772,16 +1772,16 @@ Per-Host Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservation=1
    tuple-ids=0 row-size=89B cardinality=50
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional_hbase.alltypessmall
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_hbase.alltypessmall.id, functional_hbase.alltypessmall.bigint_col, functional_hbase.alltypessmall.bool_col, functional_hbase.alltypessmall.date_string_col, functional_hbase.alltypessmall.double_col, functional_hbase.alltypessmall.float_col, functional_hbase.alltypessmall.int_col, functional_hbase.alltypessmall.month, functional_hbase.alltypessmall.smallint_col, functional_hbase.alltypessmall.string_col, functional_hbase.alltypessmall.timestamp_col, functional_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -1801,34 +1801,34 @@ Per-Instance Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservatio
 # Data source scan
 select * from functional.alltypes_datasource
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=1
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=1
 Per-Host Resource Estimates: Memory=1.00GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional.alltypes_datasource
 Analyzed query: SELECT * FROM functional.alltypes_datasource
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=1.00GB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes_datasource.id, functional.alltypes_datasource.bool_col, functional.alltypes_datasource.tinyint_col, functional.alltypes_datasource.smallint_col, functional.alltypes_datasource.int_col, functional.alltypes_datasource.bigint_col, functional.alltypes_datasource.float_col, functional.alltypes_datasource.double_col, functional.alltypes_datasource.timestamp_col, functional.alltypes_datasource.string_col, functional.alltypes_datasource.dec_col1, functional.a [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
    mem-estimate=1.00GB mem-reservation=0B thread-reservation=0
    tuple-ids=0 row-size=116B cardinality=5.00K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=1.00GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional.alltypes_datasource
 Analyzed query: SELECT * FROM functional.alltypes_datasource
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=686.41KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.67MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes_datasource.id, functional.alltypes_datasource.bool_col, functional.alltypes_datasource.tinyint_col, functional.alltypes_datasource.smallint_col, functional.alltypes_datasource.int_col, functional.alltypes_datasource.bigint_col, functional.alltypes_datasource.float_col, functional.alltypes_datasource.double_col, functional.alltypes_datasource.timestamp_col, functional.alltypes_datasource.string_col, functional.alltypes_datasource.dec_col1, functional.a [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=686.41KB mem-reservation=0B thread-reservation=0
@@ -1842,17 +1842,17 @@ Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B thread-reservation=1
    tuple-ids=0 row-size=116B cardinality=5.00K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=1.00GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional.alltypes_datasource
 Analyzed query: SELECT * FROM functional.alltypes_datasource
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=686.41KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.67MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes_datasource.id, functional.alltypes_datasource.bool_col, functional.alltypes_datasource.tinyint_col, functional.alltypes_datasource.smallint_col, functional.alltypes_datasource.int_col, functional.alltypes_datasource.bigint_col, functional.alltypes_datasource.float_col, functional.alltypes_datasource.double_col, functional.alltypes_datasource.timestamp_col, functional.alltypes_datasource.string_col, functional.alltypes_datasource.dec_col1, functional.a [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=686.41KB mem-reservation=0B thread-reservation=0
@@ -1871,16 +1871,16 @@ select * from tpch.lineitem
 union all
 select * from tpch.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=2
+Per-Host Resource Estimates: Memory=188MB
 Analyzed query: SELECT * FROM tpch.lineitem UNION ALL SELECT * FROM
 tpch.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=188.00MB mem-reservation=12.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:UNION
 |  pass-through-operands: all
@@ -1908,16 +1908,16 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=99MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=3
+Per-Host Resource Estimates: Memory=199MB
 Analyzed query: SELECT * FROM tpch.lineitem UNION ALL SELECT * FROM
 tpch.lineitem
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
@@ -1952,16 +1952,16 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservati
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=16.00MB Threads=3
-Per-Host Resource Estimates: Memory=187MB
+Max Per-Host Resource Reservation: Memory=20.00MB Threads=3
+Per-Host Resource Estimates: Memory=287MB
 Analyzed query: SELECT * FROM tpch.lineitem UNION ALL SELECT * FROM
 tpch.lineitem
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=111.38MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.38MB mem-reservation=0B thread-reservation=0
@@ -2010,7 +2010,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=114.00MB mem-reservation=38.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=23.86MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -2029,16 +2029,16 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=8B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=46.50MB Threads=4
-Per-Host Resource Estimates: Memory=140MB
+Max Per-Host Resource Reservation: Memory=50.50MB Threads=4
+Per-Host Resource Estimates: Memory=164MB
 Analyzed query: SELECT l_orderkey, count(*) FROM tpch_parquet.lineitem GROUP BY
 l_orderkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=8.01MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=31.87MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=23.86MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=8.01MB mem-reservation=0B thread-reservation=0
@@ -2078,16 +2078,16 @@ Per-Host Resources: mem-estimate=114.00MB mem-reservation=38.00MB thread-reserva
    tuple-ids=0 row-size=8B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=46.50MB Threads=3
-Per-Host Resource Estimates: Memory=140MB
+Max Per-Host Resource Reservation: Memory=50.50MB Threads=3
+Per-Host Resource Estimates: Memory=164MB
 Analyzed query: SELECT l_orderkey, count(*) FROM tpch_parquet.lineitem GROUP BY
 l_orderkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=8.01MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=31.87MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=23.86MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=8.01MB mem-reservation=0B thread-reservation=0
@@ -2130,15 +2130,15 @@ Per-Instance Resources: mem-estimate=114.00MB mem-reservation=38.00MB thread-res
 # Non-grouping aggregation with zero-slot parquet scan
 select count(*) from tpch_parquet.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=128.00KB Threads=2
-Per-Host Resource Estimates: Memory=11MB
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
+Per-Host Resource Estimates: Memory=14MB
 Analyzed query: SELECT count(*) FROM tpch_parquet.lineitem
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.00MB mem-reservation=128.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: sum_init_zero(tpch_parquet.lineitem.stats: num_rows)
@@ -2156,15 +2156,15 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=8B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=128.00KB Threads=3
-Per-Host Resource Estimates: Memory=21MB
+Max Per-Host Resource Reservation: Memory=4.12MB Threads=3
+Per-Host Resource Estimates: Memory=25MB
 Analyzed query: SELECT count(*) FROM tpch_parquet.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
@@ -2195,15 +2195,15 @@ Per-Host Resources: mem-estimate=11.00MB mem-reservation=128.00KB thread-reserva
    tuple-ids=0 row-size=8B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=128.00KB Threads=2
-Per-Host Resource Estimates: Memory=100MB
+Max Per-Host Resource Reservation: Memory=4.12MB Threads=2
+Per-Host Resource Estimates: Memory=104MB
 Analyzed query: SELECT count(*) FROM tpch_parquet.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
@@ -2240,14 +2240,14 @@ from tpch_parquet.lineitem
 order by l_comment
 ---- PLAN
 Max Per-Host Resource Reservation: Memory=52.00MB Threads=2
-Per-Host Resource Estimates: Memory=118MB
+Per-Host Resource Estimates: Memory=138MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem ORDER BY l_comment ASC
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=118.00MB mem-reservation=52.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=52.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SORT
 |  order by: l_comment ASC
@@ -2265,15 +2265,15 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=52.00MB Threads=3
-Per-Host Resource Estimates: Memory=149MB
+Max Per-Host Resource Reservation: Memory=56.00MB Threads=3
+Per-Host Resource Estimates: Memory=249MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem ORDER BY l_comment ASC
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=30.69MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=130.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_comment ASC
@@ -2299,15 +2299,15 @@ Per-Host Resources: mem-estimate=118.00MB mem-reservation=52.00MB thread-reserva
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=52.00MB Threads=2
-Per-Host Resource Estimates: Memory=149MB
+Max Per-Host Resource Reservation: Memory=56.00MB Threads=2
+Per-Host Resource Estimates: Memory=249MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem ORDER BY l_comment ASC
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=30.69MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=130.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_comment ASC
@@ -2348,7 +2348,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=80.02MB mem-reservation=40.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:TOP-N [LIMIT=100]
 |  order by: l_comment ASC
@@ -2366,16 +2366,16 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=40.00MB Threads=3
-Per-Host Resource Estimates: Memory=80MB
+Max Per-Host Resource Reservation: Memory=44.00MB Threads=3
+Per-Host Resource Estimates: Memory=84MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem ORDER BY l_comment ASC LIMIT
 CAST(100 AS TINYINT)
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=76.29KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_comment ASC
@@ -2402,16 +2402,16 @@ Per-Host Resources: mem-estimate=80.02MB mem-reservation=40.00MB thread-reservat
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=40.00MB Threads=2
-Per-Host Resource Estimates: Memory=80MB
+Max Per-Host Resource Reservation: Memory=44.00MB Threads=2
+Per-Host Resource Estimates: Memory=84MB
 Analyzed query: SELECT * FROM tpch_parquet.lineitem ORDER BY l_comment ASC LIMIT
 CAST(100 AS TINYINT)
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=76.29KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_comment ASC
@@ -2457,7 +2457,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=104.00MB mem-reservation=24.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_shipdate, l_quantity, l_orderkey, rank()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:SELECT
 |  predicates: rank() <= CAST(10 AS BIGINT)
@@ -2493,17 +2493,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=38B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=36.00MB Threads=4
-Per-Host Resource Estimates: Memory=117MB
+Max Per-Host Resource Reservation: Memory=40.00MB Threads=4
+Per-Host Resource Estimates: Memory=121MB
 Analyzed query: SELECT * FROM (SELECT l_shipdate, l_quantity, l_orderkey, rank()
 OVER (PARTITION BY l_shipdate ORDER BY l_orderkey DESC) rk FROM tpch.lineitem)
 dt WHERE rk <= CAST(10 AS BIGINT)
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=555.67KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.54MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_shipdate, l_quantity, l_orderkey, rank()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=555.67KB mem-reservation=0B thread-reservation=0
@@ -2561,17 +2561,17 @@ Per-Host Resources: mem-estimate=100.00MB mem-reservation=20.00MB thread-reserva
    tuple-ids=0 row-size=38B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=72.00MB Threads=5
-Per-Host Resource Estimates: Memory=234MB
+Max Per-Host Resource Reservation: Memory=76.00MB Threads=5
+Per-Host Resource Estimates: Memory=238MB
 Analyzed query: SELECT * FROM (SELECT l_shipdate, l_quantity, l_orderkey, rank()
 OVER (PARTITION BY l_shipdate ORDER BY l_orderkey DESC) rk FROM tpch.lineitem)
 dt WHERE rk <= CAST(10 AS BIGINT)
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=717.67KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.70MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_shipdate, l_quantity, l_orderkey, rank()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=717.67KB mem-reservation=0B thread-reservation=0
@@ -2634,15 +2634,15 @@ select *
 from tpch.lineitem inner join tpch.orders on l_orderkey = o_orderkey
 ---- PLAN
 Max Per-Host Resource Reservation: Memory=51.00MB Threads=3
-Per-Host Resource Estimates: Memory=446MB
+Per-Host Resource Estimates: Memory=458MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey
 = o_orderkey
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=445.94MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=457.94MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: l_orderkey = o_orderkey
@@ -2673,16 +2673,16 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=51.00MB Threads=5
-Per-Host Resource Estimates: Memory=467MB
+Max Per-Host Resource Reservation: Memory=55.00MB Threads=5
+Per-Host Resource Estimates: Memory=567MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey
 = o_orderkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=111.20MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.20MB mem-reservation=0B thread-reservation=0
@@ -2727,16 +2727,16 @@ Per-Host Resources: mem-estimate=368.29MB mem-reservation=43.00MB thread-reserva
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=94.00MB Threads=5
-Per-Host Resource Estimates: Memory=558MB
+Max Per-Host Resource Reservation: Memory=98.00MB Threads=5
+Per-Host Resource Estimates: Memory=658MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey
 = o_orderkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=12.40MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=112.40MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=12.40MB mem-reservation=0B thread-reservation=0
@@ -2795,15 +2795,15 @@ select *
 from tpch.lineitem inner join /* +shuffle */ tpch.orders on l_orderkey = o_orderkey
 ---- PLAN
 Max Per-Host Resource Reservation: Memory=51.00MB Threads=3
-Per-Host Resource Estimates: Memory=446MB
+Per-Host Resource Estimates: Memory=458MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN /* +shuffle */
 tpch.orders ON l_orderkey = o_orderkey
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=445.94MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=457.94MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: l_orderkey = o_orderkey
@@ -2834,16 +2834,16 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=52.00MB Threads=6
-Per-Host Resource Estimates: Memory=300MB
+Max Per-Host Resource Reservation: Memory=56.00MB Threads=6
+Per-Host Resource Estimates: Memory=400MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN /* +shuffle */
 tpch.orders ON l_orderkey = o_orderkey
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=111.20MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.20MB mem-reservation=0B thread-reservation=0
@@ -2895,16 +2895,16 @@ Per-Host Resources: mem-estimate=89.00MB mem-reservation=9.00MB thread-reservati
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=95.00MB Threads=8
-Per-Host Resource Estimates: Memory=412MB
+Max Per-Host Resource Reservation: Memory=99.00MB Threads=8
+Per-Host Resource Estimates: Memory=512MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN /* +shuffle */
 tpch.orders ON l_orderkey = o_orderkey
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=12.40MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=112.40MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=12.40MB mem-reservation=0B thread-reservation=0
@@ -2969,15 +2969,15 @@ Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reser
 select *
 from tpch.lineitem, tpch.orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=16.00MB Threads=3
-Per-Host Resource Estimates: Memory=420MB
+Max Per-Host Resource Reservation: Memory=20.00MB Threads=3
+Per-Host Resource Estimates: Memory=520MB
 Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=420.49MB mem-reservation=16.00MB thread-reservation=3
+|  Per-Host Resources: mem-estimate=520.49MB mem-reservation=20.00MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:NESTED LOOP JOIN [CROSS JOIN]
 |  mem-estimate=244.49MB mem-reservation=0B thread-reservation=0
@@ -3004,15 +3004,15 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=16.00MB Threads=5
-Per-Host Resource Estimates: Memory=442MB
+Max Per-Host Resource Reservation: Memory=20.00MB Threads=5
+Per-Host Resource Estimates: Memory=542MB
 Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=111.20MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.20MB mem-reservation=0B thread-reservation=0
@@ -3053,15 +3053,15 @@ Per-Host Resources: mem-estimate=342.84MB mem-reservation=8.00MB thread-reservat
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=24.00MB Threads=5
-Per-Host Resource Estimates: Memory=531MB
+Max Per-Host Resource Reservation: Memory=28.00MB Threads=5
+Per-Host Resource Estimates: Memory=631MB
 Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=12.40MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=112.40MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=12.40MB mem-reservation=0B thread-reservation=0
@@ -3112,48 +3112,48 @@ Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reser
 # Empty set node
 select * from functional.alltypes where 1 = 2
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=1
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=1
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional.alltypes WHERE FALSE
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:EMPTYSET
    mem-estimate=0B mem-reservation=0B thread-reservation=0
    tuple-ids=0
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=1
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=1
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional.alltypes WHERE FALSE
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:EMPTYSET
    mem-estimate=0B mem-reservation=0B thread-reservation=0
    tuple-ids=0
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B Threads=1
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=1
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional.alltypes WHERE FALSE
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:EMPTYSET
    mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -3164,17 +3164,17 @@ PLAN-ROOT SINK
 select max(tinyint_col) over(partition by int_col)
 from functional.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=10.03MB Threads=2
+Max Per-Host Resource Reservation: Memory=14.00MB Threads=2
 Per-Host Resource Estimates: Memory=26MB
 Codegen disabled by planner
 Analyzed query: SELECT max(tinyint_col) OVER (PARTITION BY int_col) FROM
 functional.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=10.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=14.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: max(tinyint_col)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:ANALYTIC
 |  functions: max(tinyint_col)
@@ -3200,17 +3200,17 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=5B cardinality=7.30K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=10.03MB Threads=4
-Per-Host Resource Estimates: Memory=26MB
+Max Per-Host Resource Reservation: Memory=14.03MB Threads=4
+Per-Host Resource Estimates: Memory=30MB
 Codegen disabled by planner
 Analyzed query: SELECT max(tinyint_col) OVER (PARTITION BY int_col) FROM
 functional.alltypes
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=56.26KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: max(tinyint_col)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=56.26KB mem-reservation=0B thread-reservation=0
@@ -3250,17 +3250,17 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservat
    tuple-ids=0 row-size=5B cardinality=7.30K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=20.06MB Threads=5
-Per-Host Resource Estimates: Memory=52MB
+Max Per-Host Resource Reservation: Memory=24.06MB Threads=5
+Per-Host Resource Estimates: Memory=56MB
 Codegen disabled by planner
 Analyzed query: SELECT max(tinyint_col) OVER (PARTITION BY int_col) FROM
 functional.alltypes
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=98.26KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.10MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: max(tinyint_col)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=98.26KB mem-reservation=0B thread-reservation=0
@@ -3308,16 +3308,16 @@ select *, row_number() over (order by o_totalprice) rnum_price,
 from tpch_parquet.orders
 ---- PLAN
 Max Per-Host Resource Reservation: Memory=48.00MB Threads=2
-Per-Host Resource Estimates: Memory=68MB
+Per-Host Resource Estimates: Memory=122MB
 Analyzed query: SELECT *, row_number() OVER (ORDER BY o_totalprice ASC)
 rnum_price, row_number() OVER (ORDER BY o_orderdate ASC) rnum_date, row_number()
 OVER (ORDER BY o_orderpriority ASC) rnum_priority FROM tpch_parquet.orders
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=68.00MB mem-reservation=48.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=122.00MB mem-reservation=48.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, row_number(), row_number(), row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:ANALYTIC
 |  functions: row_number()
@@ -3372,16 +3372,16 @@ PLAN-ROOT SINK
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 Max Per-Host Resource Reservation: Memory=68.00MB Threads=3
-Per-Host Resource Estimates: Memory=104MB
+Per-Host Resource Estimates: Memory=178MB
 Analyzed query: SELECT *, row_number() OVER (ORDER BY o_totalprice ASC)
 rnum_price, row_number() OVER (ORDER BY o_orderdate ASC) rnum_date, row_number()
 OVER (ORDER BY o_orderpriority ASC) rnum_priority FROM tpch_parquet.orders
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=48.34MB mem-reservation=32.00MB thread-reservation=1
+|  Per-Host Resources: mem-estimate=122.00MB mem-reservation=32.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, row_number(), row_number(), row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:ANALYTIC
 |  functions: row_number()
@@ -3444,16 +3444,16 @@ Per-Host Resources: mem-estimate=56.00MB mem-reservation=36.00MB thread-reservat
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=68.00MB Threads=2
-Per-Host Resource Estimates: Memory=104MB
+Per-Host Resource Estimates: Memory=178MB
 Analyzed query: SELECT *, row_number() OVER (ORDER BY o_totalprice ASC)
 rnum_price, row_number() OVER (ORDER BY o_orderdate ASC) rnum_date, row_number()
 OVER (ORDER BY o_orderpriority ASC) rnum_priority FROM tpch_parquet.orders
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=48.34MB mem-reservation=32.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=122.00MB mem-reservation=32.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, row_number(), row_number(), row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:ANALYTIC
 |  functions: row_number()
@@ -3528,8 +3528,8 @@ select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
 from tpch_parquet.lineitem join tpch_parquet.orders on l_orderkey = o_orderkey
 where l_shipmode = 'F'
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=78.00MB Threads=3
-Per-Host Resource Estimates: Memory=140MB
+Max Per-Host Resource Reservation: Memory=82.00MB Threads=3
+Per-Host Resource Estimates: Memory=240MB
 Analyzed query: SELECT DISTINCT l_orderkey, l_partkey, l_suppkey, l_linenumber,
 l_comment FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON
 l_orderkey = o_orderkey WHERE l_tax > CAST(10 AS DECIMAL(3,0)) UNION ALL SELECT
@@ -3540,10 +3540,10 @@ l_suppkey, l_linenumber, l_comment FROM tpch_parquet.lineitem INNER JOIN
 tpch_parquet.orders ON l_orderkey = o_orderkey WHERE l_shipmode = 'F'
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.16MB mem-reservation=78.00MB thread-reservation=3 runtime-filters-memory=3.00MB
+|  Per-Host Resources: mem-estimate=240.16MB mem-reservation=82.00MB thread-reservation=3 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:UNION
 |  pass-through-operands: 04
@@ -3653,8 +3653,8 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=74B cardinality=600.12K
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=123.75MB Threads=12
-Per-Host Resource Estimates: Memory=376MB
+Max Per-Host Resource Reservation: Memory=127.75MB Threads=12
+Per-Host Resource Estimates: Memory=476MB
 Analyzed query: SELECT DISTINCT l_orderkey, l_partkey, l_suppkey, l_linenumber,
 l_comment FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON
 l_orderkey = o_orderkey WHERE l_tax > CAST(10 AS DECIMAL(3,0)) UNION ALL SELECT
@@ -3665,10 +3665,10 @@ l_suppkey, l_linenumber, l_comment FROM tpch_parquet.lineitem INNER JOIN
 tpch_parquet.orders ON l_orderkey = o_orderkey WHERE l_shipmode = 'F'
 
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.21MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.21MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.21MB mem-reservation=0B thread-reservation=0
@@ -3826,8 +3826,8 @@ Per-Host Resources: mem-estimate=81.00MB mem-reservation=25.00MB thread-reservat
    tuple-ids=0 row-size=74B cardinality=600.12K
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=186.75MB Threads=10
-Per-Host Resource Estimates: Memory=437MB
+Max Per-Host Resource Reservation: Memory=190.75MB Threads=10
+Per-Host Resource Estimates: Memory=537MB
 Analyzed query: SELECT DISTINCT l_orderkey, l_partkey, l_suppkey, l_linenumber,
 l_comment FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON
 l_orderkey = o_orderkey WHERE l_tax > CAST(10 AS DECIMAL(3,0)) UNION ALL SELECT
@@ -3838,10 +3838,10 @@ l_suppkey, l_linenumber, l_comment FROM tpch_parquet.lineitem INNER JOIN
 tpch_parquet.orders ON l_orderkey = o_orderkey WHERE l_shipmode = 'F'
 
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.21MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.21MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.21MB mem-reservation=0B thread-reservation=0
@@ -4075,7 +4075,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=419.25MB mem-reservation=111.50MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 09:TOP-N [LIMIT=100]
 |  order by: o_totalprice DESC, o_orderdate ASC
@@ -4163,8 +4163,8 @@ PLAN-ROOT SINK
    tuple-ids=2 row-size=16B cardinality=6.00M
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=220.38MB Threads=11
-Per-Host Resource Estimates: Memory=533MB
+Max Per-Host Resource Reservation: Memory=224.38MB Threads=11
+Per-Host Resource Estimates: Memory=537MB
 Analyzed query: SELECT c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice,
 sum(l_quantity) FROM tpch.customer, tpch.orders, tpch.lineitem LEFT SEMI JOIN
 (SELECT l_orderkey FROM tpch.lineitem GROUP BY l_orderkey HAVING sum(l_quantity)
@@ -4174,10 +4174,10 @@ c_custkey, o_orderkey, o_orderdate, o_totalprice ORDER BY o_totalprice DESC,
 o_orderdate ASC LIMIT CAST(100 AS TINYINT)
 
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=31.12KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_totalprice DESC, o_orderdate ASC
@@ -4323,8 +4323,8 @@ Per-Host Resources: mem-estimate=90.00MB mem-reservation=10.00MB thread-reservat
    tuple-ids=2 row-size=16B cardinality=6.00M
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=281.88MB Threads=16
-Per-Host Resource Estimates: Memory=797MB
+Max Per-Host Resource Reservation: Memory=285.88MB Threads=16
+Per-Host Resource Estimates: Memory=801MB
 Analyzed query: SELECT c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice,
 sum(l_quantity) FROM tpch.customer, tpch.orders, tpch.lineitem LEFT SEMI JOIN
 (SELECT l_orderkey FROM tpch.lineitem GROUP BY l_orderkey HAVING sum(l_quantity)
@@ -4334,10 +4334,10 @@ c_custkey, o_orderkey, o_orderdate, o_totalprice ORDER BY o_totalprice DESC,
 o_orderdate ASC LIMIT CAST(100 AS TINYINT)
 
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=59.24KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_totalprice DESC, o_orderdate ASC
@@ -4799,7 +4799,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=345.94MB mem-reservation=85.94MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c_name, v.o_orderkey, v.o_orderstatus
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=71.53MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 09:AGGREGATE [FINALIZE]
 |  group by: c_name, o1.o_orderkey, o2.o_orderstatus
@@ -4864,18 +4864,18 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=54B cardinality=150.00K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=119.94MB Threads=4
-Per-Host Resource Estimates: Memory=494MB
+Max Per-Host Resource Reservation: Memory=123.94MB Threads=4
+Per-Host Resource Estimates: Memory=566MB
 Analyzed query: SELECT DISTINCT c_name, v.o_orderkey, v.o_orderstatus FROM
 tpch_nested_parquet.customer c, (SELECT DISTINCT o1.o_orderkey, o2.o_orderstatus
 FROM c.c_orders o1 INNER JOIN c.c_orders o2 ON o1.o_orderkey = o2.o_orderkey
 ORDER BY o1.o_orderkey ASC LIMIT CAST(100 AS TINYINT)) v
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.16MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=81.68MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_name, v.o_orderkey, v.o_orderstatus
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=71.53MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 12:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.16MB mem-reservation=0B thread-reservation=0
@@ -4960,18 +4960,18 @@ Per-Host Resources: mem-estimate=345.94MB mem-reservation=85.94MB thread-reserva
    tuple-ids=0 row-size=54B cardinality=150.00K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=239.88MB Threads=5
-Per-Host Resource Estimates: Memory=979MB
+Max Per-Host Resource Reservation: Memory=243.88MB Threads=5
+Per-Host Resource Estimates: Memory=1.03GB
 Analyzed query: SELECT DISTINCT c_name, v.o_orderkey, v.o_orderstatus FROM
 tpch_nested_parquet.customer c, (SELECT DISTINCT o1.o_orderkey, o2.o_orderstatus
 FROM c.c_orders o1 INNER JOIN c.c_orders o2 ON o1.o_orderkey = o2.o_orderkey
 ORDER BY o1.o_orderkey ASC LIMIT CAST(100 AS TINYINT)) v
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.21MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=81.74MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_name, v.o_orderkey, v.o_orderstatus
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=71.53MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 12:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.21MB mem-reservation=0B thread-reservation=0
@@ -5065,18 +5065,18 @@ from tpch_nested_parquet.customer c,
     row_number() over (order by o_orderpriority) rnum_priority
    from c.c_orders) v;
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=104.00MB Threads=2
-Per-Host Resource Estimates: Memory=136MB
+Max Per-Host Resource Reservation: Memory=108.00MB Threads=2
+Per-Host Resource Estimates: Memory=236MB
 Analyzed query: SELECT * FROM tpch_nested_parquet.customer c, (SELECT *,
 row_number() OVER (ORDER BY o_totalprice ASC) rnum_price, row_number() OVER
 (ORDER BY o_orderdate ASC) rnum_date, row_number() OVER (ORDER BY
 o_orderpriority ASC) rnum_priority FROM c.c_orders) v
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=136.00MB mem-reservation=104.00MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=236.00MB mem-reservation=108.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, item.o_orderkey, item.o_orderstatus, item.o_totalprice, item.o_orderdate, item.o_orderpriority, item.o_clerk, item.o_shippriority, item.o_comment, row_number(), row_number(), row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -5152,18 +5152,18 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=230B cardinality=150.00K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=104.00MB Threads=3
-Per-Host Resource Estimates: Memory=147MB
+Max Per-Host Resource Reservation: Memory=108.00MB Threads=3
+Per-Host Resource Estimates: Memory=247MB
 Analyzed query: SELECT * FROM tpch_nested_parquet.customer c, (SELECT *,
 row_number() OVER (ORDER BY o_totalprice ASC) rnum_price, row_number() OVER
 (ORDER BY o_orderdate ASC) rnum_date, row_number() OVER (ORDER BY
 o_orderpriority ASC) rnum_priority FROM c.c_orders) v
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.11MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=111.11MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, item.o_orderkey, item.o_orderstatus, item.o_totalprice, item.o_orderdate, item.o_orderpriority, item.o_clerk, item.o_shippriority, item.o_comment, row_number(), row_number(), row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.11MB mem-reservation=0B thread-reservation=0
@@ -5246,18 +5246,18 @@ Per-Host Resources: mem-estimate=136.00MB mem-reservation=104.00MB thread-reserv
    tuple-ids=0 row-size=230B cardinality=150.00K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=208.00MB Threads=3
-Per-Host Resource Estimates: Memory=283MB
+Max Per-Host Resource Reservation: Memory=212.00MB Threads=3
+Per-Host Resource Estimates: Memory=383MB
 Analyzed query: SELECT * FROM tpch_nested_parquet.customer c, (SELECT *,
 row_number() OVER (ORDER BY o_totalprice ASC) rnum_price, row_number() OVER
 (ORDER BY o_orderdate ASC) rnum_date, row_number() OVER (ORDER BY
 o_orderpriority ASC) rnum_priority FROM c.c_orders) v
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=11.47MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=111.47MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, item.o_orderkey, item.o_orderstatus, item.o_totalprice, item.o_orderdate, item.o_orderpriority, item.o_clerk, item.o_shippriority, item.o_comment, row_number(), row_number(), row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.47MB mem-reservation=0B thread-reservation=0
@@ -5355,7 +5355,7 @@ join (
 ) v1 on v1.k3 = t1.o_orderkey
 ---- PLAN
 Max Per-Host Resource Reservation: Memory=99.00MB Threads=5
-Per-Host Resource Estimates: Memory=180MB
+Per-Host Resource Estimates: Memory=181MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders t1 INNER
 JOIN (SELECT /* +straight_join */ t2.o_orderkey k2, k3, k4 FROM
 tpch_parquet.orders t2 INNER JOIN (SELECT /* +straight_join */ t3.o_orderkey k3,
@@ -5364,10 +5364,10 @@ ON t3.o_orderkey = t4.o_orderkey) v2 ON v2.k3 = t2.o_orderkey) v1 ON v1.k3 =
 t1.o_orderkey
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=180.00MB mem-reservation=99.00MB thread-reservation=5 runtime-filters-memory=3.00MB
+|  Per-Host Resources: mem-estimate=180.77MB mem-reservation=99.00MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: t1.o_orderkey, t1.o_custkey, t1.o_orderstatus, t1.o_totalprice, t1.o_orderdate, t1.o_orderpriority, t1.o_clerk, t1.o_shippriority, t1.o_comment, t2.o_orderkey, t3.o_orderkey, t4.o_orderkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.o_orderkey = t3.o_orderkey
@@ -5436,8 +5436,8 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=171B cardinality=1.50M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=100.50MB Threads=10
-Per-Host Resource Estimates: Memory=260MB
+Max Per-Host Resource Reservation: Memory=104.50MB Threads=10
+Per-Host Resource Estimates: Memory=360MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders t1 INNER
 JOIN (SELECT /* +straight_join */ t2.o_orderkey k2, k3, k4 FROM
 tpch_parquet.orders t2 INNER JOIN (SELECT /* +straight_join */ t3.o_orderkey k3,
@@ -5446,10 +5446,10 @@ ON t3.o_orderkey = t4.o_orderkey) v2 ON v2.k3 = t2.o_orderkey) v1 ON v1.k3 =
 t1.o_orderkey
 
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.41MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.41MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: t1.o_orderkey, t1.o_custkey, t1.o_orderstatus, t1.o_totalprice, t1.o_orderdate, t1.o_orderpriority, t1.o_clerk, t1.o_shippriority, t1.o_comment, t2.o_orderkey, t3.o_orderkey, t4.o_orderkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.41MB mem-reservation=0B thread-reservation=0
@@ -5553,8 +5553,8 @@ Per-Host Resources: mem-estimate=88.84MB mem-reservation=59.00MB thread-reservat
    tuple-ids=0 row-size=171B cardinality=1.50M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=135.50MB Threads=9
-Per-Host Resource Estimates: Memory=297MB
+Max Per-Host Resource Reservation: Memory=139.50MB Threads=9
+Per-Host Resource Estimates: Memory=397MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders t1 INNER
 JOIN (SELECT /* +straight_join */ t2.o_orderkey k2, k3, k4 FROM
 tpch_parquet.orders t2 INNER JOIN (SELECT /* +straight_join */ t3.o_orderkey k3,
@@ -5563,10 +5563,10 @@ ON t3.o_orderkey = t4.o_orderkey) v2 ON v2.k3 = t2.o_orderkey) v1 ON v1.k3 =
 t1.o_orderkey
 
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.41MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.41MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: t1.o_orderkey, t1.o_custkey, t1.o_orderstatus, t1.o_totalprice, t1.o_orderdate, t1.o_orderpriority, t1.o_clerk, t1.o_shippriority, t1.o_comment, t2.o_orderkey, t3.o_orderkey, t4.o_orderkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.41MB mem-reservation=0B thread-reservation=0
@@ -5713,8 +5713,8 @@ join (
   ) v2
 ) v1
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=176.00KB Threads=5
-Per-Host Resource Estimates: Memory=138MB
+Max Per-Host Resource Reservation: Memory=4.17MB Threads=5
+Per-Host Resource Estimates: Memory=238MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.nation t1 INNER
 JOIN (SELECT /* +straight_join */ t2.n_nationkey k2, k3, k4 FROM
 tpch_parquet.nation t2 INNER JOIN (SELECT /* +straight_join */ t3.n_nationkey
@@ -5722,10 +5722,10 @@ k3, t4.s_suppkey k4 FROM tpch_parquet.nation t3 INNER JOIN tpch_parquet.supplier
 t4) v2) v1
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=137.99MB mem-reservation=176.00KB thread-reservation=5
+|  Per-Host Resources: mem-estimate=237.99MB mem-reservation=4.17MB thread-reservation=5
 PLAN-ROOT SINK
 |  output exprs: t1.n_nationkey, t1.n_name, t1.n_regionkey, t1.n_comment, t2.n_nationkey, t3.n_nationkey, t4.s_suppkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:NESTED LOOP JOIN [CROSS JOIN]
 |  mem-estimate=71.53MB mem-reservation=0B thread-reservation=0
@@ -5782,8 +5782,8 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=109B cardinality=25
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=176.00KB Threads=9
-Per-Host Resource Estimates: Memory=161MB
+Max Per-Host Resource Reservation: Memory=4.17MB Threads=9
+Per-Host Resource Estimates: Memory=261MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.nation t1 INNER
 JOIN (SELECT /* +straight_join */ t2.n_nationkey k2, k3, k4 FROM
 tpch_parquet.nation t2 INNER JOIN (SELECT /* +straight_join */ t3.n_nationkey
@@ -5791,10 +5791,10 @@ k3, t4.s_suppkey k4 FROM tpch_parquet.nation t3 INNER JOIN tpch_parquet.supplier
 t4) v2) v1
 
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.13MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.13MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: t1.n_nationkey, t1.n_name, t1.n_regionkey, t1.n_comment, t2.n_nationkey, t3.n_nationkey, t4.s_suppkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.13MB mem-reservation=0B thread-reservation=0
@@ -5879,8 +5879,8 @@ Per-Host Resources: mem-estimate=97.55MB mem-reservation=32.00KB thread-reservat
    tuple-ids=0 row-size=109B cardinality=25
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=176.00KB Threads=8
-Per-Host Resource Estimates: Memory=161MB
+Max Per-Host Resource Reservation: Memory=4.17MB Threads=8
+Per-Host Resource Estimates: Memory=261MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.nation t1 INNER
 JOIN (SELECT /* +straight_join */ t2.n_nationkey k2, k3, k4 FROM
 tpch_parquet.nation t2 INNER JOIN (SELECT /* +straight_join */ t3.n_nationkey
@@ -5888,10 +5888,10 @@ k3, t4.s_suppkey k4 FROM tpch_parquet.nation t3 INNER JOIN tpch_parquet.supplier
 t4) v2) v1
 
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.13MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.13MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: t1.n_nationkey, t1.n_name, t1.n_regionkey, t1.n_comment, t2.n_nationkey, t3.n_nationkey, t4.s_suppkey
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.13MB mem-reservation=0B thread-reservation=0
@@ -6006,7 +6006,7 @@ sum(smallint_col) over (partition by tinyint_col order by smallint_col
                                                 rows between 1 following and 2 following)
                                                 from functional.alltypesagg
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=18.12MB Threads=2
+Max Per-Host Resource Reservation: Memory=22.00MB Threads=2
 Per-Host Resource Estimates: Memory=34MB
 Codegen disabled by planner
 Analyzed query: SELECT tinyint_col, smallint_col, sum(smallint_col) OVER
@@ -6017,10 +6017,10 @@ ORDER BY smallint_col ASC ROWS BETWEEN 1 FOLLOWING AND 2 FOLLOWING) FROM
 functional.alltypesagg
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=34.00MB mem-reservation=18.12MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=34.00MB mem-reservation=22.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tinyint_col, smallint_col, sum(smallint_col), sum(smallint_col), sum(smallint_col)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:ANALYTIC
 |  functions: sum(smallint_col)
@@ -6069,16 +6069,16 @@ PLAN-ROOT SINK
 # Kudu Scan
 select * from functional_kudu.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM functional_kudu.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=4.88MB mem-reservation=0B thread-reservation=2
+|  Per-Host Resources: mem-estimate=8.88MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN KUDU [functional_kudu.alltypes]
    mem-estimate=4.88MB mem-reservation=0B thread-reservation=1
@@ -6088,16 +6088,16 @@ PLAN-ROOT SINK
 # Kudu Scan of single column
 select int_col from functional_kudu.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT int_col FROM functional_kudu.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=384.00KB mem-reservation=0B thread-reservation=2
+|  Per-Host Resources: mem-estimate=4.38MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: int_col
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN KUDU [functional_kudu.alltypes]
    mem-estimate=384.00KB mem-reservation=0B thread-reservation=1
@@ -6107,16 +6107,16 @@ PLAN-ROOT SINK
 # Kudu Scan count(*)
 select count(*) from functional_kudu.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=2
-Per-Host Resource Estimates: Memory=10MB
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
+Per-Host Resource Estimates: Memory=14MB
 Codegen disabled by planner
 Analyzed query: SELECT count(*) FROM functional_kudu.alltypes
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.38MB mem-reservation=0B thread-reservation=2
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: sum_init_zero(functional_kudu.alltypes.stats: num_rows)
@@ -6132,16 +6132,16 @@ PLAN-ROOT SINK
 # Kudu Scan
 select * from tpch_kudu.nation
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B Threads=2
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
 Per-Host Resource Estimates: Memory=10MB
 Codegen disabled by planner
 Analyzed query: SELECT * FROM tpch_kudu.nation
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=1.50MB mem-reservation=0B thread-reservation=2
+|  Per-Host Resources: mem-estimate=5.50MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch_kudu.nation.n_nationkey, tpch_kudu.nation.n_name, tpch_kudu.nation.n_regionkey, tpch_kudu.nation.n_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN KUDU [tpch_kudu.nation]
    mem-estimate=1.50MB mem-reservation=0B thread-reservation=1
@@ -6163,7 +6163,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.50GB mem-reservation=74.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ship [...]
@@ -6182,16 +6182,16 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=108.00MB Threads=4
+Max Per-Host Resource Reservation: Memory=112.00MB Threads=4
 Per-Host Resource Estimates: Memory=1.51GB
 Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem LIMIT CAST(5 AS
 TINYINT)
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  limit: 5
@@ -6302,7 +6302,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=84.00MB mem-reservation=20.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_partkey, l_quantity, l_orderkey, rk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:TOP-N [LIMIT=100]
 |  order by: l_partkey ASC, l_quantity ASC, l_orderkey ASC, rk ASC
@@ -6343,18 +6343,18 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=24B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=20.00MB Threads=4
-Per-Host Resource Estimates: Memory=84MB
+Max Per-Host Resource Reservation: Memory=24.00MB Threads=4
+Per-Host Resource Estimates: Memory=88MB
 Analyzed query: SELECT * FROM (SELECT l_partkey, l_quantity, l_orderkey, rank()
 OVER (PARTITION BY l_partkey ORDER BY l_orderkey DESC) rk FROM
 tpch_parquet.lineitem) dt WHERE rk <= CAST(100 AS BIGINT) ORDER BY l_partkey
 ASC, l_quantity ASC, l_orderkey ASC, rk ASC LIMIT CAST(100 AS TINYINT)
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_partkey, l_quantity, l_orderkey, rk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 07:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_partkey ASC, l_quantity ASC, l_orderkey ASC, rk ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test b/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
index 8094095..3735b37 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-query-options.test
@@ -673,10 +673,10 @@ ENABLED_RUNTIME_FILTER_TYPES=BLOOM
 EXPLAIN_LEVEL=2
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=13.69MB mem-reservation=2.94MB thread-reservation=3 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=15.00MB mem-reservation=5.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -712,10 +712,10 @@ ENABLED_RUNTIME_FILTER_TYPES=MIN_MAX
 EXPLAIN_LEVEL=2
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=12.69MB mem-reservation=1.94MB thread-reservation=3
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -751,10 +751,10 @@ ENABLED_RUNTIME_FILTER_TYPES=ALL
 EXPLAIN_LEVEL=2
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=13.69MB mem-reservation=2.94MB thread-reservation=3 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=15.00MB mem-reservation=5.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test b/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
index 5cb6b35..a6e56f7 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
@@ -2,10 +2,10 @@
 select * from functional.alltypes order by random()
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SORT
 |  order by: random() ASC
@@ -29,10 +29,10 @@ PLAN-ROOT SINK
 select * from functional.alltypes order by abs(id) + abs(id)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SORT
 |  order by: abs(id) + abs(id) ASC
@@ -56,10 +56,10 @@ PLAN-ROOT SINK
 select * from functional.alltypes order by tinyint_col + 1
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SORT
 |  order by: tinyint_col + 1 ASC
@@ -83,10 +83,10 @@ select * from functional.alltypes
 order by dayofweek(timestamp_col), true, id + 1, string_col = date_string_col, id = tinyint_col
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SORT
 |  order by: dayofweek(timestamp_col) ASC, TRUE ASC, id + 1 ASC, string_col = date_string_col ASC, id = tinyint_col ASC
@@ -111,10 +111,10 @@ select last_value(id) over (order by to_date(timestamp_col), bool_col is null)
 from functional.alltypes
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=16.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=20.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: last_value(id)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:ANALYTIC
 |  functions: last_value(id)
@@ -147,10 +147,10 @@ select id from functional.alltypes order by string_col like 'a.*b', id * bigint_
 regexp_replace(string_col, 'a.*b', 'c') limit 10
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=128.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:TOP-N [LIMIT=10]
 |  order by: string_col LIKE 'a.*b' ASC, id * bigint_col ASC, regexp_replace(string_col, 'a.*b', 'c') ASC
@@ -174,10 +174,10 @@ PLAN-ROOT SINK
 select * from functional.alltypes order by TestFn(double_col)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SORT
 |  order by: default.testfn(double_col) /* NATIVE UDF */ ASC
@@ -201,10 +201,10 @@ PLAN-ROOT SINK
 select concat(date_string_col, string_col) c from functional.alltypes order by c
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: concat(date_string_col, string_col)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 01:SORT
 |  order by: concat(date_string_col, string_col) ASC
@@ -235,7 +235,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=42.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id, row_number()
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 03:SORT
 |  order by: id ASC
@@ -294,7 +294,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=422.00MB mem-reservation=32.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_linenumber, l_quantity, evt_ts, rn
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=22.89MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:SORT
 |  order by: rn ASC
@@ -357,7 +357,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=422.00MB mem-reservation=32.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, lno_f, l_quantity, evt_ts, rn
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=22.89MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:SORT
 |  order by: rn ASC
@@ -419,7 +419,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=422.00MB mem-reservation=32.00MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, l_partkey, l_linenumber, l_quantity, evt_ts, rn
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=33.19MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:SORT
 |  order by: rn ASC
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 f7969aa..2bf1c74 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
@@ -3,16 +3,16 @@ select straight_join *
 from tpch_parquet.customer
     inner join tpch_parquet.nation on c_nationkey = n_nationkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=18.97MB Threads=5
-Per-Host Resource Estimates: Memory=53MB
+Max Per-Host Resource Reservation: Memory=22.97MB Threads=5
+Per-Host Resource Estimates: Memory=100MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.customer INNER
 JOIN tpch_parquet.nation ON c_nationkey = n_nationkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.33MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=57.09MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.customer.c_acctbal, tpch_parquet.customer.c_mktsegment, tpch_parquet.customer.c_comment, tpch_parquet.nation.n_nationkey, tpch_parquet.nation.n_name, tpch_parquet.nation.n_regionkey, tpch_parquet.nation.n_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=46.77MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.33MB mem-reservation=0B thread-reservation=0
@@ -57,16 +57,16 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=18.94MB thread-reservat
    tuple-ids=0 row-size=218B cardinality=150.00K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=21.91MB Threads=4
-Per-Host Resource Estimates: Memory=56MB
+Max Per-Host Resource Reservation: Memory=25.91MB Threads=4
+Per-Host Resource Estimates: Memory=103MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.customer INNER
 JOIN tpch_parquet.nation ON c_nationkey = n_nationkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.33MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=57.09MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.customer.c_acctbal, tpch_parquet.customer.c_mktsegment, tpch_parquet.customer.c_comment, tpch_parquet.nation.n_nationkey, tpch_parquet.nation.n_name, tpch_parquet.nation.n_regionkey, tpch_parquet.nation.n_comment
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=46.77MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.33MB mem-reservation=0B thread-reservation=0
@@ -125,16 +125,16 @@ select straight_join *
 from tpch_parquet.lineitem
     left join tpch_parquet.orders on l_orderkey = o_orderkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=98.00MB Threads=5
-Per-Host Resource Estimates: Memory=410MB
+Max Per-Host Resource Reservation: Memory=102.00MB Threads=5
+Per-Host Resource Estimates: Memory=510MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.lineitem LEFT
 OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=111.20MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.20MB mem-reservation=0B thread-reservation=0
@@ -177,16 +177,16 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=74.00MB thread-reserva
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=132.00MB Threads=4
-Per-Host Resource Estimates: Memory=410MB
+Max Per-Host Resource Reservation: Memory=136.00MB Threads=4
+Per-Host Resource Estimates: Memory=510MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.lineitem LEFT
 OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=111.20MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=11.20MB mem-reservation=0B thread-reservation=0
@@ -242,16 +242,16 @@ select straight_join *
 from tpch_parquet.orders
     join /*+shuffle*/ tpch_parquet.customer on o_custkey = c_custkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=59.00MB Threads=6
-Per-Host Resource Estimates: Memory=114MB
+Max Per-Host Resource Reservation: Memory=63.00MB Threads=6
+Per-Host Resource Estimates: Memory=214MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER
 JOIN /* +shuffle */ tpch_parquet.customer ON o_custkey = c_custkey
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.77MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.77MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.orders.o_orderkey, tpch_parquet.orders.o_custkey, tpch_parquet.orders.o_orderstatus, tpch_parquet.orders.o_totalprice, tpch_parquet.orders.o_orderdate, tpch_parquet.orders.o_orderpriority, tpch_parquet.orders.o_clerk, tpch_parquet.orders.o_shippriority, tpch_parquet.orders.o_comment, tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.cu [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.77MB mem-reservation=0B thread-reservation=0
@@ -303,16 +303,16 @@ Per-Host Resources: mem-estimate=41.00MB mem-reservation=25.00MB thread-reservat
    tuple-ids=0 row-size=171B cardinality=1.50M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=59.00MB Threads=5
-Per-Host Resource Estimates: Memory=114MB
+Max Per-Host Resource Reservation: Memory=63.00MB Threads=5
+Per-Host Resource Estimates: Memory=214MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER
 JOIN /* +shuffle */ tpch_parquet.customer ON o_custkey = c_custkey
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.77MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.77MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.orders.o_orderkey, tpch_parquet.orders.o_custkey, tpch_parquet.orders.o_orderstatus, tpch_parquet.orders.o_totalprice, tpch_parquet.orders.o_orderdate, tpch_parquet.orders.o_orderpriority, tpch_parquet.orders.o_clerk, tpch_parquet.orders.o_shippriority, tpch_parquet.orders.o_comment, tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.cu [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.77MB mem-reservation=0B thread-reservation=0
@@ -378,16 +378,16 @@ select straight_join *
 from tpch_parquet.orders
     join /*+broadcast*/ tpch_parquet.customer on o_custkey = c_custkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=75.00MB Threads=5
-Per-Host Resource Estimates: Memory=120MB
+Max Per-Host Resource Reservation: Memory=79.00MB Threads=5
+Per-Host Resource Estimates: Memory=220MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER
 JOIN /* +broadcast */ tpch_parquet.customer ON o_custkey = c_custkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.77MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.77MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.orders.o_orderkey, tpch_parquet.orders.o_custkey, tpch_parquet.orders.o_orderstatus, tpch_parquet.orders.o_totalprice, tpch_parquet.orders.o_orderdate, tpch_parquet.orders.o_orderpriority, tpch_parquet.orders.o_clerk, tpch_parquet.orders.o_shippriority, tpch_parquet.orders.o_comment, tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.cu [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.77MB mem-reservation=0B thread-reservation=0
@@ -432,16 +432,16 @@ Per-Host Resources: mem-estimate=85.45MB mem-reservation=59.00MB thread-reservat
    tuple-ids=0 row-size=171B cardinality=1.50M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=110.00MB Threads=4
-Per-Host Resource Estimates: Memory=155MB
+Max Per-Host Resource Reservation: Memory=114.00MB Threads=4
+Per-Host Resource Estimates: Memory=255MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER
 JOIN /* +broadcast */ tpch_parquet.customer ON o_custkey = c_custkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.77MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.77MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.orders.o_orderkey, tpch_parquet.orders.o_custkey, tpch_parquet.orders.o_orderstatus, tpch_parquet.orders.o_totalprice, tpch_parquet.orders.o_orderdate, tpch_parquet.orders.o_orderpriority, tpch_parquet.orders.o_clerk, tpch_parquet.orders.o_shippriority, tpch_parquet.orders.o_comment, tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.cu [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.77MB mem-reservation=0B thread-reservation=0
@@ -500,18 +500,18 @@ select straight_join *
 from functional_parquet.alltypes
     left join functional_parquet.alltypestiny on alltypes.id = alltypestiny.id
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=34.17MB Threads=5
-Per-Host Resource Estimates: Memory=2.03GB
+Max Per-Host Resource Reservation: Memory=38.17MB Threads=5
+Per-Host Resource Estimates: Memory=2.04GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes, functional_parquet.alltypestiny
 Analyzed query: SELECT /* +straight_join */ * FROM functional_parquet.alltypes
 LEFT OUTER JOIN functional_parquet.alltypestiny ON alltypes.id = alltypestiny.id
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=503.95KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.49MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_parquet.alltypes.id, functional_parquet.alltypes.bool_col, functional_parquet.alltypes.tinyint_col, functional_parquet.alltypes.smallint_col, functional_parquet.alltypes.int_col, functional_parquet.alltypes.bigint_col, functional_parquet.alltypes.float_col, functional_parquet.alltypes.double_col, functional_parquet.alltypes.date_string_col, functional_parquet.alltypes.string_col, functional_parquet.alltypes.timestamp_col, functional_parquet.alltypes.year, func [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=503.95KB mem-reservation=0B thread-reservation=0
@@ -556,18 +556,18 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.09MB thread-reservati
    tuple-ids=0 row-size=80B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=68.34MB Threads=6
-Per-Host Resource Estimates: Memory=2.06GB
+Max Per-Host Resource Reservation: Memory=72.34MB Threads=6
+Per-Host Resource Estimates: Memory=2.07GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypestiny
 Analyzed query: SELECT /* +straight_join */ * FROM functional_parquet.alltypes
 LEFT OUTER JOIN functional_parquet.alltypestiny ON alltypes.id = alltypestiny.id
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=1007.95KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.98MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional_parquet.alltypes.id, functional_parquet.alltypes.bool_col, functional_parquet.alltypes.tinyint_col, functional_parquet.alltypes.smallint_col, functional_parquet.alltypes.int_col, functional_parquet.alltypes.bigint_col, functional_parquet.alltypes.float_col, functional_parquet.alltypes.double_col, functional_parquet.alltypes.date_string_col, functional_parquet.alltypes.string_col, functional_parquet.alltypes.timestamp_col, functional_parquet.alltypes.year, func [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=1007.95KB mem-reservation=0B thread-reservation=0
@@ -625,16 +625,16 @@ select c_nationkey, avg(c_acctbal)
 from tpch_parquet.customer
 group by c_nationkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=5.94MB Threads=4
-Per-Host Resource Estimates: Memory=44MB
+Max Per-Host Resource Reservation: Memory=9.94MB Threads=4
+Per-Host Resource Estimates: Memory=48MB
 Analyzed query: SELECT c_nationkey, avg(c_acctbal) FROM tpch_parquet.customer
 GROUP BY c_nationkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_nationkey, avg(c_acctbal)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -674,16 +674,16 @@ Per-Host Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reservati
    tuple-ids=0 row-size=10B cardinality=150.00K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=5.94MB Threads=3
-Per-Host Resource Estimates: Memory=44MB
+Max Per-Host Resource Reservation: Memory=9.94MB Threads=3
+Per-Host Resource Estimates: Memory=48MB
 Analyzed query: SELECT c_nationkey, avg(c_acctbal) FROM tpch_parquet.customer
 GROUP BY c_nationkey
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_nationkey, avg(c_acctbal)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -730,18 +730,18 @@ from tpch_parquet.lineitem
 group by 1, 2
 having count(*) = 1
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=82.00MB Threads=7
-Per-Host Resource Estimates: Memory=244MB
+Max Per-Host Resource Reservation: Memory=86.00MB Threads=7
+Per-Host Resource Estimates: Memory=344MB
 Analyzed query: SELECT /* +straight_join */ l_orderkey, o_orderstatus, count(*)
 FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON o_orderkey =
 l_orderkey GROUP BY l_orderkey, o_orderstatus HAVING count(*) = CAST(1 AS
 BIGINT)
 
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.10MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.10MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, o_orderstatus, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.10MB mem-reservation=0B thread-reservation=0
@@ -815,18 +815,18 @@ Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservati
    tuple-ids=0 row-size=8B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=82.00MB Threads=6
-Per-Host Resource Estimates: Memory=244MB
+Max Per-Host Resource Reservation: Memory=86.00MB Threads=6
+Per-Host Resource Estimates: Memory=344MB
 Analyzed query: SELECT /* +straight_join */ l_orderkey, o_orderstatus, count(*)
 FROM tpch_parquet.lineitem INNER JOIN tpch_parquet.orders ON o_orderkey =
 l_orderkey GROUP BY l_orderkey, o_orderstatus HAVING count(*) = CAST(1 AS
 BIGINT)
 
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.10MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.10MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, o_orderstatus, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.10MB mem-reservation=0B thread-reservation=0
@@ -913,15 +913,15 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reser
 select distinct *
 from tpch_parquet.lineitem
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=108.00MB Threads=4
-Per-Host Resource Estimates: Memory=1.52GB
+Max Per-Host Resource Reservation: Memory=112.00MB Threads=4
+Per-Host Resource Estimates: Memory=1.62GB
 Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=110.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
@@ -959,15 +959,15 @@ Per-Host Resources: mem-estimate=806.43MB mem-reservation=74.00MB thread-reserva
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=108.00MB Threads=3
-Per-Host Resource Estimates: Memory=1.52GB
+Max Per-Host Resource Reservation: Memory=112.00MB Threads=3
+Per-Host Resource Estimates: Memory=1.62GB
 Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=110.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
@@ -1010,18 +1010,18 @@ select string_col, count(*)
 from functional_parquet.alltypestiny
 group by string_col
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=68.01MB Threads=4
-Per-Host Resource Estimates: Memory=272MB
+Max Per-Host Resource Reservation: Memory=72.01MB Threads=4
+Per-Host Resource Estimates: Memory=282MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypestiny
 Analyzed query: SELECT string_col, count(*) FROM functional_parquet.alltypestiny
 GROUP BY string_col
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=71.99KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: string_col, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=71.99KB mem-reservation=0B thread-reservation=0
@@ -1062,18 +1062,18 @@ Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.01MB thread-reserva
    tuple-ids=0 row-size=12B cardinality=unavailable
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=136.02MB Threads=5
-Per-Host Resource Estimates: Memory=544MB
+Max Per-Host Resource Reservation: Memory=140.02MB Threads=5
+Per-Host Resource Estimates: Memory=554MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypestiny
 Analyzed query: SELECT string_col, count(*) FROM functional_parquet.alltypestiny
 GROUP BY string_col
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=95.99KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.09MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: string_col, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=95.99KB mem-reservation=0B thread-reservation=0
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test b/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
index 2ff75a2..39b44bd 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
@@ -2,10 +2,10 @@
 select * from functional.alltypes tablesample system(10) repeatable(1234)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=36.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    HDFS partitions=3/24 files=3 size=60.68KB
@@ -22,10 +22,10 @@ PLAN-ROOT SINK
 select * from functional.alltypes tablesample system(50) repeatable(1234)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=84.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    HDFS partitions=12/24 files=12 size=239.26KB
@@ -44,10 +44,10 @@ select * from functional.alltypes tablesample system(50) repeatable(1234)
 where id < 10
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=84.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    HDFS partitions=12/24 files=12 size=239.26KB
@@ -66,10 +66,10 @@ select * from functional.alltypes tablesample system(50) repeatable(1234)
 where year = 2009
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=52.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    partition predicates: `year` = CAST(2009 AS INT)
@@ -87,10 +87,10 @@ PLAN-ROOT SINK
 select * from functional.alltypes tablesample system(0)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    partitions=0/24 files=0 size=0B
@@ -107,10 +107,10 @@ PLAN-ROOT SINK
 select * from functional.alltypes tablesample system(1) repeatable(1234)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=36.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    HDFS partitions=1/24 files=1 size=19.95KB
@@ -128,10 +128,10 @@ select * from functional.alltypes tablesample system(1) repeatable(1234)
 where year = 2010
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=36.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    partition predicates: `year` = CAST(2010 AS INT)
@@ -149,10 +149,10 @@ PLAN-ROOT SINK
 select * from functional.alltypes tablesample system (100)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=132.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    HDFS partitions=24/24 files=24 size=478.45KB
@@ -169,10 +169,10 @@ PLAN-ROOT SINK
 select id from functional_parquet.alltypes tablesample system(10) repeatable(1234)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=20.00MB mem-reservation=4.02MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
    HDFS partitions=3/24 files=3 size=25.50KB
@@ -191,10 +191,10 @@ select id from functional.alltypes t1 where exists (
   where t1.id = t2.id)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=162.94MB mem-reservation=2.98MB thread-reservation=3 runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=162.94MB mem-reservation=6.97MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.id = t2.id
@@ -231,10 +231,10 @@ with t as (select * from functional.alltypes tablesample system(10) repeatable(1
 select id from t
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2
+|  Per-Host Resources: mem-estimate=36.00MB mem-reservation=4.03MB thread-reservation=2
 PLAN-ROOT SINK
 |  output exprs: functional.alltypes.id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    HDFS partitions=3/24 files=3 size=60.68KB
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 4cd5dc3..9a3b5c1 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q01.test
@@ -29,7 +29,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=330.81MB mem-reservation=26.58MB thread-reservation=7 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: c_customer_id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:TOP-N [LIMIT=100]
 |  order by: c_customer_id ASC
@@ -171,13 +171,13 @@ PLAN-ROOT SINK
    tuple-ids=5 row-size=32B cardinality=100.00K
    in pipelines: 05(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=45.33MB Threads=16
-Per-Host Resource Estimates: Memory=402MB
+Max Per-Host Resource Reservation: Memory=49.33MB Threads=16
+Per-Host Resource Estimates: Memory=406MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_customer_id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 26:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_customer_id ASC
@@ -405,13 +405,13 @@ Per-Host Resources: mem-estimate=43.04MB mem-reservation=11.75MB thread-reservat
    tuple-ids=5 row-size=32B cardinality=100.00K
    in pipelines: 05(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=58.95MB Threads=15
-Per-Host Resource Estimates: Memory=208MB
+Max Per-Host Resource Reservation: Memory=62.95MB Threads=15
+Per-Host Resource Estimates: Memory=212MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_customer_id
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 26:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_customer_id ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test
index d71433e..0a196c6 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q02.test
@@ -85,7 +85,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=366.62MB mem-reservation=28.88MB thread-reservation=7 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: d_week_seq1, round(sun_sales1 / sun_sales2, CAST(2 AS TINYINT)), round(mon_sales1 / mon_sales2, CAST(2 AS TINYINT)), round(tue_sales1 / tue_sales2, CAST(2 AS TINYINT)), round(wed_sales1 / wed_sales2, CAST(2 AS TINYINT)), round(thu_sales1 / thu_sales2, CAST(2 AS TINYINT)), round(fri_sales1 / fri_sales2, CAST(2 AS TINYINT)), round(sat_sales1 / sat_sales2, CAST(2 AS TINYINT))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:SORT
 |  order by: d_week_seq1 ASC
@@ -252,13 +252,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=8B cardinality=719.38K
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=45.94MB Threads=15
-Per-Host Resource Estimates: Memory=424MB
+Max Per-Host Resource Reservation: Memory=49.94MB Threads=15
+Per-Host Resource Estimates: Memory=428MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=886.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.87MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: d_week_seq1, round(sun_sales1 / sun_sales2, CAST(2 AS TINYINT)), round(mon_sales1 / mon_sales2, CAST(2 AS TINYINT)), round(tue_sales1 / tue_sales2, CAST(2 AS TINYINT)), round(wed_sales1 / wed_sales2, CAST(2 AS TINYINT)), round(thu_sales1 / thu_sales2, CAST(2 AS TINYINT)), round(fri_sales1 / fri_sales2, CAST(2 AS TINYINT)), round(sat_sales1 / sat_sales2, CAST(2 AS TINYINT))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: d_week_seq1 ASC
@@ -496,13 +496,13 @@ Per-Host Resources: mem-estimate=111.80MB mem-reservation=10.88MB thread-reserva
    tuple-ids=0 row-size=8B cardinality=719.38K
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=53.69MB Threads=14
-Per-Host Resource Estimates: Memory=240MB
+Max Per-Host Resource Reservation: Memory=57.69MB Threads=14
+Per-Host Resource Estimates: Memory=244MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=886.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.87MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: d_week_seq1, round(sun_sales1 / sun_sales2, CAST(2 AS TINYINT)), round(mon_sales1 / mon_sales2, CAST(2 AS TINYINT)), round(tue_sales1 / tue_sales2, CAST(2 AS TINYINT)), round(wed_sales1 / wed_sales2, CAST(2 AS TINYINT)), round(thu_sales1 / thu_sales2, CAST(2 AS TINYINT)), round(fri_sales1 / fri_sales2, CAST(2 AS TINYINT)), round(sat_sales1 / sat_sales2, CAST(2 AS TINYINT))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: d_week_seq1 ASC
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 97ccf42..10d437f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q03.test
@@ -29,7 +29,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=149.88MB mem-reservation=8.81MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_brand_id, item.i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:TOP-N [LIMIT=100]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
@@ -98,13 +98,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=12.56MB Threads=9
-Per-Host Resource Estimates: Memory=171MB
+Max Per-Host Resource Reservation: Memory=16.56MB Threads=9
+Per-Host Resource Estimates: Memory=175MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=18.16KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_brand_id, item.i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 12:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
@@ -217,13 +217,13 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati
    tuple-ids=1 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=23.38MB Threads=12
-Per-Host Resource Estimates: Memory=119MB
+Max Per-Host Resource Reservation: Memory=27.38MB Threads=12
+Per-Host Resource Estimates: Memory=123MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=34.63KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_brand_id, item.i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 12:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
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 0be2431..127e80b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q04.test
@@ -125,7 +125,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=2.12GB mem-reservation=377.94MB thread-reservation=19 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 47:TOP-N [LIMIT=100]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
@@ -558,13 +558,13 @@ PLAN-ROOT SINK
    tuple-ids=15 row-size=24B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=652.69MB Threads=49
-Per-Host Resource Estimates: Memory=2.78GB
+Max Per-Host Resource Reservation: Memory=656.69MB Threads=49
+Per-Host Resource Estimates: Memory=2.79GB
 F36:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=26.23KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 83:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
@@ -1250,13 +1250,13 @@ Per-Host Resources: mem-estimate=82.95MB mem-reservation=6.94MB thread-reservati
    tuple-ids=15 row-size=24B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=851.38MB Threads=56
+Max Per-Host Resource Reservation: Memory=855.38MB Threads=56
 Per-Host Resource Estimates: Memory=1.33GB
 F36:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=49.95KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 83:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
index 78f0f4f..a9255e7 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q05.test
@@ -131,7 +131,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=297.88MB mem-reservation=31.38MB thread-reservation=5 runtime-filters-memory=8.00MB
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(27,28,29) WHEN 27 THEN channel WHEN 28 THEN channel WHEN 29 THEN NULL END, CASE valid_tid(27,28,29) WHEN 27 THEN id WHEN 28 THEN NULL WHEN 29 THEN NULL END, aggif(valid_tid(27,28,29) IN (27, 28, 29), CASE valid_tid(27,28,29) WHEN 27 THEN sum(sales) WHEN 28 THEN sum(sales) WHEN 29 THEN sum(sales) END), aggif(valid_tid(27,28,29) IN (27, 28, 29), CASE valid_tid(27,28,29) WHEN 27 THEN sum(`returns`) WHEN 28 THEN sum(`returns`) WHEN 29 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 29:TOP-N [LIMIT=100]
 |  order by: CASE valid_tid(27,28,29) WHEN 27 THEN channel WHEN 28 THEN channel WHEN 29 THEN NULL END ASC, CASE valid_tid(27,28,29) WHEN 27 THEN id WHEN 28 THEN NULL WHEN 29 THEN NULL END ASC
@@ -404,13 +404,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=77.53MB Threads=25
-Per-Host Resource Estimates: Memory=919MB
+Max Per-Host Resource Reservation: Memory=81.53MB Threads=25
+Per-Host Resource Estimates: Memory=923MB
 F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=24.61KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(27,28,29) WHEN 27 THEN channel WHEN 28 THEN channel WHEN 29 THEN NULL END, CASE valid_tid(27,28,29) WHEN 27 THEN id WHEN 28 THEN NULL WHEN 29 THEN NULL END, aggif(valid_tid(27,28,29) IN (27, 28, 29), CASE valid_tid(27,28,29) WHEN 27 THEN sum(sales) WHEN 28 THEN sum(sales) WHEN 29 THEN sum(sales) END), aggif(valid_tid(27,28,29) IN (27, 28, 29), CASE valid_tid(27,28,29) WHEN 27 THEN sum(`returns`) WHEN 28 THEN sum(`returns`) WHEN 29 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 46:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(27,28,29) WHEN 27 THEN channel WHEN 28 THEN channel WHEN 29 THEN NULL END ASC, CASE valid_tid(27,28,29) WHEN 27 THEN id WHEN 28 THEN NULL WHEN 29 THEN NULL END ASC
@@ -811,13 +811,13 @@ Per-Host Resources: mem-estimate=112.10MB mem-reservation=11.88MB thread-reserva
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=116.84MB Threads=24
-Per-Host Resource Estimates: Memory=519MB
+Max Per-Host Resource Reservation: Memory=120.84MB Threads=24
+Per-Host Resource Estimates: Memory=523MB
 F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=46.88KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(27,28,29) WHEN 27 THEN channel WHEN 28 THEN channel WHEN 29 THEN NULL END, CASE valid_tid(27,28,29) WHEN 27 THEN id WHEN 28 THEN NULL WHEN 29 THEN NULL END, aggif(valid_tid(27,28,29) IN (27, 28, 29), CASE valid_tid(27,28,29) WHEN 27 THEN sum(sales) WHEN 28 THEN sum(sales) WHEN 29 THEN sum(sales) END), aggif(valid_tid(27,28,29) IN (27, 28, 29), CASE valid_tid(27,28,29) WHEN 27 THEN sum(`returns`) WHEN 28 THEN sum(`returns`) WHEN 29 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 46:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(27,28,29) WHEN 27 THEN channel WHEN 28 THEN channel WHEN 29 THEN NULL END ASC, CASE valid_tid(27,28,29) WHEN 27 THEN id WHEN 28 THEN NULL WHEN 29 THEN NULL END ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test
index 1c326a3..e7d5ba0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q06.test
@@ -31,7 +31,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=289.75MB mem-reservation=24.50MB thread-reservation=8 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: a.ca_state, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:TOP-N [LIMIT=100]
 |  order by: count(*) ASC
@@ -186,13 +186,13 @@ PLAN-ROOT SINK
    tuple-ids=2 row-size=16B cardinality=2.88M
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=35.81MB Threads=19
-Per-Host Resource Estimates: Memory=340MB
+Max Per-Host Resource Reservation: Memory=39.81MB Threads=19
+Per-Host Resource Estimates: Memory=344MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: a.ca_state, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) ASC
@@ -447,13 +447,13 @@ Per-Host Resources: mem-estimate=62.43MB mem-reservation=20.62MB thread-reservat
    tuple-ids=2 row-size=16B cardinality=2.88M
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=55.38MB Threads=20
-Per-Host Resource Estimates: Memory=246MB
+Max Per-Host Resource Reservation: Memory=59.38MB Threads=20
+Per-Host Resource Estimates: Memory=250MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: a.ca_state, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) ASC
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 af23a74..32254b5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q07.test
@@ -34,7 +34,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=306.31MB mem-reservation=31.34MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, avg(ss_quantity), avg(ss_list_price), avg(ss_coupon_amt), avg(ss_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC
@@ -140,13 +140,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=35.28MB Threads=12
-Per-Host Resource Estimates: Memory=332MB
+Max Per-Host Resource Reservation: Memory=39.28MB Threads=12
+Per-Host Resource Estimates: Memory=336MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=20.70KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, avg(ss_quantity), avg(ss_list_price), avg(ss_coupon_amt), avg(ss_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
@@ -303,13 +303,13 @@ Per-Host Resources: mem-estimate=145.85MB mem-reservation=24.31MB thread-reserva
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=61.53MB Threads=13
-Per-Host Resource Estimates: Memory=179MB
+Max Per-Host Resource Reservation: Memory=65.53MB Threads=13
+Per-Host Resource Estimates: Memory=183MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=39.45KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, avg(ss_quantity), avg(ss_list_price), avg(ss_coupon_amt), avg(ss_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
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 d5f918e..b995a50 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q08.test
@@ -84,7 +84,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=227.81MB mem-reservation=15.84MB thread-reservation=7 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: s_store_name, sum(ss_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:TOP-N [LIMIT=100]
 |  order by: s_store_name ASC
@@ -223,13 +223,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=29.15MB Threads=15
-Per-Host Resource Estimates: Memory=284MB
+Max Per-Host Resource Reservation: Memory=33.15MB Threads=15
+Per-Host Resource Estimates: Memory=288MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name, sum(ss_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 25:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC
@@ -446,13 +446,13 @@ Per-Host Resources: mem-estimate=50.86MB mem-reservation=11.81MB thread-reservat
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=44.84MB Threads=16
-Per-Host Resource Estimates: Memory=218MB
+Max Per-Host Resource Reservation: Memory=48.84MB Threads=16
+Per-Host Resource Estimates: Memory=222MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name, sum(ss_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 25:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test
index 30966a1..3bf7d73 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q09.test
@@ -53,7 +53,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=566.00MB mem-reservation=10.63MB thread-reservation=17
 PLAN-ROOT SINK
 |  output exprs: CASE WHEN count(*) > CAST(74129 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(122840 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(56580 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(10097 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(165306 AS BIGINT) THEN avg(ss_ext_discou [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 45:NESTED LOOP JOIN [CROSS JOIN]
 |  mem-estimate=8B mem-reservation=0B thread-reservation=0
@@ -443,13 +443,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=4B cardinality=288.04K
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=10.63MB Threads=47
-Per-Host Resource Estimates: Memory=716MB
+Max Per-Host Resource Reservation: Memory=14.63MB Threads=47
+Per-Host Resource Estimates: Memory=720MB
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.25MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.23MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE WHEN count(*) > CAST(74129 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(122840 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(56580 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(10097 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(165306 AS BIGINT) THEN avg(ss_ext_discou [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 45:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  mem-estimate=8B mem-reservation=0B thread-reservation=0
@@ -1139,13 +1139,13 @@ Per-Host Resources: mem-estimate=26.00MB mem-reservation=128.00KB thread-reserva
    tuple-ids=1 row-size=4B cardinality=288.04K
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=21.26MB Threads=61
-Per-Host Resource Estimates: Memory=946MB
+Max Per-Host Resource Reservation: Memory=25.26MB Threads=61
+Per-Host Resource Estimates: Memory=950MB
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE WHEN count(*) > CAST(74129 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(122840 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(56580 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(10097 AS BIGINT) THEN avg(ss_ext_discount_amt) ELSE avg(ss_net_profit) END, CASE WHEN count(*) > CAST(165306 AS BIGINT) THEN avg(ss_ext_discou [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 45:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  join table id: 00
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 493fd48..163587c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q10a.test
@@ -65,7 +65,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=431.69MB mem-reservation=31.69MB thread-reservation=8 runtime-filters-memory=7.00MB
 PLAN-ROOT SINK
 |  output exprs: cd_gender, cd_marital_status, cd_education_status, count(*), cd_purchase_estimate, count(*), cd_credit_rating, count(*), cd_dep_count, count(*), cd_dep_employed_count, count(*), cd_dep_college_count, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:TOP-N [LIMIT=100]
 |  order by: cd_gender ASC, cd_marital_status ASC, cd_education_status ASC, cd_purchase_estimate ASC, cd_credit_rating ASC, cd_dep_count ASC, cd_dep_employed_count ASC, cd_dep_college_count ASC
@@ -247,13 +247,13 @@ PLAN-ROOT SINK
    tuple-ids=6 row-size=8B cardinality=719.38K
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=43.00MB Threads=19
-Per-Host Resource Estimates: Memory=509MB
+Max Per-Host Resource Reservation: Memory=47.00MB Threads=19
+Per-Host Resource Estimates: Memory=513MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=30.66KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: cd_gender, cd_marital_status, cd_education_status, count(*), cd_purchase_estimate, count(*), cd_credit_rating, count(*), cd_dep_count, count(*), cd_dep_employed_count, count(*), cd_dep_college_count, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 29:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cd_gender ASC, cd_marital_status ASC, cd_education_status ASC, cd_purchase_estimate ASC, cd_credit_rating ASC, cd_dep_count ASC, cd_dep_employed_count ASC, cd_dep_college_count ASC
@@ -514,13 +514,13 @@ Per-Host Resources: mem-estimate=100.95MB mem-reservation=8.94MB thread-reservat
    tuple-ids=6 row-size=8B cardinality=719.38K
    in pipelines: 07(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=60.06MB Threads=19
-Per-Host Resource Estimates: Memory=238MB
+Max Per-Host Resource Reservation: Memory=64.06MB Threads=19
+Per-Host Resource Estimates: Memory=242MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=30.66KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: cd_gender, cd_marital_status, cd_education_status, count(*), cd_purchase_estimate, count(*), cd_credit_rating, count(*), cd_dep_count, count(*), cd_dep_employed_count, count(*), cd_dep_college_count, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 29:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cd_gender ASC, cd_marital_status ASC, cd_education_status ASC, cd_purchase_estimate ASC, cd_credit_rating ASC, cd_dep_count ASC, cd_dep_employed_count ASC, cd_dep_college_count ASC
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 7f4cba1..5da19b0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q11.test
@@ -86,7 +86,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.11GB mem-reservation=265.06MB thread-reservation=13 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 31:TOP-N [LIMIT=100]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
@@ -375,13 +375,13 @@ PLAN-ROOT SINK
    tuple-ids=10 row-size=153B cardinality=100.00K
    in pipelines: 08(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=403.94MB Threads=33
-Per-Host Resource Estimates: Memory=1.49GB
+Max Per-Host Resource Reservation: Memory=407.94MB Threads=33
+Per-Host Resource Estimates: Memory=1.50GB
 F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=26.23KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 55:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
@@ -839,13 +839,13 @@ Per-Host Resources: mem-estimate=51.95MB mem-reservation=5.94MB thread-reservati
    tuple-ids=11 row-size=16B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=572.88MB Threads=40
-Per-Host Resource Estimates: Memory=936MB
+Max Per-Host Resource Reservation: Memory=576.88MB Threads=40
+Per-Host Resource Estimates: Memory=940MB
 F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=49.95KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name, customer_preferred_cust_flag
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 55:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC, customer_first_name ASC, customer_last_name ASC, customer_preferred_cust_flag ASC
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 63546df..2dd2044 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q12.test
@@ -33,7 +33,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=233.88MB mem-reservation=52.00MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_category, i_class, i_current_price, sum(ws_ext_sales_price), sum(ws_ext_sales_price) * CAST(100.0000 AS DECIMAL(7,4)) / sum(sum(ws_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:TOP-N [LIMIT=100]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ws_ext_sales_price) * 100.0000 / sum(sum(ws_ext_sales_price)) ASC
@@ -114,13 +114,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=100.88MB Threads=8
-Per-Host Resource Estimates: Memory=315MB
+Max Per-Host Resource Reservation: Memory=104.88MB Threads=8
+Per-Host Resource Estimates: Memory=319MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=53.05KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_category, i_class, i_current_price, sum(ws_ext_sales_price), sum(ws_ext_sales_price) * CAST(100.0000 AS DECIMAL(7,4)) / sum(sum(ws_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ws_ext_sales_price) * 100.0000 / sum(sum(ws_ext_sales_price)) ASC
@@ -238,13 +238,13 @@ Per-Host Resources: mem-estimate=137.25MB mem-reservation=47.88MB thread-reserva
    tuple-ids=0 row-size=16B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=106.75MB Threads=7
-Per-Host Resource Estimates: Memory=161MB
+Max Per-Host Resource Reservation: Memory=110.75MB Threads=7
+Per-Host Resource Estimates: Memory=165MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=53.05KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_category, i_class, i_current_price, sum(ws_ext_sales_price), sum(ws_ext_sales_price) * CAST(100.0000 AS DECIMAL(7,4)) / sum(sum(ws_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ws_ext_sales_price) * 100.0000 / sum(sum(ws_ext_sales_price)) ASC
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 55d6a19..cf3f03b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q13.test
@@ -54,7 +54,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=316.75MB mem-reservation=26.70MB thread-reservation=7 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: avg(ss_quantity), avg(ss_ext_sales_price), avg(ss_ext_wholesale_cost), sum(ss_ext_wholesale_cost)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:AGGREGATE [FINALIZE]
 |  output: avg(CAST(ss_quantity AS BIGINT)), avg(ss_ext_sales_price), avg(ss_ext_wholesale_cost), sum(ss_ext_wholesale_cost)
@@ -180,13 +180,13 @@ PLAN-ROOT SINK
    tuple-ids=2 row-size=39B cardinality=181.75K
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=32.33MB Threads=14
-Per-Host Resource Estimates: Memory=344MB
+Max Per-Host Resource Reservation: Memory=36.33MB Threads=14
+Per-Host Resource Estimates: Memory=348MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: avg(ss_quantity), avg(ss_ext_sales_price), avg(ss_ext_wholesale_cost), sum(ss_ext_wholesale_cost)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 19:AGGREGATE [FINALIZE]
 |  output: avg:merge(ss_quantity), avg:merge(ss_ext_sales_price), avg:merge(ss_ext_wholesale_cost), sum:merge(ss_ext_wholesale_cost)
@@ -366,13 +366,13 @@ Per-Host Resources: mem-estimate=139.26MB mem-reservation=14.81MB thread-reserva
    tuple-ids=0 row-size=40B cardinality=288.04K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=49.08MB Threads=16
-Per-Host Resource Estimates: Memory=190MB
+Max Per-Host Resource Reservation: Memory=53.08MB Threads=16
+Per-Host Resource Estimates: Memory=194MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: avg(ss_quantity), avg(ss_ext_sales_price), avg(ss_ext_wholesale_cost), sum(ss_ext_wholesale_cost)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 19:AGGREGATE [FINALIZE]
 |  output: avg:merge(ss_quantity), avg:merge(ss_ext_sales_price), avg:merge(ss_ext_wholesale_cost), sum:merge(ss_ext_wholesale_cost)
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 a05b176..b869d51 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14a.test
@@ -107,7 +107,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.19GB mem-reservation=130.75MB thread-reservation=16 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: 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, 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, 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, CASE valid_tid(104,105,10 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 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 [...]
@@ -1295,13 +1295,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=391.06MB Threads=120
+Max Per-Host Resource Reservation: Memory=395.06MB Threads=120
 Per-Host Resource Estimates: Memory=3.66GB
 F80:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.80KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: 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, 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, 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, CASE valid_tid(104,105,10 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 216:MERGING-EXCHANGE [UNPARTITIONED]
 |  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 [...]
@@ -3098,13 +3098,13 @@ Per-Host Resources: mem-estimate=67.37MB mem-reservation=14.81MB thread-reservat
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=624.75MB Threads=141
+Max Per-Host Resource Reservation: Memory=628.75MB Threads=141
 Per-Host Resource Estimates: Memory=2.35GB
 F80:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=32.03KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: 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, 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, 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, CASE valid_tid(104,105,10 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 216:MERGING-EXCHANGE [UNPARTITIONED]
 |  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 [...]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test
index 3e96c3c..af05b6e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q14b.test
@@ -135,7 +135,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.85GB mem-reservation=99.88MB thread-reservation=33 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: channel, i_brand_id, i_class_id, i_category_id, sales, number_sales, channel, i_brand_id, i_class_id, i_category_id, sales, number_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 91:TOP-N [LIMIT=100]
 |  order by: channel ASC, i_brand_id ASC, i_class_id ASC, i_category_id ASC
@@ -966,13 +966,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=157.06MB Threads=86
+Max Per-Host Resource Reservation: Memory=161.06MB Threads=86
 Per-Host Resource Estimates: Memory=2.23GB
 F56:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: channel, i_brand_id, i_class_id, i_category_id, sales, number_sales, channel, i_brand_id, i_class_id, i_category_id, sales, number_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 152:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: channel ASC, i_brand_id ASC, i_class_id ASC, i_category_id ASC
@@ -2224,13 +2224,13 @@ Per-Host Resources: mem-estimate=70.87MB mem-reservation=14.75MB thread-reservat
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=273.38MB Threads=103
+Max Per-Host Resource Reservation: Memory=277.38MB Threads=103
 Per-Host Resource Estimates: Memory=1.42GB
 F56:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: channel, i_brand_id, i_class_id, i_category_id, sales, number_sales, channel, i_brand_id, i_class_id, i_category_id, sales, number_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 152:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: channel ASC, i_brand_id ASC, i_class_id ASC, i_category_id ASC
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 35d94f1..f8c3ecd 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q15.test
@@ -33,7 +33,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=281.75MB mem-reservation=15.75MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: ca_zip, sum(cs_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:TOP-N [LIMIT=100]
 |  order by: ca_zip ASC
@@ -118,13 +118,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=20.75MB Threads=12
-Per-Host Resource Estimates: Memory=308MB
+Max Per-Host Resource Reservation: Memory=24.75MB Threads=12
+Per-Host Resource Estimates: Memory=312MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ca_zip, sum(cs_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: ca_zip ASC
@@ -267,13 +267,13 @@ Per-Host Resources: mem-estimate=147.95MB mem-reservation=7.94MB thread-reservat
    tuple-ids=0 row-size=12B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=23.69MB Threads=11
-Per-Host Resource Estimates: Memory=135MB
+Max Per-Host Resource Reservation: Memory=27.69MB Threads=11
+Per-Host Resource Estimates: Memory=139MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ca_zip, sum(cs_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: ca_zip ASC
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 4484cce..2786e23 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q16.test
@@ -30,7 +30,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=564.69MB mem-reservation=39.44MB thread-reservation=7 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: count(cs_order_number), sum(cs_ext_ship_cost), sum(cs_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:TOP-N [LIMIT=100]
 |  order by: count(cs_order_number) ASC
@@ -160,13 +160,13 @@ PLAN-ROOT SINK
    tuple-ids=4 row-size=12B cardinality=1.44M
    in pipelines: 04(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=40.83MB Threads=14
+Max Per-Host Resource Reservation: Memory=44.83MB Threads=14
 Per-Host Resource Estimates: Memory=593MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(cs_order_number), sum(cs_ext_ship_cost), sum(cs_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:TOP-N [LIMIT=100]
 |  order by: count(cs_order_number) ASC
@@ -351,13 +351,13 @@ Per-Host Resources: mem-estimate=97.00MB mem-reservation=5.00MB thread-reservati
    tuple-ids=4 row-size=12B cardinality=1.44M
    in pipelines: 04(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=49.64MB Threads=13
+Max Per-Host Resource Reservation: Memory=53.64MB Threads=13
 Per-Host Resource Estimates: Memory=225MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(cs_order_number), sum(cs_ext_ship_cost), sum(cs_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:TOP-N [LIMIT=100]
 |  order by: count(cs_order_number) ASC
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 21b4dbd..d49d8af 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q17.test
@@ -53,7 +53,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=585.75MB mem-reservation=49.27MB thread-reservation=9 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_state, count(ss_quantity), avg(ss_quantity), stddev_samp(ss_quantity), stddev_samp(ss_quantity) / avg(ss_quantity), count(sr_return_quantity), avg(sr_return_quantity), stddev_samp(sr_return_quantity), stddev_samp(sr_return_quantity) / avg(sr_return_quantity), count(cs_quantity), avg(cs_quantity), stddev_samp(cs_quantity), stddev_samp(cs_quantity) / avg(cs_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC, i_item_desc ASC, s_state ASC
@@ -218,13 +218,13 @@ PLAN-ROOT SINK
    tuple-ids=7 row-size=148B cardinality=18.00K
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=62.77MB Threads=20
-Per-Host Resource Estimates: Memory=640MB
+Max Per-Host Resource Reservation: Memory=66.77MB Threads=20
+Per-Host Resource Estimates: Memory=644MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=74.88KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_state, count(ss_quantity), avg(ss_quantity), stddev_samp(ss_quantity), stddev_samp(ss_quantity) / avg(ss_quantity), count(sr_return_quantity), avg(sr_return_quantity), stddev_samp(sr_return_quantity), stddev_samp(sr_return_quantity) / avg(sr_return_quantity), count(cs_quantity), avg(cs_quantity), stddev_samp(cs_quantity), stddev_samp(cs_quantity) / avg(cs_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, i_item_desc ASC, s_state ASC
@@ -474,13 +474,13 @@ Per-Host Resources: mem-estimate=205.57MB mem-reservation=19.75MB thread-reserva
    tuple-ids=2 row-size=20B cardinality=1.44M
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=85.39MB Threads=22
-Per-Host Resource Estimates: Memory=285MB
+Max Per-Host Resource Reservation: Memory=89.39MB Threads=22
+Per-Host Resource Estimates: Memory=289MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=74.88KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_state, count(ss_quantity), avg(ss_quantity), stddev_samp(ss_quantity), stddev_samp(ss_quantity) / avg(ss_quantity), count(sr_return_quantity), avg(sr_return_quantity), stddev_samp(sr_return_quantity), stddev_samp(sr_return_quantity) / avg(sr_return_quantity), count(cs_quantity), avg(cs_quantity), stddev_samp(cs_quantity), stddev_samp(cs_quantity) / avg(cs_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, i_item_desc ASC, s_state ASC
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 747f434..b8e2562 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q18.test
@@ -37,7 +37,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=685.75MB mem-reservation=107.38MB thread-reservation=8 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(16,8,10,12,14) WHEN 8 THEN i_item_id WHEN 10 THEN i_item_id WHEN 12 THEN i_item_id WHEN 14 THEN i_item_id WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_country WHEN 10 THEN ca_country WHEN 12 THEN ca_country WHEN 14 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_state WHEN 10 THEN ca_state WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_county WHEN  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:TOP-N [LIMIT=100]
 |  order by: CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_country WHEN 10 THEN ca_country WHEN 12 THEN ca_country WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_state WHEN 10 THEN ca_state WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_county WHEN 10 THEN NULL WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN i_item_id WHEN 10 THEN [...]
@@ -206,13 +206,13 @@ PLAN-ROOT SINK
    tuple-ids=2 row-size=4B cardinality=1.92M
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=204.44MB Threads=18
-Per-Host Resource Estimates: Memory=895MB
+Max Per-Host Resource Reservation: Memory=208.44MB Threads=18
+Per-Host Resource Estimates: Memory=899MB
 F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=35.03KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(16,8,10,12,14) WHEN 8 THEN i_item_id WHEN 10 THEN i_item_id WHEN 12 THEN i_item_id WHEN 14 THEN i_item_id WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_country WHEN 10 THEN ca_country WHEN 12 THEN ca_country WHEN 14 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_state WHEN 10 THEN ca_state WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_county WHEN  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 26:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_country WHEN 10 THEN ca_country WHEN 12 THEN ca_country WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_state WHEN 10 THEN ca_state WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_county WHEN 10 THEN NULL WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN i_item_id WHEN 10 THEN [...]
@@ -471,13 +471,13 @@ Per-Host Resources: mem-estimate=396.33MB mem-reservation=26.81MB thread-reserva
    tuple-ids=0 row-size=40B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=214.19MB Threads=17
-Per-Host Resource Estimates: Memory=376MB
+Max Per-Host Resource Reservation: Memory=218.19MB Threads=17
+Per-Host Resource Estimates: Memory=380MB
 F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=35.03KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(16,8,10,12,14) WHEN 8 THEN i_item_id WHEN 10 THEN i_item_id WHEN 12 THEN i_item_id WHEN 14 THEN i_item_id WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_country WHEN 10 THEN ca_country WHEN 12 THEN ca_country WHEN 14 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_state WHEN 10 THEN ca_state WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_county WHEN  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 26:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_country WHEN 10 THEN ca_country WHEN 12 THEN ca_country WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_state WHEN 10 THEN ca_state WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN ca_county WHEN 10 THEN NULL WHEN 12 THEN NULL WHEN 14 THEN NULL WHEN 16 THEN NULL END ASC, CASE valid_tid(16,8,10,12,14) WHEN 8 THEN i_item_id WHEN 10 THEN [...]
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 0ee62e9..7d617d9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q19.test
@@ -41,7 +41,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=314.81MB mem-reservation=16.27MB thread-reservation=7 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, i_manufact_id, i_manufact, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 12:TOP-N [LIMIT=100]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC
@@ -167,13 +167,13 @@ PLAN-ROOT SINK
    tuple-ids=4 row-size=21B cardinality=50.00K
    in pipelines: 04(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=26.14MB Threads=16
-Per-Host Resource Estimates: Memory=345MB
+Max Per-Host Resource Reservation: Memory=30.14MB Threads=16
+Per-Host Resource Estimates: Memory=349MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=25.76KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, i_manufact_id, i_manufact, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 22:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC
@@ -370,13 +370,13 @@ Per-Host Resources: mem-estimate=71.92MB mem-reservation=9.88MB thread-reservati
    tuple-ids=1 row-size=24B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=45.77MB Threads=21
-Per-Host Resource Estimates: Memory=191MB
+Max Per-Host Resource Reservation: Memory=49.77MB Threads=21
+Per-Host Resource Estimates: Memory=195MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=49.06KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, i_manufact_id, i_manufact, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 22:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC
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 80e12a2..c809598 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q20.test
@@ -35,7 +35,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=281.88MB mem-reservation=52.00MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_category, i_class, i_current_price, sum(cs_ext_sales_price), sum(cs_ext_sales_price) * CAST(100.0000 AS DECIMAL(7,4)) / sum(sum(cs_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:TOP-N [LIMIT=100]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(cs_ext_sales_price) * 100.0000 / sum(sum(cs_ext_sales_price)) ASC
@@ -116,13 +116,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=100.88MB Threads=8
-Per-Host Resource Estimates: Memory=385MB
+Max Per-Host Resource Reservation: Memory=104.88MB Threads=8
+Per-Host Resource Estimates: Memory=389MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=70.87KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_category, i_class, i_current_price, sum(cs_ext_sales_price), sum(cs_ext_sales_price) * CAST(100.0000 AS DECIMAL(7,4)) / sum(sum(cs_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(cs_ext_sales_price) * 100.0000 / sum(sum(cs_ext_sales_price)) ASC
@@ -240,13 +240,13 @@ Per-Host Resources: mem-estimate=196.19MB mem-reservation=47.88MB thread-reserva
    tuple-ids=0 row-size=16B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=106.75MB Threads=7
-Per-Host Resource Estimates: Memory=199MB
+Max Per-Host Resource Reservation: Memory=110.75MB Threads=7
+Per-Host Resource Estimates: Memory=203MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=70.87KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_category, i_class, i_current_price, sum(cs_ext_sales_price), sum(cs_ext_sales_price) * CAST(100.0000 AS DECIMAL(7,4)) / sum(sum(cs_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(cs_ext_sales_price) * 100.0000 / sum(sum(cs_ext_sales_price)) ASC
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 de5d71d..17f1750 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q21.test
@@ -37,7 +37,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=248.81MB mem-reservation=29.56MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_name, i_item_id, inv_before, inv_after
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:TOP-N [LIMIT=100]
 |  order by: w_warehouse_name ASC, i_item_id ASC
@@ -124,13 +124,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=11.74M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=33.27MB Threads=10
-Per-Host Resource Estimates: Memory=271MB
+Max Per-Host Resource Reservation: Memory=37.27MB Threads=10
+Per-Host Resource Estimates: Memory=275MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=18.73KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_name, i_item_id, inv_before, inv_after
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_warehouse_name ASC, i_item_id ASC
@@ -261,13 +261,13 @@ Per-Host Resources: mem-estimate=147.15MB mem-reservation=29.81MB thread-reserva
    tuple-ids=0 row-size=20B cardinality=11.74M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=42.08MB Threads=9
-Per-Host Resource Estimates: Memory=119MB
+Max Per-Host Resource Reservation: Memory=46.08MB Threads=9
+Per-Host Resource Estimates: Memory=123MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=18.73KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_name, i_item_id, inv_before, inv_after
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_warehouse_name ASC, i_item_id ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test
index c357bd3..3a5637a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q22.test
@@ -25,7 +25,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=17.43GB mem-reservation=149.44MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_product_name WHEN 7 THEN i_product_name WHEN 9 THEN i_product_name WHEN 11 THEN i_product_name WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_brand WHEN 7 THEN i_brand WHEN 9 THEN i_brand WHEN 11 THEN NULL WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_class WHEN 7 THEN i_class WHEN 9 THEN NULL WHEN 11 THEN NULL WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_category WHEN 7 TH [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 09:TOP-N [LIMIT=100]
 |  order by: aggif(valid_tid(5,7,9,11,13) IN (5, 7, 9, 11, 13), CASE valid_tid(5,7,9,11,13) WHEN 5 THEN avg(inv_quantity_on_hand) WHEN 7 THEN avg(inv_quantity_on_hand) WHEN 9 THEN avg(inv_quantity_on_hand) WHEN 11 THEN avg(inv_quantity_on_hand) WHEN 13 THEN avg(inv_quantity_on_hand) END) ASC, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_product_name WHEN 7 THEN i_product_name WHEN 9 THEN i_product_name WHEN 11 THEN i_product_name WHEN 13 THEN NULL END ASC, CASE valid_tid(5,7,9,11,13) WHEN 5 [...]
@@ -128,13 +128,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=11.74M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=286.70MB Threads=10
+Max Per-Host Resource Reservation: Memory=290.70MB Threads=10
 Per-Host Resource Estimates: Memory=41.61GB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_product_name WHEN 7 THEN i_product_name WHEN 9 THEN i_product_name WHEN 11 THEN i_product_name WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_brand WHEN 7 THEN i_brand WHEN 9 THEN i_brand WHEN 11 THEN NULL WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_class WHEN 7 THEN i_class WHEN 9 THEN NULL WHEN 11 THEN NULL WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_category WHEN 7 TH [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: aggif(valid_tid(5,7,9,11,13) IN (5, 7, 9, 11, 13), CASE valid_tid(5,7,9,11,13) WHEN 5 THEN avg(inv_quantity_on_hand) WHEN 7 THEN avg(inv_quantity_on_hand) WHEN 9 THEN avg(inv_quantity_on_hand) WHEN 11 THEN avg(inv_quantity_on_hand) WHEN 13 THEN avg(inv_quantity_on_hand) END) ASC, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_product_name WHEN 7 THEN i_product_name WHEN 9 THEN i_product_name WHEN 11 THEN i_product_name WHEN 13 THEN NULL END ASC, CASE valid_tid(5,7,9,11,13) WHEN 5 [...]
@@ -294,13 +294,13 @@ Per-Host Resources: mem-estimate=11.59GB mem-reservation=138.75MB thread-reserva
    tuple-ids=0 row-size=20B cardinality=11.74M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=296.45MB Threads=9
-Per-Host Resource Estimates: Memory=41.44GB
+Max Per-Host Resource Reservation: Memory=300.45MB Threads=9
+Per-Host Resource Estimates: Memory=41.45GB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_product_name WHEN 7 THEN i_product_name WHEN 9 THEN i_product_name WHEN 11 THEN i_product_name WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_brand WHEN 7 THEN i_brand WHEN 9 THEN i_brand WHEN 11 THEN NULL WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_class WHEN 7 THEN i_class WHEN 9 THEN NULL WHEN 11 THEN NULL WHEN 13 THEN NULL END, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_category WHEN 7 TH [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: aggif(valid_tid(5,7,9,11,13) IN (5, 7, 9, 11, 13), CASE valid_tid(5,7,9,11,13) WHEN 5 THEN avg(inv_quantity_on_hand) WHEN 7 THEN avg(inv_quantity_on_hand) WHEN 9 THEN avg(inv_quantity_on_hand) WHEN 11 THEN avg(inv_quantity_on_hand) WHEN 13 THEN avg(inv_quantity_on_hand) END) ASC, CASE valid_tid(5,7,9,11,13) WHEN 5 THEN i_product_name WHEN 7 THEN i_product_name WHEN 9 THEN i_product_name WHEN 11 THEN i_product_name WHEN 13 THEN NULL END ASC, CASE valid_tid(5,7,9,11,13) WHEN 5 [...]
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 1e21e0b..d0d3a3c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23a.test
@@ -55,7 +55,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=621.94MB mem-reservation=75.19MB thread-reservation=11 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: sum(sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 49:AGGREGATE [FINALIZE]
 |  output: sum(sales)
@@ -507,13 +507,13 @@ PLAN-ROOT SINK
    tuple-ids=8 row-size=12B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=243.38MB Threads=50
+Max Per-Host Resource Reservation: Memory=247.38MB Threads=50
 Per-Host Resource Estimates: Memory=1.47GB
 F31:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 89:AGGREGATE [FINALIZE]
 |  output: sum:merge(sales)
@@ -1240,13 +1240,13 @@ Per-Host Resources: mem-estimate=62.33MB mem-reservation=7.94MB thread-reservati
    tuple-ids=8 row-size=12B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=458.62MB Threads=63
+Max Per-Host Resource Reservation: Memory=462.62MB Threads=63
 Per-Host Resource Estimates: Memory=1.17GB
 F31:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 89:AGGREGATE [FINALIZE]
 |  output: sum:merge(sales)
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 93168ef..e66c798 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q23b.test
@@ -90,7 +90,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=659.94MB mem-reservation=73.81MB thread-reservation=12 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 53:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, c_first_name ASC, sales ASC
@@ -584,13 +584,13 @@ PLAN-ROOT SINK
    tuple-ids=3 row-size=12B cardinality=2.88M
    in pipelines: 04(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=260.81MB Threads=56
-Per-Host Resource Estimates: Memory=1.56GB
+Max Per-Host Resource Reservation: Memory=264.81MB Threads=56
+Per-Host Resource Estimates: Memory=1.57GB
 F35:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=18.09KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 96:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, sales ASC
@@ -1387,13 +1387,13 @@ Per-Host Resources: mem-estimate=88.70MB mem-reservation=43.19MB thread-reservat
    tuple-ids=3 row-size=12B cardinality=2.88M
    in pipelines: 04(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=476.12MB Threads=67
+Max Per-Host Resource Reservation: Memory=480.12MB Threads=67
 Per-Host Resource Estimates: Memory=1.19GB
 F35:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=34.49KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 96:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, sales ASC
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 a6f5af1..06fc18f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24a.test
@@ -58,7 +58,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=936.44MB mem-reservation=91.81MB thread-reservation=13 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: $c$1, $c$2, $c$3, $c$4
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:SORT
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
@@ -320,13 +320,13 @@ PLAN-ROOT SINK
    tuple-ids=5 row-size=60B cardinality=50.00K
    in pipelines: 05(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=165.45MB Threads=32
-Per-Host Resource Estimates: Memory=1.14GB
+Max Per-Host Resource Reservation: Memory=169.45MB Threads=32
+Per-Host Resource Estimates: Memory=1.15GB
 F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=387.40KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.38MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: $c$1, $c$2, $c$3, $c$4
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 50:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
@@ -749,13 +749,13 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=305.70MB Threads=42
-Per-Host Resource Estimates: Memory=667MB
+Max Per-Host Resource Reservation: Memory=309.70MB Threads=42
+Per-Host Resource Estimates: Memory=671MB
 F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=604.05KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.59MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: $c$1, $c$2, $c$3, $c$4
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 50:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
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 cc79cda..727d6e0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q24b.test
@@ -58,7 +58,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=936.44MB mem-reservation=91.81MB thread-reservation=13 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: $c$1, $c$2, $c$3, $c$4
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:SORT
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
@@ -320,13 +320,13 @@ PLAN-ROOT SINK
    tuple-ids=5 row-size=60B cardinality=50.00K
    in pipelines: 05(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=165.45MB Threads=32
-Per-Host Resource Estimates: Memory=1.14GB
+Max Per-Host Resource Reservation: Memory=169.45MB Threads=32
+Per-Host Resource Estimates: Memory=1.15GB
 F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=387.40KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.38MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: $c$1, $c$2, $c$3, $c$4
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 50:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
@@ -749,13 +749,13 @@ Per-Host Resources: mem-estimate=88.92MB mem-reservation=12.88MB thread-reservat
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=305.70MB Threads=42
-Per-Host Resource Estimates: Memory=667MB
+Max Per-Host Resource Reservation: Memory=309.70MB Threads=42
+Per-Host Resource Estimates: Memory=671MB
 F19:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=604.05KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.59MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: $c$1, $c$2, $c$3, $c$4
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 50:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: $c$1 ASC, $c$2 ASC, $c$3 ASC
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 518e9e5..46156f1 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q25.test
@@ -47,7 +47,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=649.75MB mem-reservation=49.27MB thread-reservation=9 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_store_id, s_store_name, sum(ss_net_profit), sum(sr_net_loss), sum(cs_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC, i_item_desc ASC, s_store_id ASC, s_store_name ASC
@@ -212,13 +212,13 @@ PLAN-ROOT SINK
    tuple-ids=7 row-size=148B cardinality=18.00K
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=61.84MB Threads=20
-Per-Host Resource Estimates: Memory=701MB
+Max Per-Host Resource Reservation: Memory=65.84MB Threads=20
+Per-Host Resource Estimates: Memory=705MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=76.92KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.08MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_store_id, s_store_name, sum(ss_net_profit), sum(sr_net_loss), sum(cs_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, i_item_desc ASC, s_store_id ASC, s_store_name ASC
@@ -468,13 +468,13 @@ Per-Host Resources: mem-estimate=203.73MB mem-reservation=18.81MB thread-reserva
    tuple-ids=2 row-size=20B cardinality=1.44M
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=83.52MB Threads=22
-Per-Host Resource Estimates: Memory=281MB
+Max Per-Host Resource Reservation: Memory=87.52MB Threads=22
+Per-Host Resource Estimates: Memory=285MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=76.92KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.08MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_store_id, s_store_name, sum(ss_net_profit), sum(sr_net_loss), sum(cs_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, i_item_desc ASC, s_store_id ASC, s_store_name ASC
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 ece8043..2298376 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q26.test
@@ -29,7 +29,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=578.31MB mem-reservation=43.34MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, avg(cs_quantity), avg(cs_list_price), avg(cs_coupon_amt), avg(cs_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC
@@ -134,13 +134,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=36B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=44.66MB Threads=13
-Per-Host Resource Estimates: Memory=606MB
+Max Per-Host Resource Reservation: Memory=48.66MB Threads=13
+Per-Host Resource Estimates: Memory=610MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=20.70KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, avg(cs_quantity), avg(cs_list_price), avg(cs_coupon_amt), avg(cs_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
@@ -303,13 +303,13 @@ Per-Host Resources: mem-estimate=389.95MB mem-reservation=21.94MB thread-reserva
    tuple-ids=0 row-size=36B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=51.47MB Threads=12
-Per-Host Resource Estimates: Memory=165MB
+Max Per-Host Resource Reservation: Memory=55.47MB Threads=12
+Per-Host Resource Estimates: Memory=169MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=20.70KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, avg(cs_quantity), avg(cs_list_price), avg(cs_coupon_amt), avg(cs_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
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 cea109b..c20d314 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q27.test
@@ -26,7 +26,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=290.31MB mem-reservation=31.33MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(6,8,10) WHEN 6 THEN i_item_id WHEN 8 THEN i_item_id WHEN 10 THEN NULL END, CASE valid_tid(6,8,10) WHEN 6 THEN s_state WHEN 8 THEN NULL WHEN 10 THEN NULL END, aggif(valid_tid(6,8,10) IN (6, 8, 10), CASE valid_tid(6,8,10) WHEN 6 THEN 0 WHEN 8 THEN 1 WHEN 10 THEN 1 END), aggif(valid_tid(6,8,10) IN (6, 8, 10), CASE valid_tid(6,8,10) WHEN 6 THEN avg(ss_quantity) WHEN 8 THEN avg(ss_quantity) WHEN 10 THEN avg(ss_quantity) END), aggif(valid_tid(6,8,10) IN (6, 8, 1 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:TOP-N [LIMIT=100]
 |  order by: CASE valid_tid(6,8,10) WHEN 6 THEN i_item_id WHEN 8 THEN i_item_id WHEN 10 THEN NULL END ASC, CASE valid_tid(6,8,10) WHEN 6 THEN s_state WHEN 8 THEN NULL WHEN 10 THEN NULL END ASC
@@ -148,13 +148,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=48.95MB Threads=12
-Per-Host Resource Estimates: Memory=366MB
+Max Per-Host Resource Reservation: Memory=52.95MB Threads=12
+Per-Host Resource Estimates: Memory=370MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=19.73KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(6,8,10) WHEN 6 THEN i_item_id WHEN 8 THEN i_item_id WHEN 10 THEN NULL END, CASE valid_tid(6,8,10) WHEN 6 THEN s_state WHEN 8 THEN NULL WHEN 10 THEN NULL END, aggif(valid_tid(6,8,10) IN (6, 8, 10), CASE valid_tid(6,8,10) WHEN 6 THEN 0 WHEN 8 THEN 1 WHEN 10 THEN 1 END), aggif(valid_tid(6,8,10) IN (6, 8, 10), CASE valid_tid(6,8,10) WHEN 6 THEN avg(ss_quantity) WHEN 8 THEN avg(ss_quantity) WHEN 10 THEN avg(ss_quantity) END), aggif(valid_tid(6,8,10) IN (6, 8, 1 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(6,8,10) WHEN 6 THEN i_item_id WHEN 8 THEN i_item_id WHEN 10 THEN NULL END ASC, CASE valid_tid(6,8,10) WHEN 6 THEN s_state WHEN 8 THEN NULL WHEN 10 THEN NULL END ASC
@@ -334,13 +334,13 @@ Per-Host Resources: mem-estimate=165.85MB mem-reservation=30.31MB thread-reserva
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=85.14MB Threads=13
-Per-Host Resource Estimates: Memory=278MB
+Max Per-Host Resource Reservation: Memory=89.14MB Threads=13
+Per-Host Resource Estimates: Memory=282MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=37.60KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(6,8,10) WHEN 6 THEN i_item_id WHEN 8 THEN i_item_id WHEN 10 THEN NULL END, CASE valid_tid(6,8,10) WHEN 6 THEN s_state WHEN 8 THEN NULL WHEN 10 THEN NULL END, aggif(valid_tid(6,8,10) IN (6, 8, 10), CASE valid_tid(6,8,10) WHEN 6 THEN 0 WHEN 8 THEN 1 WHEN 10 THEN 1 END), aggif(valid_tid(6,8,10) IN (6, 8, 10), CASE valid_tid(6,8,10) WHEN 6 THEN avg(ss_quantity) WHEN 8 THEN avg(ss_quantity) WHEN 10 THEN avg(ss_quantity) END), aggif(valid_tid(6,8,10) IN (6, 8, 1 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(6,8,10) WHEN 6 THEN i_item_id WHEN 8 THEN i_item_id WHEN 10 THEN NULL END ASC, CASE valid_tid(6,8,10) WHEN 6 THEN s_state WHEN 8 THEN NULL WHEN 10 THEN NULL END ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test
index 9b81cf8..1b0d30d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q28.test
@@ -57,7 +57,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=444.00MB mem-reservation=23.62MB thread-reservation=7
 PLAN-ROOT SINK
 |  output exprs: avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 22:NESTED LOOP JOIN [CROSS JOIN]
 |  limit: 100
@@ -247,13 +247,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=288.04K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=35.62MB Threads=24
-Per-Host Resource Estimates: Memory=624MB
+Max Per-Host Resource Reservation: Memory=39.62MB Threads=24
+Per-Host Resource Estimates: Memory=628MB
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.08MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 22:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  limit: 100
@@ -640,13 +640,13 @@ Per-Host Resources: mem-estimate=74.00MB mem-reservation=4.00MB thread-reservati
    tuple-ids=0 row-size=16B cardinality=288.04K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=71.25MB Threads=35
-Per-Host Resource Estimates: Memory=612MB
+Max Per-Host Resource Reservation: Memory=75.25MB Threads=35
+Per-Host Resource Estimates: Memory=616MB
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price), avg(ss_list_price), zeroifnull(count(ss_list_price)), count(ss_list_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 22:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  join table id: 00
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 25f9ebe..da86db4 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q29.test
@@ -47,7 +47,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=633.75MB mem-reservation=49.27MB thread-reservation=9 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_store_id, s_store_name, sum(ss_quantity), sum(sr_return_quantity), sum(cs_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC, i_item_desc ASC, s_store_id ASC, s_store_name ASC
@@ -212,13 +212,13 @@ PLAN-ROOT SINK
    tuple-ids=7 row-size=148B cardinality=18.00K
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=61.84MB Threads=20
-Per-Host Resource Estimates: Memory=686MB
+Max Per-Host Resource Reservation: Memory=65.84MB Threads=20
+Per-Host Resource Estimates: Memory=690MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=69.10KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_store_id, s_store_name, sum(ss_quantity), sum(sr_return_quantity), sum(cs_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, i_item_desc ASC, s_store_id ASC, s_store_name ASC
@@ -468,13 +468,13 @@ Per-Host Resources: mem-estimate=203.74MB mem-reservation=18.81MB thread-reserva
    tuple-ids=2 row-size=20B cardinality=1.44M
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=83.52MB Threads=22
-Per-Host Resource Estimates: Memory=283MB
+Max Per-Host Resource Reservation: Memory=87.52MB Threads=22
+Per-Host Resource Estimates: Memory=287MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=69.10KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, s_store_id, s_store_name, sum(ss_quantity), sum(sr_return_quantity), sum(cs_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, i_item_desc ASC, s_store_id ASC, s_store_name ASC
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 4734179..8f3f595 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q30.test
@@ -55,7 +55,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=428.81MB mem-reservation=27.44MB thread-reservation=9 runtime-filters-memory=7.00MB
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_day, c_birth_month, c_birth_year, c_birth_country, c_login, c_email_address, c_last_review_date, ctr_total_return
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:TOP-N [LIMIT=100]
 |  order by: c_customer_id ASC, c_salutation ASC, c_first_name ASC, c_last_name ASC, c_preferred_cust_flag ASC, c_birth_day ASC, c_birth_month ASC, c_birth_year ASC, c_birth_country ASC, c_login ASC, c_email_address ASC, c_last_review_date ASC, ctr_total_return ASC
@@ -234,13 +234,13 @@ PLAN-ROOT SINK
    tuple-ids=6 row-size=203B cardinality=100.00K
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=45.12MB Threads=20
-Per-Host Resource Estimates: Memory=500MB
+Max Per-Host Resource Reservation: Memory=49.12MB Threads=20
+Per-Host Resource Estimates: Memory=504MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=41.67KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_day, c_birth_month, c_birth_year, c_birth_country, c_login, c_email_address, c_last_review_date, ctr_total_return
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 32:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_customer_id ASC, c_salutation ASC, c_first_name ASC, c_last_name ASC, c_preferred_cust_flag ASC, c_birth_day ASC, c_birth_month ASC, c_birth_year ASC, c_birth_country ASC, c_login ASC, c_email_address ASC, c_last_review_date ASC, ctr_total_return ASC
@@ -519,13 +519,13 @@ Per-Host Resources: mem-estimate=137.32MB mem-reservation=16.81MB thread-reserva
    tuple-ids=6 row-size=203B cardinality=100.00K
    in pipelines: 07(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=64.69MB Threads=19
-Per-Host Resource Estimates: Memory=231MB
+Max Per-Host Resource Reservation: Memory=68.69MB Threads=19
+Per-Host Resource Estimates: Memory=235MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=41.67KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_day, c_birth_month, c_birth_year, c_birth_country, c_login, c_email_address, c_last_review_date, ctr_total_return
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 32:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_customer_id ASC, c_salutation ASC, c_first_name ASC, c_last_name ASC, c_preferred_cust_flag ASC, c_birth_day ASC, c_birth_month ASC, c_birth_year ASC, c_birth_country ASC, c_login ASC, c_email_address ASC, c_last_review_date ASC, ctr_total_return ASC
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 6d0b730..43503a5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q31.test
@@ -71,12 +71,12 @@ WHERE ss1.d_qoy = 1
 ORDER BY ss1.ca_county;
 ---- PLAN
 Max Per-Host Resource Reservation: Memory=153.50MB Threads=19
-Per-Host Resource Estimates: Memory=240.18GB
+Per-Host Resource Estimates: Memory=240.21GB
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=240.18GB mem-reservation=153.50MB thread-reservation=19 runtime-filters-memory=10.00MB
+|  Per-Host Resources: mem-estimate=240.21GB mem-reservation=153.50MB thread-reservation=19 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: ca_county, d_year, (web_sales * CAST(1.0000 AS DECIMAL(5,4))) / web_sales, (store_sales * CAST(1.0000 AS DECIMAL(5,4))) / store_sales, (web_sales * CAST(1.0000 AS DECIMAL(5,4))) / web_sales, (store_sales * CAST(1.0000 AS DECIMAL(5,4))) / store_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 41:SORT
 |  order by: ca_county ASC
@@ -476,13 +476,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=174.56MB Threads=47
-Per-Host Resource Estimates: Memory=241.19GB
+Max Per-Host Resource Reservation: Memory=178.56MB Threads=47
+Per-Host Resource Estimates: Memory=241.29GB
 F28:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=30.38MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=130.38MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ca_county, d_year, (web_sales * CAST(1.0000 AS DECIMAL(5,4))) / web_sales, (store_sales * CAST(1.0000 AS DECIMAL(5,4))) / store_sales, (web_sales * CAST(1.0000 AS DECIMAL(5,4))) / web_sales, (store_sales * CAST(1.0000 AS DECIMAL(5,4))) / store_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 75:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: ca_county ASC
@@ -1121,13 +1121,13 @@ Per-Host Resources: mem-estimate=50.29MB mem-reservation=16.81MB thread-reservat
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=227.81MB Threads=58
-Per-Host Resource Estimates: Memory=480.99GB
+Max Per-Host Resource Reservation: Memory=231.81MB Threads=58
+Per-Host Resource Estimates: Memory=481.09GB
 F28:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=60.76MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=160.76MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ca_county, d_year, (web_sales * CAST(1.0000 AS DECIMAL(5,4))) / web_sales, (store_sales * CAST(1.0000 AS DECIMAL(5,4))) / store_sales, (web_sales * CAST(1.0000 AS DECIMAL(5,4))) / web_sales, (store_sales * CAST(1.0000 AS DECIMAL(5,4))) / store_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 75:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: ca_county ASC
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 509e489..82ff977 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q32.test
@@ -22,7 +22,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=391.88MB mem-reservation=28.75MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: sum(cs_ext_discount_amt)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:AGGREGATE [FINALIZE]
 |  output: sum(cs_ext_discount_amt)
@@ -132,13 +132,13 @@ PLAN-ROOT SINK
    tuple-ids=3 row-size=16B cardinality=1.44M
    in pipelines: 03(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=35.94MB Threads=13
-Per-Host Resource Estimates: Memory=438MB
+Max Per-Host Resource Reservation: Memory=39.94MB Threads=13
+Per-Host Resource Estimates: Memory=442MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(cs_ext_discount_amt)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 19:AGGREGATE [FINALIZE]
 |  output: sum:merge(cs_ext_discount_amt)
@@ -309,13 +309,13 @@ Per-Host Resources: mem-estimate=158.15MB mem-reservation=13.94MB thread-reserva
    tuple-ids=3 row-size=16B cardinality=1.44M
    in pipelines: 03(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=41.81MB Threads=12
-Per-Host Resource Estimates: Memory=204MB
+Max Per-Host Resource Reservation: Memory=45.81MB Threads=12
+Per-Host Resource Estimates: Memory=208MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(cs_ext_discount_amt)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 19:AGGREGATE [FINALIZE]
 |  output: sum:merge(cs_ext_discount_amt)
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 ff6d097..05f159f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q33.test
@@ -78,7 +78,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=363.75MB mem-reservation=29.62MB thread-reservation=6 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: i_manufact_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 32:TOP-N [LIMIT=100]
 |  order by: sum(total_sales) ASC
@@ -396,13 +396,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=69.50MB Threads=35
-Per-Host Resource Estimates: Memory=898MB
+Max Per-Host Resource Reservation: Memory=73.50MB Threads=35
+Per-Host Resource Estimates: Memory=902MB
 F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_manufact_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 55:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(total_sales) ASC
@@ -883,13 +883,13 @@ Per-Host Resources: mem-estimate=70.13MB mem-reservation=15.75MB thread-reservat
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=105.75MB Threads=35
-Per-Host Resource Estimates: Memory=469MB
+Max Per-Host Resource Reservation: Memory=109.75MB Threads=35
+Per-Host Resource Estimates: Memory=473MB
 F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_manufact_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 55:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(total_sales) ASC
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 d0e4f5a..f93f1da 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q34.test
@@ -49,7 +49,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=297.81MB mem-reservation=14.75MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, c_salutation, c_preferred_cust_flag, ss_ticket_number, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=100000]
 |  order by: c_last_name ASC, c_first_name ASC, c_salutation ASC, c_preferred_cust_flag DESC
@@ -159,13 +159,13 @@ PLAN-ROOT SINK
    tuple-ids=6 row-size=68B cardinality=100.00K
    in pipelines: 08(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=23.08MB Threads=13
-Per-Host Resource Estimates: Memory=331MB
+Max Per-Host Resource Reservation: Memory=27.08MB Threads=13
+Per-Host Resource Estimates: Memory=335MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=334.94KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.33MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, c_salutation, c_preferred_cust_flag, ss_ticket_number, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, c_salutation ASC, c_preferred_cust_flag DESC
@@ -332,13 +332,13 @@ Per-Host Resources: mem-estimate=83.87MB mem-reservation=13.81MB thread-reservat
    tuple-ids=0 row-size=24B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=39.83MB Threads=16
-Per-Host Resource Estimates: Memory=178MB
+Max Per-Host Resource Reservation: Memory=43.83MB Threads=16
+Per-Host Resource Estimates: Memory=182MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=587.56KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.57MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, c_salutation, c_preferred_cust_flag, ss_ticket_number, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, c_salutation ASC, c_preferred_cust_flag DESC
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 165a132..192faa0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q35a.test
@@ -64,7 +64,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=401.62MB mem-reservation=43.50MB thread-reservation=8 runtime-filters-memory=7.00MB
 PLAN-ROOT SINK
 |  output exprs: ca_state, cd_gender, cd_marital_status, cd_dep_count, count(*), min(cd_dep_count), max(cd_dep_count), avg(cd_dep_count), cd_dep_employed_count, count(*), min(cd_dep_employed_count), max(cd_dep_employed_count), avg(cd_dep_employed_count), cd_dep_college_count, count(*), min(cd_dep_college_count), max(cd_dep_college_count), avg(cd_dep_college_count)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:TOP-N [LIMIT=100]
 |  order by: ca_state ASC, cd_gender ASC, cd_marital_status ASC, cd_dep_count ASC, cd_dep_employed_count ASC, cd_dep_college_count ASC
@@ -243,13 +243,13 @@ PLAN-ROOT SINK
    tuple-ids=2 row-size=42B cardinality=1.92M
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=53.56MB Threads=19
-Per-Host Resource Estimates: Memory=492MB
+Max Per-Host Resource Reservation: Memory=57.56MB Threads=19
+Per-Host Resource Estimates: Memory=496MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=36.33KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ca_state, cd_gender, cd_marital_status, cd_dep_count, count(*), min(cd_dep_count), max(cd_dep_count), avg(cd_dep_count), cd_dep_employed_count, count(*), min(cd_dep_employed_count), max(cd_dep_employed_count), avg(cd_dep_employed_count), cd_dep_college_count, count(*), min(cd_dep_college_count), max(cd_dep_college_count), avg(cd_dep_college_count)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 29:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: ca_state ASC, cd_gender ASC, cd_marital_status ASC, cd_dep_count ASC, cd_dep_employed_count ASC, cd_dep_college_count ASC
@@ -507,13 +507,13 @@ Per-Host Resources: mem-estimate=20.95MB mem-reservation=5.44MB thread-reservati
    tuple-ids=3 row-size=8B cardinality=2.88M
    in pipelines: 03(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=78.50MB Threads=23
-Per-Host Resource Estimates: Memory=291MB
+Max Per-Host Resource Reservation: Memory=82.50MB Threads=23
+Per-Host Resource Estimates: Memory=295MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=69.14KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ca_state, cd_gender, cd_marital_status, cd_dep_count, count(*), min(cd_dep_count), max(cd_dep_count), avg(cd_dep_count), cd_dep_employed_count, count(*), min(cd_dep_employed_count), max(cd_dep_employed_count), avg(cd_dep_employed_count), cd_dep_college_count, count(*), min(cd_dep_college_count), max(cd_dep_college_count), avg(cd_dep_college_count)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 29:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: ca_state ASC, cd_gender ASC, cd_marital_status ASC, cd_dep_count ASC, cd_dep_employed_count ASC, cd_dep_college_count ASC
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 17c81c4..7b26f11 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q36.test
@@ -34,7 +34,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=188.81MB mem-reservation=20.94MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN sum(ss_net_profit) WHEN 5 THEN sum(ss_net_profit) WHEN 6 THEN sum(ss_net_profit) END) / aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN sum(ss_ext_sales_price) WHEN 5 THEN sum(ss_ext_sales_price) WHEN 6 THEN sum(ss_ext_sales_price) END), CASE valid_tid(4,5,6) WHEN 4 THEN i_category WHEN 5 THEN i_category WHEN 6 THEN NULL END, CASE valid_tid(4,5,6) WHEN 4 THEN i_class WHEN 5 THEN [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:TOP-N [LIMIT=100]
 |  order by: aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 0 WHEN 6 THEN 1 END) + aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 1 WHEN 6 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 0 WHEN 6 THEN 1 END) + aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 1 WHEN 6 THEN 1 END) = 0 THEN CASE valid_tid(4,5,6) WHEN [...]
@@ -150,13 +150,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=24B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=41.33MB Threads=11
-Per-Host Resource Estimates: Memory=272MB
+Max Per-Host Resource Reservation: Memory=45.33MB Threads=11
+Per-Host Resource Estimates: Memory=276MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=22.66KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN sum(ss_net_profit) WHEN 5 THEN sum(ss_net_profit) WHEN 6 THEN sum(ss_net_profit) END) / aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN sum(ss_ext_sales_price) WHEN 5 THEN sum(ss_ext_sales_price) WHEN 6 THEN sum(ss_ext_sales_price) END), CASE valid_tid(4,5,6) WHEN 4 THEN i_category WHEN 5 THEN i_category WHEN 6 THEN NULL END, CASE valid_tid(4,5,6) WHEN 4 THEN i_class WHEN 5 THEN [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 0 WHEN 6 THEN 1 END) + aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 1 WHEN 6 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 0 WHEN 6 THEN 1 END) + aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 1 WHEN 6 THEN 1 END) = 0 THEN CASE valid_tid(4,5,6) WHEN [...]
@@ -330,13 +330,13 @@ Per-Host Resources: mem-estimate=103.68MB mem-reservation=16.81MB thread-reserva
    tuple-ids=0 row-size=24B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=81.89MB Threads=13
-Per-Host Resource Estimates: Memory=271MB
+Max Per-Host Resource Reservation: Memory=85.89MB Threads=13
+Per-Host Resource Estimates: Memory=275MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=43.16KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN sum(ss_net_profit) WHEN 5 THEN sum(ss_net_profit) WHEN 6 THEN sum(ss_net_profit) END) / aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN sum(ss_ext_sales_price) WHEN 5 THEN sum(ss_ext_sales_price) WHEN 6 THEN sum(ss_ext_sales_price) END), CASE valid_tid(4,5,6) WHEN 4 THEN i_category WHEN 5 THEN i_category WHEN 6 THEN NULL END, CASE valid_tid(4,5,6) WHEN 4 THEN i_class WHEN 5 THEN [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 0 WHEN 6 THEN 1 END) + aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 1 WHEN 6 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 0 WHEN 6 THEN 1 END) + aggif(valid_tid(4,5,6) IN (4, 5, 6), CASE valid_tid(4,5,6) WHEN 4 THEN 0 WHEN 5 THEN 1 WHEN 6 THEN 1 END) = 0 THEN CASE valid_tid(4,5,6) WHEN [...]
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 414b9a0..0d46073 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q37.test
@@ -28,7 +28,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=262.88MB mem-reservation=40.94MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_current_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC
@@ -117,13 +117,13 @@ PLAN-ROOT SINK
    tuple-ids=3 row-size=8B cardinality=1.44M
    in pipelines: 03(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=64.81MB Threads=11
-Per-Host Resource Estimates: Memory=307MB
+Max Per-Host Resource Reservation: Memory=68.81MB Threads=11
+Per-Host Resource Estimates: Memory=311MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=48.19KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_current_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
@@ -262,13 +262,13 @@ Per-Host Resources: mem-estimate=68.64MB mem-reservation=21.94MB thread-reservat
    tuple-ids=3 row-size=8B cardinality=1.44M
    in pipelines: 03(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=70.69MB Threads=10
-Per-Host Resource Estimates: Memory=169MB
+Max Per-Host Resource Reservation: Memory=74.69MB Threads=10
+Per-Host Resource Estimates: Memory=173MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=48.19KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_current_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test
index 80daa11..5155064 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q38.test
@@ -34,7 +34,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=573.21MB mem-reservation=187.19MB thread-reservation=10 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 20:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -230,13 +230,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=8B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=308.56MB Threads=22
-Per-Host Resource Estimates: Memory=955MB
+Max Per-Host Resource Reservation: Memory=312.56MB Threads=22
+Per-Host Resource Estimates: Memory=959MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 36:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
@@ -540,13 +540,13 @@ Per-Host Resources: mem-estimate=116.36MB mem-reservation=43.19MB thread-reserva
    tuple-ids=0 row-size=8B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=407.12MB Threads=25
-Per-Host Resource Estimates: Memory=760MB
+Max Per-Host Resource Reservation: Memory=411.12MB Threads=25
+Per-Host Resource Estimates: Memory=764MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 36:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
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 98d8697..62acd59 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39a.test
@@ -64,7 +64,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=832.27MB mem-reservation=131.81MB thread-reservation=9 runtime-filters-memory=8.00MB
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_sk, i_item_sk, d_moy, mean, cov, w_warehouse_sk, i_item_sk, d_moy, mean, cov
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=26.33MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:SORT
 |  order by: w_warehouse_sk ASC, i_item_sk ASC, d_moy ASC, mean ASC, cov ASC, d_moy ASC, mean ASC, cov ASC
@@ -234,13 +234,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=11.74M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=182.16MB Threads=20
-Per-Host Resource Estimates: Memory=1.02GB
+Max Per-Host Resource Reservation: Memory=186.16MB Threads=20
+Per-Host Resource Estimates: Memory=1.04GB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=13.30MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=39.63MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_sk, i_item_sk, d_moy, mean, cov, w_warehouse_sk, i_item_sk, d_moy, mean, cov
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=26.33MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_warehouse_sk ASC, i_item_sk ASC, d_moy ASC, mean ASC, cov ASC, d_moy ASC, mean ASC, cov ASC
@@ -502,13 +502,13 @@ Per-Host Resources: mem-estimate=372.37MB mem-reservation=60.81MB thread-reserva
    tuple-ids=0 row-size=20B cardinality=11.74M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=199.78MB Threads=19
-Per-Host Resource Estimates: Memory=781MB
+Max Per-Host Resource Reservation: Memory=203.78MB Threads=19
+Per-Host Resource Estimates: Memory=807MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=13.30MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=39.63MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_sk, i_item_sk, d_moy, mean, cov, w_warehouse_sk, i_item_sk, d_moy, mean, cov
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=26.33MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_warehouse_sk ASC, i_item_sk ASC, d_moy ASC, mean ASC, cov ASC, d_moy ASC, mean ASC, cov ASC
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 8eb049e..1d3896d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q39b.test
@@ -65,7 +65,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=832.27MB mem-reservation=131.81MB thread-reservation=9 runtime-filters-memory=8.00MB
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_sk, i_item_sk, d_moy, mean, cov, w_warehouse_sk, i_item_sk, d_moy, mean, cov
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=26.33MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:SORT
 |  order by: w_warehouse_sk ASC, i_item_sk ASC, d_moy ASC, mean ASC, cov ASC, d_moy ASC, mean ASC, cov ASC
@@ -235,13 +235,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=11.74M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=182.16MB Threads=20
-Per-Host Resource Estimates: Memory=1.02GB
+Max Per-Host Resource Reservation: Memory=186.16MB Threads=20
+Per-Host Resource Estimates: Memory=1.04GB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=13.30MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=39.63MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_sk, i_item_sk, d_moy, mean, cov, w_warehouse_sk, i_item_sk, d_moy, mean, cov
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=26.33MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_warehouse_sk ASC, i_item_sk ASC, d_moy ASC, mean ASC, cov ASC, d_moy ASC, mean ASC, cov ASC
@@ -503,13 +503,13 @@ Per-Host Resources: mem-estimate=372.37MB mem-reservation=60.81MB thread-reserva
    tuple-ids=0 row-size=20B cardinality=11.74M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=199.78MB Threads=19
-Per-Host Resource Estimates: Memory=781MB
+Max Per-Host Resource Reservation: Memory=203.78MB Threads=19
+Per-Host Resource Estimates: Memory=807MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=13.30MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=39.63MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_warehouse_sk, i_item_sk, d_moy, mean, cov, w_warehouse_sk, i_item_sk, d_moy, mean, cov
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=26.33MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_warehouse_sk ASC, i_item_sk ASC, d_moy ASC, mean ASC, cov ASC, d_moy ASC, mean ASC, cov ASC
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 7b43ca6..3afd32b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q40.test
@@ -31,7 +31,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=437.56MB mem-reservation=25.08MB thread-reservation=6 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: w_state, i_item_id, sum(CASE WHEN (CAST(d_date AS DATE) < DATE '2000-03-11') THEN cs_sales_price - coalesce(cr_refunded_cash, 0) ELSE 0 END), sum(CASE WHEN (CAST(d_date AS DATE) >= DATE '2000-03-11') THEN cs_sales_price - coalesce(cr_refunded_cash, 0) ELSE 0 END)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=100]
 |  order by: w_state ASC, i_item_id ASC
@@ -135,13 +135,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=28B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=30.02MB Threads=12
-Per-Host Resource Estimates: Memory=462MB
+Max Per-Host Resource Reservation: Memory=34.02MB Threads=12
+Per-Host Resource Estimates: Memory=466MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=25.26KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_state, i_item_id, sum(CASE WHEN (CAST(d_date AS DATE) < DATE '2000-03-11') THEN cs_sales_price - coalesce(cr_refunded_cash, 0) ELSE 0 END), sum(CASE WHEN (CAST(d_date AS DATE) >= DATE '2000-03-11') THEN cs_sales_price - coalesce(cr_refunded_cash, 0) ELSE 0 END)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_state ASC, i_item_id ASC
@@ -296,13 +296,13 @@ Per-Host Resources: mem-estimate=266.67MB mem-reservation=23.56MB thread-reserva
    tuple-ids=0 row-size=28B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=43.58MB Threads=11
-Per-Host Resource Estimates: Memory=172MB
+Max Per-Host Resource Reservation: Memory=47.58MB Threads=11
+Per-Host Resource Estimates: Memory=176MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=25.26KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_state, i_item_id, sum(CASE WHEN (CAST(d_date AS DATE) < DATE '2000-03-11') THEN cs_sales_price - coalesce(cr_refunded_cash, 0) ELSE 0 END), sum(CASE WHEN (CAST(d_date AS DATE) >= DATE '2000-03-11') THEN cs_sales_price - coalesce(cr_refunded_cash, 0) ELSE 0 END)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_state ASC, i_item_id ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q41.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q41.test
index 5456b5a..0b7f4c4 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q41.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q41.test
@@ -72,7 +72,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=4.38MB thread-reservation=3
 PLAN-ROOT SINK
 |  output exprs: (i_product_name)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 05:TOP-N [LIMIT=100]
 |  order by: (i_product_name) ASC
@@ -126,13 +126,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=57B cardinality=1.80K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=10.44MB Threads=7
-Per-Host Resource Estimates: Memory=170MB
+Max Per-Host Resource Reservation: Memory=14.44MB Threads=7
+Per-Host Resource Estimates: Memory=174MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: (i_product_name)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: (i_product_name) ASC
@@ -229,13 +229,13 @@ Per-Host Resources: mem-estimate=59.95MB mem-reservation=4.44MB thread-reservati
    tuple-ids=0 row-size=57B cardinality=1.80K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=12.38MB Threads=6
-Per-Host Resource Estimates: Memory=76MB
+Max Per-Host Resource Reservation: Memory=16.38MB Threads=6
+Per-Host Resource Estimates: Memory=80MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: (i_product_name)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: (i_product_name) ASC
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 87c2fd0..2c99e4f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q42.test
@@ -31,7 +31,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=149.88MB mem-reservation=8.81MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_category_id, item.i_category, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:TOP-N [LIMIT=100]
 |  order by: sum(ss_ext_sales_price) DESC, dt.d_year ASC, item.i_category_id ASC, item.i_category ASC
@@ -100,13 +100,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=11.56MB Threads=8
-Per-Host Resource Estimates: Memory=170MB
+Max Per-Host Resource Reservation: Memory=15.56MB Threads=8
+Per-Host Resource Estimates: Memory=174MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_category_id, item.i_category, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, dt.d_year ASC, item.i_category_id ASC, item.i_category ASC
@@ -212,13 +212,13 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
    tuple-ids=1 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=22.38MB Threads=9
-Per-Host Resource Estimates: Memory=116MB
+Max Per-Host Resource Reservation: Memory=26.38MB Threads=9
+Per-Host Resource Estimates: Memory=120MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=28.26KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_category_id, item.i_category, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, dt.d_year ASC, item.i_category_id ASC, item.i_category ASC
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 cd8f543..b2478af 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q43.test
@@ -39,7 +39,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=149.88MB mem-reservation=8.31MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: s_store_name, s_store_id, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:TOP-N [LIMIT=100]
 |  order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sal [...]
@@ -108,13 +108,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=12B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=10.84MB Threads=8
-Per-Host Resource Estimates: Memory=170MB
+Max Per-Host Resource Reservation: Memory=14.84MB Threads=8
+Per-Host Resource Estimates: Memory=174MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=24.98KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name, s_store_id, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sal [...]
@@ -220,13 +220,13 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.38MB thread-reservati
    tuple-ids=1 row-size=12B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=21.16MB Threads=9
-Per-Host Resource Estimates: Memory=116MB
+Max Per-Host Resource Reservation: Memory=25.16MB Threads=9
+Per-Host Resource Estimates: Memory=120MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=47.51KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name, s_store_id, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END), sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sal [...]
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 1a75df2..8f44e4f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
@@ -53,7 +53,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=306.00MB mem-reservation=22.75MB thread-reservation=7 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: rnk, i_product_name, i_product_name
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:TOP-N [LIMIT=100]
 |  order by: rnk ASC
@@ -266,7 +266,7 @@ F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.23MB mem-reservation=9.81MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: rnk, i_product_name, i_product_name
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:TOP-N [LIMIT=100]
 |  order by: rnk ASC
@@ -610,7 +610,7 @@ F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: rnk, i_product_name, i_product_name
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:TOP-N [LIMIT=100]
 |  order by: rnk ASC
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 5ee16e3..878ed28 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q45.test
@@ -47,7 +47,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=342.69MB mem-reservation=27.38MB thread-reservation=7 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: ca_zip, ca_city, sum(ws_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:TOP-N [LIMIT=100]
 |  order by: ca_zip ASC, ca_city ASC
@@ -176,13 +176,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=40.38MB Threads=16
-Per-Host Resource Estimates: Memory=383MB
+Max Per-Host Resource Reservation: Memory=44.38MB Threads=16
+Per-Host Resource Estimates: Memory=387MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ca_zip, ca_city, sum(ws_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: ca_zip ASC, ca_city ASC
@@ -389,13 +389,13 @@ Per-Host Resources: mem-estimate=135.67MB mem-reservation=14.88MB thread-reserva
    tuple-ids=0 row-size=20B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=50.12MB Threads=15
-Per-Host Resource Estimates: Memory=184MB
+Max Per-Host Resource Reservation: Memory=54.12MB Threads=15
+Per-Host Resource Estimates: Memory=188MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ca_zip, ca_city, sum(ws_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: ca_zip ASC, ca_city ASC
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 347395d..d83adea 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q46.test
@@ -56,7 +56,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=394.06MB mem-reservation=49.56MB thread-reservation=8 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number, amt, profit
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, c_first_name ASC, ca_city ASC, bought_city ASC, ss_ticket_number ASC
@@ -200,13 +200,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=47.52MB Threads=17
-Per-Host Resource Estimates: Memory=428MB
+Max Per-Host Resource Reservation: Memory=51.52MB Threads=17
+Per-Host Resource Estimates: Memory=432MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=39.54KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number, amt, profit
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, ca_city ASC, bought_city ASC, ss_ticket_number ASC
@@ -422,13 +422,13 @@ Per-Host Resources: mem-estimate=136.97MB mem-reservation=26.69MB thread-reserva
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=71.08MB Threads=20
-Per-Host Resource Estimates: Memory=241MB
+Max Per-Host Resource Reservation: Memory=75.08MB Threads=20
+Per-Host Resource Estimates: Memory=245MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=75.24KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number, amt, profit
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, ca_city ASC, bought_city ASC, ss_ticket_number ASC
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 9c376fc..80ffd12 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q47.test
@@ -55,7 +55,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=618.44MB mem-reservation=163.00MB thread-reservation=13 runtime-filters-memory=9.00MB
 PLAN-ROOT SINK
 |  output exprs: i_category, i_brand, d_year, avg_monthly_sales, sum_sales, psum, nsum
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 35:TOP-N [LIMIT=100]
 |  order by: sum_sales - avg_monthly_sales ASC, d_year ASC
@@ -363,13 +363,13 @@ PLAN-ROOT SINK
    tuple-ids=9 row-size=20B cardinality=2.88M
    in pipelines: 14(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=323.70MB Threads=29
-Per-Host Resource Estimates: Memory=942MB
+Max Per-Host Resource Reservation: Memory=327.70MB Threads=29
+Per-Host Resource Estimates: Memory=946MB
 F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=38.31KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_category, i_brand, d_year, avg_monthly_sales, sum_sales, psum, nsum
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 54:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC, d_year ASC
@@ -812,13 +812,13 @@ Per-Host Resources: mem-estimate=95.70MB mem-reservation=43.81MB thread-reservat
    tuple-ids=9 row-size=20B cardinality=2.88M
    in pipelines: 14(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=611.07MB Threads=36
-Per-Host Resource Estimates: Memory=891MB
+Max Per-Host Resource Reservation: Memory=615.07MB Threads=36
+Per-Host Resource Estimates: Memory=895MB
 F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=72.91KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_category, i_brand, d_year, avg_monthly_sales, sum_sales, psum, nsum
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 54:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC, d_year ASC
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 ee4abbe..ab10d1e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q48.test
@@ -46,7 +46,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=249.81MB mem-reservation=21.77MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: sum(ss_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 09:AGGREGATE [FINALIZE]
 |  output: sum(CAST(ss_quantity AS BIGINT))
@@ -150,13 +150,13 @@ PLAN-ROOT SINK
    tuple-ids=2 row-size=39B cardinality=181.75K
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=27.33MB Threads=12
-Per-Host Resource Estimates: Memory=278MB
+Max Per-Host Resource Reservation: Memory=31.33MB Threads=12
+Per-Host Resource Estimates: Memory=282MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(ss_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_quantity)
@@ -307,13 +307,13 @@ Per-Host Resources: mem-estimate=104.30MB mem-reservation=9.88MB thread-reservat
    tuple-ids=0 row-size=28B cardinality=288.04K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=39.14MB Threads=14
-Per-Host Resource Estimates: Memory=168MB
+Max Per-Host Resource Reservation: Memory=43.14MB Threads=14
+Per-Host Resource Estimates: Memory=172MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(ss_quantity)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_quantity)
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 2d4fefa..43487e1 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q49.test
@@ -109,7 +109,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=456.88MB mem-reservation=37.31MB thread-reservation=4 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: channel, item, return_ratio, return_rank, currency_rank
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 35:TOP-N [LIMIT=100]
 |  order by: channel ASC, return_rank ASC, currency_rank ASC, item ASC
@@ -432,7 +432,7 @@ F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=24.61MB mem-reservation=15.94MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: channel, item, return_ratio, return_rank, currency_rank
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 35:TOP-N [LIMIT=100]
 |  order by: channel ASC, return_rank ASC, currency_rank ASC, item ASC
@@ -863,7 +863,7 @@ F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=24.84MB mem-reservation=15.94MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: channel, item, return_ratio, return_rank, currency_rank
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 35:TOP-N [LIMIT=100]
 |  order by: channel ASC, return_rank ASC, currency_rank ASC, item ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test
index dd87d46..34729a0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q50.test
@@ -74,7 +74,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=372.31MB mem-reservation=56.31MB thread-reservation=6 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip, sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 30) AND (sr_returned_date_sk - ss_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 60) AND (sr_returned_date_sk - ss_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHE [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=100]
 |  order by: s_store_name ASC, s_company_id ASC, s_street_number ASC, s_street_name ASC, s_street_type ASC, s_suite_number ASC, s_city ASC, s_county ASC, s_state ASC, s_zip ASC
@@ -177,13 +177,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=47.34MB Threads=12
-Per-Host Resource Estimates: Memory=416MB
+Max Per-Host Resource Reservation: Memory=51.34MB Threads=12
+Per-Host Resource Estimates: Memory=420MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=69.29KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip, sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 30) AND (sr_returned_date_sk - ss_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 60) AND (sr_returned_date_sk - ss_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHE [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, s_company_id ASC, s_street_number ASC, s_street_name ASC, s_street_type ASC, s_suite_number ASC, s_city ASC, s_county ASC, s_state ASC, s_zip ASC
@@ -337,13 +337,13 @@ Per-Host Resources: mem-estimate=108.24MB mem-reservation=39.31MB thread-reserva
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=71.59MB Threads=13
-Per-Host Resource Estimates: Memory=209MB
+Max Per-Host Resource Reservation: Memory=75.59MB Threads=13
+Per-Host Resource Estimates: Memory=213MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=131.76KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.13MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip, sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 30) AND (sr_returned_date_sk - ss_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 60) AND (sr_returned_date_sk - ss_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHE [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, s_company_id ASC, s_street_number ASC, s_street_name ASC, s_street_type ASC, s_suite_number ASC, s_city ASC, s_county ASC, s_state ASC, s_zip ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test
index 84340e0..d7d8b9e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q51.test
@@ -59,7 +59,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=332.87MB mem-reservation=106.00MB thread-reservation=5 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: item_sk, d_date, web_sales, store_sales, web_cumulative, store_cumulative
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:TOP-N [LIMIT=100]
 |  order by: item_sk ASC, d_date ASC
@@ -208,13 +208,13 @@ PLAN-ROOT SINK
    tuple-ids=5 row-size=16B cardinality=2.88M
    in pipelines: 06(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=215.88MB Threads=13
-Per-Host Resource Estimates: Memory=543MB
+Max Per-Host Resource Reservation: Memory=219.88MB Threads=13
+Per-Host Resource Estimates: Memory=547MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=21.29KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: item_sk, d_date, web_sales, store_sales, web_cumulative, store_cumulative
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 26:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: item_sk ASC, d_date ASC
@@ -435,13 +435,13 @@ Per-Host Resources: mem-estimate=104.68MB mem-reservation=37.94MB thread-reserva
    tuple-ids=5 row-size=16B cardinality=2.88M
    in pipelines: 06(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=306.75MB Threads=16
-Per-Host Resource Estimates: Memory=458MB
+Max Per-Host Resource Reservation: Memory=310.75MB Threads=16
+Per-Host Resource Estimates: Memory=462MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=21.29KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: item_sk, d_date, web_sales, store_sales, web_cumulative, store_cumulative
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 26:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: item_sk ASC, d_date ASC
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 19e8b79..019d103 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q52.test
@@ -30,7 +30,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=149.88MB mem-reservation=8.81MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_brand_id, item.i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:TOP-N [LIMIT=100]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
@@ -99,13 +99,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=11.56MB Threads=8
-Per-Host Resource Estimates: Memory=170MB
+Max Per-Host Resource Reservation: Memory=15.56MB Threads=8
+Per-Host Resource Estimates: Memory=174MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=18.16KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_brand_id, item.i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
@@ -211,13 +211,13 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
    tuple-ids=1 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=22.38MB Threads=9
-Per-Host Resource Estimates: Memory=117MB
+Max Per-Host Resource Reservation: Memory=26.38MB Threads=9
+Per-Host Resource Estimates: Memory=121MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=34.63KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: dt.d_year, item.i_brand_id, item.i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
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 1a96682..7fd67f3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q53.test
@@ -36,7 +36,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=200.81MB mem-reservation=11.75MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: i_manufact_id, sum_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:TOP-N [LIMIT=100]
 |  order by: sum_sales ASC, i_manufact_id ASC
@@ -121,13 +121,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=14.52MB Threads=10
-Per-Host Resource Estimates: Memory=221MB
+Max Per-Host Resource Reservation: Memory=18.52MB Threads=10
+Per-Host Resource Estimates: Memory=225MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_manufact_id, sum_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales ASC, i_manufact_id ASC
@@ -256,13 +256,13 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat
    tuple-ids=1 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=28.27MB Threads=11
-Per-Host Resource Estimates: Memory=138MB
+Max Per-Host Resource Reservation: Memory=32.27MB Threads=11
+Per-Host Resource Estimates: Memory=142MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_manufact_id, sum_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales ASC, i_manufact_id ASC
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 0f49bf4..1f44246 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q54.test
@@ -63,7 +63,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=554.44MB mem-reservation=91.02MB thread-reservation=11 runtime-filters-memory=8.00MB
 PLAN-ROOT SINK
 |  output exprs: SEGMENT, count(*), CAST(SEGMENT AS BIGINT) * CAST(50 AS BIGINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:TOP-N [LIMIT=100]
 |  order by: SEGMENT ASC, count(*) ASC
@@ -319,13 +319,13 @@ PLAN-ROOT SINK
    tuple-ids=9 row-size=12B cardinality=2.88M
    in pipelines: 10(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=93.95MB Threads=30
-Per-Host Resource Estimates: Memory=688MB
+Max Per-Host Resource Reservation: Memory=97.95MB Threads=30
+Per-Host Resource Estimates: Memory=692MB
 F21:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: SEGMENT, count(*), CAST(SEGMENT AS BIGINT) * CAST(50 AS BIGINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 52:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: SEGMENT ASC, count(*) ASC
@@ -750,13 +750,13 @@ Per-Host Resources: mem-estimate=34.00MB mem-reservation=3.00MB thread-reservati
    tuple-ids=9 row-size=12B cardinality=2.88M
    in pipelines: 10(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=123.27MB Threads=36
-Per-Host Resource Estimates: Memory=463MB
+Max Per-Host Resource Reservation: Memory=127.27MB Threads=36
+Per-Host Resource Estimates: Memory=467MB
 F21:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: SEGMENT, count(*), CAST(SEGMENT AS BIGINT) * CAST(50 AS BIGINT)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 52:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: SEGMENT ASC, count(*) ASC
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 32feaea..cba0f05 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q55.test
@@ -27,7 +27,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=149.88MB mem-reservation=8.81MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:TOP-N [LIMIT=100]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC
@@ -96,13 +96,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=11.56MB Threads=8
-Per-Host Resource Estimates: Memory=170MB
+Max Per-Host Resource Reservation: Memory=15.56MB Threads=8
+Per-Host Resource Estimates: Memory=174MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.86KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC
@@ -208,13 +208,13 @@ Per-Host Resources: mem-estimate=47.91MB mem-reservation=8.88MB thread-reservati
    tuple-ids=1 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=22.38MB Threads=9
-Per-Host Resource Estimates: Memory=116MB
+Max Per-Host Resource Reservation: Memory=26.38MB Threads=9
+Per-Host Resource Estimates: Memory=120MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=32.15KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, sum(ss_ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC
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 47b005e..51b57e7 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q56.test
@@ -80,7 +80,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=361.81MB mem-reservation=28.44MB thread-reservation=6 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 32:TOP-N [LIMIT=100]
 |  order by: sum(total_sales) ASC, i_item_id ASC
@@ -398,13 +398,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=71.62MB Threads=36
-Per-Host Resource Estimates: Memory=901MB
+Max Per-Host Resource Reservation: Memory=75.62MB Threads=36
+Per-Host Resource Estimates: Memory=905MB
 F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 56:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(total_sales) ASC, i_item_id ASC
@@ -892,13 +892,13 @@ Per-Host Resources: mem-estimate=54.97MB mem-reservation=8.88MB thread-reservati
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=107.88MB Threads=38
-Per-Host Resource Estimates: Memory=474MB
+Max Per-Host Resource Reservation: Memory=111.88MB Threads=38
+Per-Host Resource Estimates: Memory=478MB
 F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=29.56KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 56:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(total_sales) ASC, i_item_id ASC
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 0de0f32..2a30c53 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q57.test
@@ -71,7 +71,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1002.44MB mem-reservation=163.00MB thread-reservation=13 runtime-filters-memory=9.00MB
 PLAN-ROOT SINK
 |  output exprs: i_category, i_brand, cc_name, d_year, d_moy, avg_monthly_sales, sum_sales, psum, nsum
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 35:TOP-N [LIMIT=100]
 |  order by: sum_sales - avg_monthly_sales ASC
@@ -376,13 +376,13 @@ PLAN-ROOT SINK
    tuple-ids=9 row-size=20B cardinality=1.44M
    in pipelines: 14(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=213.67MB Threads=29
+Max Per-Host Resource Reservation: Memory=217.67MB Threads=29
 Per-Host Resource Estimates: Memory=1.16GB
 F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=47.37KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_category, i_brand, cc_name, d_year, d_moy, avg_monthly_sales, sum_sales, psum, nsum
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 54:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC
@@ -822,13 +822,13 @@ Per-Host Resources: mem-estimate=218.83MB mem-reservation=29.81MB thread-reserva
    tuple-ids=9 row-size=20B cardinality=1.44M
    in pipelines: 14(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=242.05MB Threads=28
-Per-Host Resource Estimates: Memory=540MB
+Max Per-Host Resource Reservation: Memory=246.05MB Threads=28
+Per-Host Resource Estimates: Memory=544MB
 F16:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=47.37KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_category, i_brand, cc_name, d_year, d_moy, avg_monthly_sales, sum_sales, psum, nsum
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 54:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC
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 75cf1f2..1961e43 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q58.test
@@ -76,7 +76,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=680.44MB mem-reservation=60.56MB thread-reservation=16 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: item_id, ss_item_rev, ss_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), cs_item_rev, cs_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), ws_item_rev, ws_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), (ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 35:TOP-N [LIMIT=100]
 |  order by: item_id ASC, ss_item_rev ASC
@@ -405,13 +405,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=83.25MB Threads=35
-Per-Host Resource Estimates: Memory=768MB
+Max Per-Host Resource Reservation: Memory=87.25MB Threads=35
+Per-Host Resource Estimates: Memory=772MB
 F21:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=25.91KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: item_id, ss_item_rev, ss_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), cs_item_rev, cs_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), ws_item_rev, ws_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), (ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 57:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: item_id ASC, ss_item_rev ASC
@@ -899,13 +899,13 @@ Per-Host Resources: mem-estimate=54.26MB mem-reservation=12.75MB thread-reservat
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=129.00MB Threads=38
-Per-Host Resource Estimates: Memory=493MB
+Max Per-Host Resource Reservation: Memory=133.00MB Threads=38
+Per-Host Resource Estimates: Memory=497MB
 F21:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=49.35KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: item_id, ss_item_rev, ss_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), cs_item_rev, cs_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), ws_item_rev, ws_item_rev / ((ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))) * CAST(100 AS DECIMAL(3,0)), (ss_item_rev + cs_item_rev + ws_item_rev) / CAST(3 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 57:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: item_id ASC, ss_item_rev ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test
index c512c9d..3246e6c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q59.test
@@ -87,7 +87,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=325.50MB mem-reservation=45.15MB thread-reservation=9 runtime-filters-memory=8.00MB
 PLAN-ROOT SINK
 |  output exprs: s_store_name1, s_store_id1, d_week_seq1, sun_sales1 / sun_sales2, mon_sales1 / mon_sales2, tue_sales1 / tue_sales2, wed_sales1 / wed_sales2, thu_sales1 / thu_sales2, fri_sales1 / fri_sales2, sat_sales1 / sat_sales2
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:TOP-N [LIMIT=100]
 |  order by: s_store_name1 ASC, s_store_id1 ASC, d_week_seq1 ASC
@@ -259,13 +259,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=58.16MB Threads=20
-Per-Host Resource Estimates: Memory=393MB
+Max Per-Host Resource Reservation: Memory=62.16MB Threads=20
+Per-Host Resource Estimates: Memory=397MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=89.79KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.09MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name1, s_store_id1, d_week_seq1, sun_sales1 / sun_sales2, mon_sales1 / mon_sales2, tue_sales1 / tue_sales2, wed_sales1 / wed_sales2, thu_sales1 / thu_sales2, fri_sales1 / fri_sales2, sat_sales1 / sat_sales2
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name1 ASC, s_store_id1 ASC, d_week_seq1 ASC
@@ -530,13 +530,13 @@ Per-Host Resources: mem-estimate=48.80MB mem-reservation=14.38MB thread-reservat
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=99.54MB Threads=25
-Per-Host Resource Estimates: Memory=324MB
+Max Per-Host Resource Reservation: Memory=103.54MB Threads=25
+Per-Host Resource Estimates: Memory=328MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=170.73KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.17MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name1, s_store_id1, d_week_seq1, sun_sales1 / sun_sales2, mon_sales1 / mon_sales2, tue_sales1 / tue_sales2, wed_sales1 / wed_sales2, thu_sales1 / thu_sales2, fri_sales1 / fri_sales2, sat_sales1 / sat_sales2
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name1 ASC, s_store_id1 ASC, d_week_seq1 ASC
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 b11338f..d3cf472 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q60.test
@@ -81,7 +81,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=361.81MB mem-reservation=28.44MB thread-reservation=6 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 32:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC, sum(total_sales) ASC
@@ -399,13 +399,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=71.62MB Threads=36
-Per-Host Resource Estimates: Memory=902MB
+Max Per-Host Resource Reservation: Memory=75.62MB Threads=36
+Per-Host Resource Estimates: Memory=906MB
 F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 56:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, sum(total_sales) ASC
@@ -893,13 +893,13 @@ Per-Host Resources: mem-estimate=54.97MB mem-reservation=8.88MB thread-reservati
    tuple-ids=0 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=107.88MB Threads=38
-Per-Host Resource Estimates: Memory=474MB
+Max Per-Host Resource Reservation: Memory=111.88MB Threads=38
+Per-Host Resource Estimates: Memory=478MB
 F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=29.56KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, sum(total_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 56:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, sum(total_sales) ASC
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 1f9052f..29ed57d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q61.test
@@ -53,7 +53,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=591.31MB mem-reservation=39.38MB thread-reservation=14 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: promotions, total, CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4)) * CAST(100 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:TOP-N [LIMIT=100]
 |  order by: promotions ASC, total ASC
@@ -327,13 +327,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=43.38MB Threads=30
+Max Per-Host Resource Reservation: Memory=47.38MB Threads=30
 Per-Host Resource Estimates: Memory=637MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.03MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: promotions, total, CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4)) * CAST(100 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:TOP-N [LIMIT=100]
 |  order by: promotions ASC, total ASC
@@ -731,13 +731,13 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=76.69MB Threads=35
+Max Per-Host Resource Reservation: Memory=80.69MB Threads=35
 Per-Host Resource Estimates: Memory=373MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: promotions, total, CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4)) * CAST(100 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:TOP-N [LIMIT=100]
 |  order by: promotions ASC, total ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test
index 082e469..b5b763b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q62.test
@@ -52,7 +52,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=299.75MB mem-reservation=17.69MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: w_substr, sm_type, web_name, sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 30) AND (ws_ship_date_sk - ws_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 60) AND (ws_ship_date_sk - ws_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 90) AND (ws_ship_date_sk - ws_sold_date_sk <= 120) THEN 1 ELSE 0 END), sum(CA [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=100]
 |  order by: w_substr ASC, sm_type ASC, web_name ASC
@@ -153,13 +153,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=20.23MB Threads=12
-Per-Host Resource Estimates: Memory=320MB
+Max Per-Host Resource Reservation: Memory=24.23MB Threads=12
+Per-Host Resource Estimates: Memory=324MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=22.63KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_substr, sm_type, web_name, sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 30) AND (ws_ship_date_sk - ws_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 60) AND (ws_ship_date_sk - ws_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 90) AND (ws_ship_date_sk - ws_sold_date_sk <= 120) THEN 1 ELSE 0 END), sum(CA [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_substr ASC, sm_type ASC, web_name ASC
@@ -311,13 +311,13 @@ Per-Host Resources: mem-estimate=181.86MB mem-reservation=17.75MB thread-reserva
    tuple-ids=0 row-size=20B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=31.98MB Threads=11
-Per-Host Resource Estimates: Memory=140MB
+Max Per-Host Resource Reservation: Memory=35.98MB Threads=11
+Per-Host Resource Estimates: Memory=144MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=22.63KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_substr, sm_type, web_name, sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 30) AND (ws_ship_date_sk - ws_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 60) AND (ws_ship_date_sk - ws_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 90) AND (ws_ship_date_sk - ws_sold_date_sk <= 120) THEN 1 ELSE 0 END), sum(CA [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_substr ASC, sm_type ASC, web_name ASC
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 933c823..a6a9e98 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q63.test
@@ -40,7 +40,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=204.81MB mem-reservation=15.75MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: i_manager_id, sum_sales, avg_monthly_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:TOP-N [LIMIT=100]
 |  order by: i_manager_id ASC, avg_monthly_sales ASC, sum_sales ASC
@@ -144,13 +144,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=24.52MB Threads=10
-Per-Host Resource Estimates: Memory=231MB
+Max Per-Host Resource Reservation: Memory=28.52MB Threads=10
+Per-Host Resource Estimates: Memory=235MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_manager_id, sum_sales, avg_monthly_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_manager_id ASC, avg_monthly_sales ASC, sum_sales ASC
@@ -298,13 +298,13 @@ Per-Host Resources: mem-estimate=66.86MB mem-reservation=11.81MB thread-reservat
    tuple-ids=1 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=48.27MB Threads=11
-Per-Host Resource Estimates: Memory=158MB
+Max Per-Host Resource Reservation: Memory=52.27MB Threads=11
+Per-Host Resource Estimates: Memory=162MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_manager_id, sum_sales, avg_monthly_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_manager_id ASC, avg_monthly_sales ASC, sum_sales ASC
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 958c4ba..ce76bd3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q64.test
@@ -131,7 +131,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=2.17GB mem-reservation=154.39MB thread-reservation=39 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: product_name, store_name, store_zip, b_street_number, b_street_name, b_city, b_zip, c_street_number, c_street_name, c_city, c_zip, syear, cnt, s1, s2, s3, s1, s2, s3, syear, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=90.78MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 79:SORT
 |  order by: product_name ASC, store_name ASC, cnt ASC, s1 ASC, s1 ASC
@@ -884,13 +884,13 @@ PLAN-ROOT SINK
    tuple-ids=17 row-size=78B cardinality=50.00K
    in pipelines: 17(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=436.52MB Threads=98
-Per-Host Resource Estimates: Memory=2.68GB
+Max Per-Host Resource Reservation: Memory=440.52MB Threads=98
+Per-Host Resource Estimates: Memory=2.76GB
 F59:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=30.98MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=121.76MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: product_name, store_name, store_zip, b_street_number, b_street_name, b_city, b_zip, c_street_number, c_street_name, c_city, c_zip, syear, cnt, s1, s2, s3, s1, s2, s3, syear, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=90.78MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 142:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: product_name ASC, store_name ASC, cnt ASC, s1 ASC, s1 ASC
@@ -2079,13 +2079,13 @@ Per-Host Resources: mem-estimate=135.93MB mem-reservation=15.88MB thread-reserva
    tuple-ids=0 row-size=56B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=672.52MB Threads=135
-Per-Host Resource Estimates: Memory=1.64GB
+Max Per-Host Resource Reservation: Memory=676.52MB Threads=135
+Per-Host Resource Estimates: Memory=1.73GB
 F59:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=32.22MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=123.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: product_name, store_name, store_zip, b_street_number, b_street_name, b_city, b_zip, c_street_number, c_street_name, c_city, c_zip, syear, cnt, s1, s2, s3, s1, s2, s3, syear, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=90.78MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 142:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: product_name ASC, store_name ASC, cnt ASC, s1 ASC, s1 ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test
index 65a2006..79640c8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q65.test
@@ -60,7 +60,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=287.56MB mem-reservation=29.08MB thread-reservation=7 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: s_store_name, i_item_desc, revenue, i_current_price, i_wholesale_cost, i_brand
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:TOP-N [LIMIT=100]
 |  order by: s_store_name ASC, i_item_desc ASC
@@ -201,13 +201,13 @@ PLAN-ROOT SINK
    tuple-ids=9 row-size=20B cardinality=2.88M
    in pipelines: 07(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=41.52MB Threads=17
-Per-Host Resource Estimates: Memory=358MB
+Max Per-Host Resource Reservation: Memory=45.52MB Threads=17
+Per-Host Resource Estimates: Memory=362MB
 F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=60.05KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name, i_item_desc, revenue, i_current_price, i_wholesale_cost, i_brand
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, i_item_desc ASC
@@ -441,13 +441,13 @@ Per-Host Resources: mem-estimate=64.00MB mem-reservation=11.94MB thread-reservat
    tuple-ids=9 row-size=20B cardinality=2.88M
    in pipelines: 07(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=74.02MB Threads=23
-Per-Host Resource Estimates: Memory=294MB
+Max Per-Host Resource Reservation: Memory=78.02MB Threads=23
+Per-Host Resource Estimates: Memory=298MB
 F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=114.22KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.11MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: s_store_name, i_item_desc, revenue, i_current_price, i_wholesale_cost, i_brand
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, i_item_desc ASC
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 610c278..90bd0ce 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q66.test
@@ -46,7 +46,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=591.31MB mem-reservation=39.38MB thread-reservation=14 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: promotions, total, CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4)) * CAST(100 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:SORT
 |  order by: promotions ASC, total ASC
@@ -320,13 +320,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=49.38MB Threads=30
+Max Per-Host Resource Reservation: Memory=53.38MB Threads=30
 Per-Host Resource Estimates: Memory=643MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.02MB mem-reservation=6.00MB thread-reservation=1
+|  Per-Host Resources: mem-estimate=16.02MB mem-reservation=10.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: promotions, total, CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4)) * CAST(100 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:SORT
 |  order by: promotions ASC, total ASC
@@ -724,13 +724,13 @@ Per-Host Resources: mem-estimate=92.88MB mem-reservation=14.75MB thread-reservat
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=82.69MB Threads=35
+Max Per-Host Resource Reservation: Memory=86.69MB Threads=35
 Per-Host Resource Estimates: Memory=379MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=6.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=10.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: promotions, total, CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4)) * CAST(100 AS DECIMAL(3,0))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 27:SORT
 |  order by: promotions ASC, total ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
index d81eb4b..30aa36d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
@@ -47,7 +47,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=18.41GB mem-reservation=251.75MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy, s_store_id, sumsales, rk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 12:TOP-N [LIMIT=100]
 |  order by: i_category ASC, i_class ASC, i_brand ASC, i_product_name ASC, d_year ASC, d_qoy ASC, d_moy ASC, s_store_id ASC, sumsales ASC, rk ASC
@@ -186,13 +186,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=24B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=472.52MB Threads=11
-Per-Host Resource Estimates: Memory=53.05GB
+Max Per-Host Resource Reservation: Memory=476.52MB Threads=11
+Per-Host Resource Estimates: Memory=53.06GB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.42KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy, s_store_id, sumsales, rk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_brand ASC, i_product_name ASC, d_year ASC, d_qoy ASC, d_moy ASC, s_store_id ASC, sumsales ASC, rk ASC
@@ -414,13 +414,13 @@ Per-Host Resources: mem-estimate=8.96GB mem-reservation=221.75MB thread-reservat
    tuple-ids=0 row-size=24B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=943.02MB Threads=13
+Max Per-Host Resource Reservation: Memory=947.02MB Threads=13
 Per-Host Resource Estimates: Memory=54.49GB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=61.72KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy, s_store_id, sumsales, rk
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_brand ASC, i_product_name ASC, d_year ASC, d_qoy ASC, d_moy ASC, s_store_id ASC, sumsales ASC, rk ASC
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 243dddd..e873404 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q68.test
@@ -54,7 +54,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=394.06MB mem-reservation=66.56MB thread-reservation=8 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number, extended_price, extended_tax, list_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, ss_ticket_number ASC
@@ -198,13 +198,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=40B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=104.14MB Threads=16
-Per-Host Resource Estimates: Memory=481MB
+Max Per-Host Resource Reservation: Memory=108.14MB Threads=16
+Per-Host Resource Estimates: Memory=485MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=44.75KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number, extended_price, extended_tax, list_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, ss_ticket_number ASC
@@ -413,13 +413,13 @@ Per-Host Resources: mem-estimate=177.17MB mem-reservation=51.69MB thread-reserva
    tuple-ids=0 row-size=40B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=132.20MB Threads=17
-Per-Host Resource Estimates: Memory=276MB
+Max Per-Host Resource Reservation: Memory=136.20MB Threads=17
+Per-Host Resource Estimates: Memory=280MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=85.13KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.08MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number, extended_price, extended_tax, list_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, ss_ticket_number ASC
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 970de54..b278708 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q69.test
@@ -58,7 +58,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=512.62MB mem-reservation=36.88MB thread-reservation=10 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: cd_gender, cd_marital_status, cd_education_status, count(*), cd_purchase_estimate, count(*), cd_credit_rating, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:TOP-N [LIMIT=100]
 |  order by: cd_gender ASC, cd_marital_status ASC, cd_education_status ASC, cd_purchase_estimate ASC, cd_credit_rating ASC
@@ -240,13 +240,13 @@ PLAN-ROOT SINK
    tuple-ids=9 row-size=8B cardinality=1.44M
    in pipelines: 09(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=48.62MB Threads=21
-Per-Host Resource Estimates: Memory=544MB
+Max Per-Host Resource Reservation: Memory=52.62MB Threads=21
+Per-Host Resource Estimates: Memory=548MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=26.75KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: cd_gender, cd_marital_status, cd_education_status, count(*), cd_purchase_estimate, count(*), cd_credit_rating, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cd_gender ASC, cd_marital_status ASC, cd_education_status ASC, cd_purchase_estimate ASC, cd_credit_rating ASC
@@ -514,13 +514,13 @@ Per-Host Resources: mem-estimate=98.95MB mem-reservation=6.94MB thread-reservati
    tuple-ids=9 row-size=8B cardinality=1.44M
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=63.75MB Threads=21
-Per-Host Resource Estimates: Memory=272MB
+Max Per-Host Resource Reservation: Memory=67.75MB Threads=21
+Per-Host Resource Estimates: Memory=276MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=26.75KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: cd_gender, cd_marital_status, cd_education_status, count(*), cd_purchase_estimate, count(*), cd_credit_rating, count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cd_gender ASC, cd_marital_status ASC, cd_education_status ASC, cd_purchase_estimate ASC, cd_credit_rating ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
index b9a6ad5..8ee4acf 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
@@ -43,7 +43,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=228.75MB mem-reservation=32.34MB thread-reservation=7 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN sum(ss_net_profit) WHEN 12 THEN sum(ss_net_profit) WHEN 13 THEN sum(ss_net_profit) END), CASE valid_tid(11,12,13) WHEN 11 THEN s_state WHEN 12 THEN s_state WHEN 13 THEN NULL END, CASE valid_tid(11,12,13) WHEN 11 THEN s_county WHEN 12 THEN NULL WHEN 13 THEN NULL END, CAST(aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) AS S [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 19:TOP-N [LIMIT=100]
 |  order by: aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) + aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 1 WHEN 13 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) + aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 1 WHEN 13 [...]
@@ -228,13 +228,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=68.41MB Threads=16
-Per-Host Resource Estimates: Memory=346MB
+Max Per-Host Resource Reservation: Memory=72.41MB Threads=16
+Per-Host Resource Estimates: Memory=350MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN sum(ss_net_profit) WHEN 12 THEN sum(ss_net_profit) WHEN 13 THEN sum(ss_net_profit) END), CASE valid_tid(11,12,13) WHEN 11 THEN s_state WHEN 12 THEN s_state WHEN 13 THEN NULL END, CASE valid_tid(11,12,13) WHEN 11 THEN s_county WHEN 12 THEN NULL WHEN 13 THEN NULL END, CAST(aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) AS S [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) + aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 1 WHEN 13 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) + aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 1 WHEN 13 [...]
@@ -505,13 +505,13 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=133.79MB Threads=20
-Per-Host Resource Estimates: Memory=402MB
+Max Per-Host Resource Reservation: Memory=137.79MB Threads=20
+Per-Host Resource Estimates: Memory=406MB
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN sum(ss_net_profit) WHEN 12 THEN sum(ss_net_profit) WHEN 13 THEN sum(ss_net_profit) END), CASE valid_tid(11,12,13) WHEN 11 THEN s_state WHEN 12 THEN s_state WHEN 13 THEN NULL END, CASE valid_tid(11,12,13) WHEN 11 THEN s_county WHEN 12 THEN NULL WHEN 13 THEN NULL END, CAST(aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) AS S [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) + aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 1 WHEN 13 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 0 WHEN 13 THEN 1 END) + aggif(valid_tid(11,12,13) IN (11, 12, 13), CASE valid_tid(11,12,13) WHEN 11 THEN 0 WHEN 12 THEN 1 WHEN 13 [...]
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 f013de8..02dbb42 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q71.test
@@ -52,7 +52,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=379.75MB mem-reservation=54.25MB thread-reservation=5 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, t_hour, t_minute, sum(ext_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=16.01MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:SORT
 |  order by: sum(ext_price) DESC, i_brand_id ASC
@@ -211,13 +211,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=20B cardinality=719.38K
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=68.00MB Threads=14
-Per-Host Resource Estimates: Memory=529MB
+Max Per-Host Resource Reservation: Memory=72.00MB Threads=14
+Per-Host Resource Estimates: Memory=545MB
 F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=5.51MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=21.52MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, t_hour, t_minute, sum(ext_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=16.01MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ext_price) DESC, i_brand_id ASC
@@ -433,13 +433,13 @@ Per-Host Resources: mem-estimate=222.51MB mem-reservation=36.75MB thread-reserva
    tuple-ids=1 row-size=20B cardinality=719.38K
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=108.50MB Threads=15
-Per-Host Resource Estimates: Memory=279MB
+Max Per-Host Resource Reservation: Memory=112.50MB Threads=15
+Per-Host Resource Estimates: Memory=295MB
 F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=5.69MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=21.70MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_brand_id, i_brand, t_hour, t_minute, sum(ext_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=16.01MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ext_price) DESC, i_brand_id ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test
index b7e6105..ae796d1 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q72.test
@@ -44,7 +44,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=932.05MB mem-reservation=123.56MB thread-reservation=12 runtime-filters-memory=9.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_desc, w_warehouse_name, d1.d_week_seq, sum(CASE WHEN p_promo_sk IS NULL THEN 1 ELSE 0 END), sum(CASE WHEN p_promo_sk IS NOT NULL THEN 1 ELSE 0 END), count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 22:TOP-N [LIMIT=100]
 |  order by: count(*) DESC, i_item_desc ASC, w_warehouse_name ASC, d1.d_week_seq ASC
@@ -263,13 +263,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=20B cardinality=11.74M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=195.17MB Threads=25
-Per-Host Resource Estimates: Memory=1.16GB
+Max Per-Host Resource Reservation: Memory=199.17MB Threads=25
+Per-Host Resource Estimates: Memory=1.17GB
 F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=56.49KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_desc, w_warehouse_name, d1.d_week_seq, sum(CASE WHEN p_promo_sk IS NULL THEN 1 ELSE 0 END), sum(CASE WHEN p_promo_sk IS NOT NULL THEN 1 ELSE 0 END), count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 36:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC, i_item_desc ASC, w_warehouse_name ASC, d1.d_week_seq ASC
@@ -586,13 +586,13 @@ Per-Host Resources: mem-estimate=389.00MB mem-reservation=21.00MB thread-reserva
    tuple-ids=0 row-size=40B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=228.73MB Threads=24
-Per-Host Resource Estimates: Memory=625MB
+Max Per-Host Resource Reservation: Memory=232.73MB Threads=24
+Per-Host Resource Estimates: Memory=629MB
 F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=56.49KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_desc, w_warehouse_name, d1.d_week_seq, sum(CASE WHEN p_promo_sk IS NULL THEN 1 ELSE 0 END), sum(CASE WHEN p_promo_sk IS NOT NULL THEN 1 ELSE 0 END), count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 36:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC, i_item_desc ASC, w_warehouse_name ASC, d1.d_week_seq ASC
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 72c4b71..b638ff3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q73.test
@@ -43,7 +43,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=265.81MB mem-reservation=17.56MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, c_salutation, c_preferred_cust_flag, ss_ticket_number, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=1000]
 |  order by: cnt DESC
@@ -150,13 +150,13 @@ PLAN-ROOT SINK
    tuple-ids=6 row-size=68B cardinality=100.00K
    in pipelines: 08(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=25.02MB Threads=13
-Per-Host Resource Estimates: Memory=300MB
+Max Per-Host Resource Reservation: Memory=29.02MB Threads=13
+Per-Host Resource Estimates: Memory=304MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=272.81KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.27MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, c_salutation, c_preferred_cust_flag, ss_ticket_number, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cnt DESC
@@ -320,13 +320,13 @@ Per-Host Resources: mem-estimate=84.14MB mem-reservation=14.81MB thread-reservat
    tuple-ids=0 row-size=24B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=39.83MB Threads=16
-Per-Host Resource Estimates: Memory=179MB
+Max Per-Host Resource Reservation: Memory=43.83MB Threads=16
+Per-Host Resource Estimates: Memory=183MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=519.50KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.51MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, c_salutation, c_preferred_cust_flag, ss_ticket_number, cnt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cnt DESC
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 3dce97b..fd883cb 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q74.test
@@ -71,7 +71,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=689.94MB mem-reservation=167.56MB thread-reservation=13 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 31:TOP-N [LIMIT=100]
 |  order by: customer_id ASC
@@ -360,13 +360,13 @@ PLAN-ROOT SINK
    tuple-ids=11 row-size=12B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=300.94MB Threads=33
-Per-Host Resource Estimates: Memory=984MB
+Max Per-Host Resource Reservation: Memory=304.94MB Threads=33
+Per-Host Resource Estimates: Memory=988MB
 F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=22.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 55:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC
@@ -824,13 +824,13 @@ Per-Host Resources: mem-estimate=35.95MB mem-reservation=4.94MB thread-reservati
    tuple-ids=11 row-size=12B cardinality=2.88M
    in pipelines: 09(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=333.88MB Threads=40
-Per-Host Resource Estimates: Memory=674MB
+Max Per-Host Resource Reservation: Memory=337.88MB Threads=40
+Per-Host Resource Estimates: Memory=678MB
 F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=41.91KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: customer_id, customer_first_name, customer_last_name
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 55:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: customer_id ASC
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 1bb213a..f7ffff9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q75.test
@@ -82,7 +82,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=965.75MB mem-reservation=60.75MB thread-reservation=9 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: d_year, d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_cnt, sales_cnt - sales_cnt, sales_amt - sales_amt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 49:TOP-N [LIMIT=100]
 |  order by: sales_cnt - sales_cnt ASC, sales_amt - sales_amt ASC
@@ -574,13 +574,13 @@ PLAN-ROOT SINK
    tuple-ids=3 row-size=24B cardinality=144.07K
    in pipelines: 04(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=136.44MB Threads=56
+Max Per-Host Resource Reservation: Memory=140.44MB Threads=56
 Per-Host Resource Estimates: Memory=2.30GB
 F37:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=24.61KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: d_year, d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_cnt, sales_cnt - sales_cnt, sales_amt - sales_amt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 84:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sales_cnt - sales_cnt ASC, sales_amt - sales_amt ASC
@@ -1305,13 +1305,13 @@ Per-Host Resources: mem-estimate=246.01MB mem-reservation=13.88MB thread-reserva
    tuple-ids=0 row-size=28B cardinality=1.44M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=201.12MB Threads=67
-Per-Host Resource Estimates: Memory=869MB
+Max Per-Host Resource Reservation: Memory=205.12MB Threads=67
+Per-Host Resource Estimates: Memory=873MB
 F37:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=46.88KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: d_year, d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id, sales_cnt, sales_cnt, sales_cnt - sales_cnt, sales_amt - sales_amt
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 84:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sales_cnt - sales_cnt ASC, sales_amt - sales_amt ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test
index e2357d6..3eb1f23 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q76.test
@@ -62,7 +62,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=296.94MB mem-reservation=33.69MB thread-reservation=4 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: channel, col_name, d_year, d_qoy, i_category, count(*), sum(ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:TOP-N [LIMIT=100]
 |  order by: channel ASC, col_name ASC, d_year ASC, d_qoy ASC, i_category ASC
@@ -229,13 +229,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=130.03K
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=45.69MB Threads=19
-Per-Host Resource Estimates: Memory=618MB
+Max Per-Host Resource Reservation: Memory=49.69MB Threads=19
+Per-Host Resource Estimates: Memory=622MB
 F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=25.23KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: channel, col_name, d_year, d_qoy, i_category, count(*), sum(ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: channel ASC, col_name ASC, d_year ASC, d_qoy ASC, i_category ASC
@@ -480,13 +480,13 @@ Per-Host Resources: mem-estimate=67.20MB mem-reservation=17.88MB thread-reservat
    tuple-ids=0 row-size=20B cardinality=130.03K
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=68.19MB Threads=20
-Per-Host Resource Estimates: Memory=277MB
+Max Per-Host Resource Reservation: Memory=72.19MB Threads=20
+Per-Host Resource Estimates: Memory=281MB
 F13:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=48.05KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: channel, col_name, d_year, d_qoy, i_category, count(*), sum(ext_sales_price)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: channel ASC, col_name ASC, d_year ASC, d_qoy ASC, i_category ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test
index 521840d..ab1ce71 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q77.test
@@ -111,7 +111,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=371.88MB mem-reservation=36.44MB thread-reservation=7 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(32,30,31) WHEN 30 THEN channel WHEN 31 THEN channel WHEN 32 THEN NULL END, CASE valid_tid(32,30,31) WHEN 30 THEN id WHEN 31 THEN NULL WHEN 32 THEN NULL END, aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(sales) WHEN 31 THEN sum(sales) WHEN 32 THEN sum(sales) END), aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(`returns`) WHEN 31 THEN sum(`returns`) WHEN 32 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 38:TOP-N [LIMIT=100]
 |  order by: CASE valid_tid(32,30,31) WHEN 30 THEN channel WHEN 31 THEN channel WHEN 32 THEN NULL END ASC, CASE valid_tid(32,30,31) WHEN 30 THEN id WHEN 31 THEN NULL WHEN 32 THEN NULL END ASC, aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(sales) WHEN 31 THEN sum(sales) WHEN 32 THEN sum(sales) END) DESC
@@ -470,13 +470,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=89.98MB Threads=36
-Per-Host Resource Estimates: Memory=1.02GB
+Max Per-Host Resource Reservation: Memory=93.98MB Threads=36
+Per-Host Resource Estimates: Memory=1.03GB
 F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=18.04KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(32,30,31) WHEN 30 THEN channel WHEN 31 THEN channel WHEN 32 THEN NULL END, CASE valid_tid(32,30,31) WHEN 30 THEN id WHEN 31 THEN NULL WHEN 32 THEN NULL END, aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(sales) WHEN 31 THEN sum(sales) WHEN 32 THEN sum(sales) END), aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(`returns`) WHEN 31 THEN sum(`returns`) WHEN 32 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 64:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(32,30,31) WHEN 30 THEN channel WHEN 31 THEN channel WHEN 32 THEN NULL END ASC, CASE valid_tid(32,30,31) WHEN 30 THEN id WHEN 31 THEN NULL WHEN 32 THEN NULL END ASC, aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(sales) WHEN 31 THEN sum(sales) WHEN 32 THEN sum(sales) END) DESC
@@ -1025,13 +1025,13 @@ Per-Host Resources: mem-estimate=64.10MB mem-reservation=9.88MB thread-reservati
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=150.67MB Threads=38
-Per-Host Resource Estimates: Memory=684MB
+Max Per-Host Resource Reservation: Memory=154.67MB Threads=38
+Per-Host Resource Estimates: Memory=688MB
 F24:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=34.38KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(32,30,31) WHEN 30 THEN channel WHEN 31 THEN channel WHEN 32 THEN NULL END, CASE valid_tid(32,30,31) WHEN 30 THEN id WHEN 31 THEN NULL WHEN 32 THEN NULL END, aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(sales) WHEN 31 THEN sum(sales) WHEN 32 THEN sum(sales) END), aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(`returns`) WHEN 31 THEN sum(`returns`) WHEN 32 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 64:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(32,30,31) WHEN 30 THEN channel WHEN 31 THEN channel WHEN 32 THEN NULL END ASC, CASE valid_tid(32,30,31) WHEN 30 THEN id WHEN 31 THEN NULL WHEN 32 THEN NULL END ASC, aggif(valid_tid(32,30,31) IN (30, 31, 32), CASE valid_tid(32,30,31) WHEN 30 THEN sum(sales) WHEN 31 THEN sum(sales) WHEN 32 THEN sum(sales) END) DESC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test
index b847c30..a3c4e99 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q78.test
@@ -82,7 +82,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=910.94MB mem-reservation=126.44MB thread-reservation=10 runtime-filters-memory=9.00MB
 PLAN-ROOT SINK
 |  output exprs: ss_sold_year, ss_item_sk, ss_customer_sk, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2), ss_qty, ss_wc, ss_sp, coalesce(ws_qty, 0) + coalesce(cs_qty, 0), coalesce(ws_wc, 0) + coalesce(cs_wc, 0), coalesce(ws_sp, 0) + coalesce(cs_sp, 0)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 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
@@ -285,13 +285,13 @@ PLAN-ROOT SINK
    tuple-ids=10 row-size=36B cardinality=1.44M
    in pipelines: 12(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=156.31MB Threads=22
-Per-Host Resource Estimates: Memory=1.00GB
+Max Per-Host Resource Reservation: Memory=160.31MB Threads=22
+Per-Host Resource Estimates: Memory=1.01GB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=37.63KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ss_sold_year, ss_item_sk, ss_customer_sk, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2), ss_qty, ss_wc, ss_sp, coalesce(ws_qty, 0) + coalesce(cs_qty, 0), coalesce(ws_wc, 0) + coalesce(cs_wc, 0), coalesce(ws_sp, 0) + coalesce(cs_sp, 0)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 35:MERGING-EXCHANGE [UNPARTITIONED]
 |  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
@@ -601,13 +601,13 @@ Per-Host Resources: mem-estimate=357.92MB mem-reservation=34.69MB thread-reserva
    tuple-ids=10 row-size=36B cardinality=1.44M
    in pipelines: 12(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=186.25MB Threads=23
-Per-Host Resource Estimates: Memory=411MB
+Max Per-Host Resource Reservation: Memory=190.25MB Threads=23
+Per-Host Resource Estimates: Memory=415MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=37.63KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ss_sold_year, ss_item_sk, ss_customer_sk, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2), ss_qty, ss_wc, ss_sp, coalesce(ws_qty, 0) + coalesce(cs_qty, 0), coalesce(ws_wc, 0) + coalesce(cs_wc, 0), coalesce(ws_sp, 0) + coalesce(cs_sp, 0)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 35:MERGING-EXCHANGE [UNPARTITIONED]
 |  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
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test
index 2ad1331..e70d29d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q79.test
@@ -48,7 +48,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=297.81MB mem-reservation=19.56MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, substr(s_city, 1, 30), ss_ticket_number, amt, profit
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, c_first_name ASC, substr(s_city, 1, 30) ASC, profit ASC
@@ -156,13 +156,13 @@ PLAN-ROOT SINK
    tuple-ids=6 row-size=40B cardinality=100.00K
    in pipelines: 08(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=25.65MB Threads=13
-Per-Host Resource Estimates: Memory=328MB
+Max Per-Host Resource Reservation: Memory=29.65MB Threads=13
+Per-Host Resource Estimates: Memory=332MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=29.81KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, substr(s_city, 1, 30), ss_ticket_number, amt, profit
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, substr(s_city, 1, 30) ASC, profit ASC
@@ -327,13 +327,13 @@ Per-Host Resources: mem-estimate=132.06MB mem-reservation=16.81MB thread-reserva
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=42.46MB Threads=16
-Per-Host Resource Estimates: Memory=174MB
+Max Per-Host Resource Reservation: Memory=46.46MB Threads=16
+Per-Host Resource Estimates: Memory=178MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=56.75KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, substr(s_city, 1, 30), ss_ticket_number, amt, profit
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, substr(s_city, 1, 30) ASC, profit ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
index 2ca28c3..6d35f6c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q80.test
@@ -99,7 +99,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=612.50MB mem-reservation=52.66MB thread-reservation=7 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(26,27,28) WHEN 26 THEN channel WHEN 27 THEN channel WHEN 28 THEN NULL END, CASE valid_tid(26,27,28) WHEN 26 THEN id WHEN 27 THEN NULL WHEN 28 THEN NULL END, aggif(valid_tid(26,27,28) IN (26, 27, 28), CASE valid_tid(26,27,28) WHEN 26 THEN sum(sales) WHEN 27 THEN sum(sales) WHEN 28 THEN sum(sales) END), aggif(valid_tid(26,27,28) IN (26, 27, 28), CASE valid_tid(26,27,28) WHEN 26 THEN sum(`returns`) WHEN 27 THEN sum(`returns`) WHEN 28 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 39:TOP-N [LIMIT=100]
 |  order by: CASE valid_tid(26,27,28) WHEN 26 THEN channel WHEN 27 THEN channel WHEN 28 THEN NULL END ASC, CASE valid_tid(26,27,28) WHEN 26 THEN id WHEN 27 THEN NULL WHEN 28 THEN NULL END ASC
@@ -485,13 +485,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=130.12MB Threads=39
+Max Per-Host Resource Reservation: Memory=134.12MB Threads=39
 Per-Host Resource Estimates: Memory=1.44GB
 F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=24.61KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(26,27,28) WHEN 26 THEN channel WHEN 27 THEN channel WHEN 28 THEN NULL END, CASE valid_tid(26,27,28) WHEN 26 THEN id WHEN 27 THEN NULL WHEN 28 THEN NULL END, aggif(valid_tid(26,27,28) IN (26, 27, 28), CASE valid_tid(26,27,28) WHEN 26 THEN sum(sales) WHEN 27 THEN sum(sales) WHEN 28 THEN sum(sales) END), aggif(valid_tid(26,27,28) IN (26, 27, 28), CASE valid_tid(26,27,28) WHEN 26 THEN sum(`returns`) WHEN 27 THEN sum(`returns`) WHEN 28 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 63:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(26,27,28) WHEN 26 THEN channel WHEN 27 THEN channel WHEN 28 THEN NULL END ASC, CASE valid_tid(26,27,28) WHEN 26 THEN id WHEN 27 THEN NULL WHEN 28 THEN NULL END ASC
@@ -1054,13 +1054,13 @@ Per-Host Resources: mem-estimate=133.14MB mem-reservation=26.25MB thread-reserva
    tuple-ids=0 row-size=36B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=199.25MB Threads=39
-Per-Host Resource Estimates: Memory=684MB
+Max Per-Host Resource Reservation: Memory=203.25MB Threads=39
+Per-Host Resource Estimates: Memory=688MB
 F23:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=46.88KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.05MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(26,27,28) WHEN 26 THEN channel WHEN 27 THEN channel WHEN 28 THEN NULL END, CASE valid_tid(26,27,28) WHEN 26 THEN id WHEN 27 THEN NULL WHEN 28 THEN NULL END, aggif(valid_tid(26,27,28) IN (26, 27, 28), CASE valid_tid(26,27,28) WHEN 26 THEN sum(sales) WHEN 27 THEN sum(sales) WHEN 28 THEN sum(sales) END), aggif(valid_tid(26,27,28) IN (26, 27, 28), CASE valid_tid(26,27,28) WHEN 26 THEN sum(`returns`) WHEN 27 THEN sum(`returns`) WHEN 28 THEN sum(`returns`) END), [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 63:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: CASE valid_tid(26,27,28) WHEN 26 THEN channel WHEN 27 THEN channel WHEN 28 THEN NULL END ASC, CASE valid_tid(26,27,28) WHEN 26 THEN id WHEN 27 THEN NULL WHEN 28 THEN NULL END ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
index 39df7cb..f2a47fe 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q81.test
@@ -62,7 +62,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=556.81MB mem-reservation=27.12MB thread-reservation=9 runtime-filters-memory=7.00MB
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, c_salutation, c_first_name, c_last_name, ca_street_number, ca_street_name, ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset, ca_location_type, ctr_total_return
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 18:TOP-N [LIMIT=100]
 |  order by: c_customer_id ASC, c_salutation ASC, c_first_name ASC, c_last_name ASC, ca_street_number ASC, ca_street_name ASC, ca_street_type ASC, ca_suite_number ASC, ca_city ASC, ca_county ASC, ca_state ASC, ca_zip ASC, ca_country ASC, ca_gmt_offset ASC, ca_location_type ASC, ctr_total_return ASC
@@ -241,13 +241,13 @@ PLAN-ROOT SINK
    tuple-ids=2 row-size=18B cardinality=50.00K
    in pipelines: 02(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=44.88MB Threads=20
-Per-Host Resource Estimates: Memory=631MB
+Max Per-Host Resource Reservation: Memory=48.88MB Threads=20
+Per-Host Resource Estimates: Memory=635MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=57.97KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, c_salutation, c_first_name, c_last_name, ca_street_number, ca_street_name, ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset, ca_location_type, ctr_total_return
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 32:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_customer_id ASC, c_salutation ASC, c_first_name ASC, c_last_name ASC, ca_street_number ASC, ca_street_name ASC, ca_street_type ASC, ca_suite_number ASC, ca_city ASC, ca_county ASC, ca_state ASC, ca_zip ASC, ca_country ASC, ca_gmt_offset ASC, ca_location_type ASC, ctr_total_return ASC
@@ -526,13 +526,13 @@ Per-Host Resources: mem-estimate=46.57MB mem-reservation=6.19MB thread-reservati
    tuple-ids=2 row-size=18B cardinality=50.00K
    in pipelines: 02(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=63.44MB Threads=19
-Per-Host Resource Estimates: Memory=250MB
+Max Per-Host Resource Reservation: Memory=67.44MB Threads=19
+Per-Host Resource Estimates: Memory=254MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=57.97KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.06MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, c_salutation, c_first_name, c_last_name, ca_street_number, ca_street_name, ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset, ca_location_type, ctr_total_return
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 32:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_customer_id ASC, c_salutation ASC, c_first_name ASC, c_last_name ASC, ca_street_number ASC, ca_street_name ASC, ca_street_type ASC, ca_suite_number ASC, ca_city ASC, ca_county ASC, ca_state ASC, ca_zip ASC, ca_country ASC, ca_gmt_offset ASC, ca_location_type ASC, ctr_total_return ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test
index 7bc1ca3..13e8e7f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q82.test
@@ -30,7 +30,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=281.08MB mem-reservation=56.88MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_current_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC
@@ -120,13 +120,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=16B cardinality=1.17M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=96.31MB Threads=10
-Per-Host Resource Estimates: Memory=512MB
+Max Per-Host Resource Reservation: Memory=100.31MB Threads=10
+Per-Host Resource Estimates: Memory=516MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=36.04KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_current_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
@@ -259,13 +259,13 @@ Per-Host Resources: mem-estimate=236.85MB mem-reservation=55.88MB thread-reserva
    tuple-ids=1 row-size=16B cardinality=1.17M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=105.62MB Threads=10
-Per-Host Resource Estimates: Memory=393MB
+Max Per-Host Resource Reservation: Memory=109.62MB Threads=10
+Per-Host Resource Estimates: Memory=397MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=36.04KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_id, i_item_desc, i_current_price
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test
index f94a0c2..70c939a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q83.test
@@ -77,7 +77,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=606.25MB mem-reservation=47.06MB thread-reservation=16 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: item_id, sr_item_qty, (CAST(sr_item_qty AS DECIMAL(19,0)) * CAST(1.0000 AS DECIMAL(5,4))) / CAST((sr_item_qty + cr_item_qty + wr_item_qty) AS DECIMAL(19,0)) / CAST(3.0000 AS DECIMAL(5,4)) * CAST(100 AS DECIMAL(3,0)), cr_item_qty, (CAST(cr_item_qty AS DECIMAL(19,0)) * CAST(1.0000 AS DECIMAL(5,4))) / CAST((sr_item_qty + cr_item_qty + wr_item_qty) AS DECIMAL(19,0)) / CAST(3.0000 AS DECIMAL(5,4)) * CAST(100 AS DECIMAL(3,0)), wr_item_qty, (CAST(wr_item_qty AS DECIMAL(19,0)) * [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 32:TOP-N [LIMIT=100]
 |  order by: item_id ASC, sr_item_qty ASC
@@ -382,13 +382,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=287.51K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=68.75MB Threads=32
-Per-Host Resource Estimates: Memory=686MB
+Max Per-Host Resource Reservation: Memory=72.75MB Threads=32
+Per-Host Resource Estimates: Memory=690MB
 F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: item_id, sr_item_qty, (CAST(sr_item_qty AS DECIMAL(19,0)) * CAST(1.0000 AS DECIMAL(5,4))) / CAST((sr_item_qty + cr_item_qty + wr_item_qty) AS DECIMAL(19,0)) / CAST(3.0000 AS DECIMAL(5,4)) * CAST(100 AS DECIMAL(3,0)), cr_item_qty, (CAST(cr_item_qty AS DECIMAL(19,0)) * CAST(1.0000 AS DECIMAL(5,4))) / CAST((sr_item_qty + cr_item_qty + wr_item_qty) AS DECIMAL(19,0)) / CAST(3.0000 AS DECIMAL(5,4)) * CAST(100 AS DECIMAL(3,0)), wr_item_qty, (CAST(wr_item_qty AS DECIMAL(19,0)) * [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 51:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: item_id ASC, sr_item_qty ASC
@@ -828,13 +828,13 @@ Per-Host Resources: mem-estimate=94.26MB mem-reservation=13.75MB thread-reservat
    tuple-ids=0 row-size=16B cardinality=287.51K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=101.81MB Threads=31
-Per-Host Resource Estimates: Memory=399MB
+Max Per-Host Resource Reservation: Memory=105.81MB Threads=31
+Per-Host Resource Estimates: Memory=403MB
 F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: item_id, sr_item_qty, (CAST(sr_item_qty AS DECIMAL(19,0)) * CAST(1.0000 AS DECIMAL(5,4))) / CAST((sr_item_qty + cr_item_qty + wr_item_qty) AS DECIMAL(19,0)) / CAST(3.0000 AS DECIMAL(5,4)) * CAST(100 AS DECIMAL(3,0)), cr_item_qty, (CAST(cr_item_qty AS DECIMAL(19,0)) * CAST(1.0000 AS DECIMAL(5,4))) / CAST((sr_item_qty + cr_item_qty + wr_item_qty) AS DECIMAL(19,0)) / CAST(3.0000 AS DECIMAL(5,4)) * CAST(100 AS DECIMAL(3,0)), wr_item_qty, (CAST(wr_item_qty AS DECIMAL(19,0)) * [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 51:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: item_id ASC, sr_item_qty ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
index 9567e47..252a306 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q84.test
@@ -22,7 +22,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=239.88MB mem-reservation=15.90MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, concat(concat(coalesce(c_last_name, ''), ', '), coalesce(c_first_name, ''))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 09:TOP-N [LIMIT=100]
 |  order by: c_customer_id ASC
@@ -121,13 +121,13 @@ PLAN-ROOT SINK
    tuple-ids=4 row-size=4B cardinality=287.51K
    in pipelines: 04(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=18.34MB Threads=11
-Per-Host Resource Estimates: Memory=244MB
+Max Per-Host Resource Reservation: Memory=22.34MB Threads=11
+Per-Host Resource Estimates: Memory=248MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, concat(concat(coalesce(c_last_name, ''), ', '), coalesce(c_first_name, ''))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_customer_id ASC
@@ -263,13 +263,13 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=4.94MB thread-reservati
    tuple-ids=4 row-size=4B cardinality=287.51K
    in pipelines: 04(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=30.09MB Threads=10
-Per-Host Resource Estimates: Memory=112MB
+Max Per-Host Resource Reservation: Memory=34.09MB Threads=10
+Per-Host Resource Estimates: Memory=116MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: c_customer_id, concat(concat(coalesce(c_last_name, ''), ', '), coalesce(c_first_name, ''))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_customer_id ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
index 27f8c28..41c33e8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q85.test
@@ -64,7 +64,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=601.31MB mem-reservation=62.09MB thread-reservation=9 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: substring(r_reason_desc, 1, 20), avg(ws_quantity), avg(wr_refunded_cash), avg(wr_fee)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 16:TOP-N [LIMIT=100]
 |  order by: substring(r_reason_desc, 1, 20) ASC, avg(ws_quantity) ASC, avg(wr_refunded_cash) ASC, avg(wr_fee) ASC
@@ -233,13 +233,13 @@ PLAN-ROOT SINK
    tuple-ids=4 row-size=39B cardinality=181.75K
    in pipelines: 04(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=75.90MB Threads=20
-Per-Host Resource Estimates: Memory=642MB
+Max Per-Host Resource Reservation: Memory=79.90MB Threads=20
+Per-Host Resource Estimates: Memory=646MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: substring(r_reason_desc, 1, 20), avg(ws_quantity), avg(wr_refunded_cash), avg(wr_fee)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: substring(r_reason_desc, 1, 20) ASC, avg(ws_quantity) ASC, avg(wr_refunded_cash) ASC, avg(wr_fee) ASC
@@ -493,13 +493,13 @@ Per-Host Resources: mem-estimate=66.08MB mem-reservation=17.88MB thread-reservat
    tuple-ids=4 row-size=39B cardinality=181.75K
    in pipelines: 04(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=92.40MB Threads=19
-Per-Host Resource Estimates: Memory=226MB
+Max Per-Host Resource Reservation: Memory=96.40MB Threads=19
+Per-Host Resource Estimates: Memory=230MB
 F11:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: substring(r_reason_desc, 1, 20), avg(ws_quantity), avg(wr_refunded_cash), avg(wr_fee)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: substring(r_reason_desc, 1, 20) ASC, avg(ws_quantity) ASC, avg(wr_refunded_cash) ASC, avg(wr_fee) ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test
index 229aa0e..a72c902 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q86.test
@@ -31,7 +31,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=185.88MB mem-reservation=23.69MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN sum(ws_net_paid) WHEN 4 THEN sum(ws_net_paid) WHEN 5 THEN sum(ws_net_paid) END), CASE valid_tid(3,4,5) WHEN 3 THEN i_category WHEN 4 THEN i_category WHEN 5 THEN NULL END, CASE valid_tid(3,4,5) WHEN 3 THEN i_class WHEN 4 THEN NULL WHEN 5 THEN NULL END, CAST(aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) AS SMALLINT) + CAST(aggif(valid_tid(3,4,5 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 09:TOP-N [LIMIT=100]
 |  order by: aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) + aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 1 WHEN 5 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) + aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 1 WHEN 5 THEN 1 END) = 0 THEN CASE valid_tid(3,4,5) WHEN [...]
@@ -125,13 +125,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=44.38MB Threads=9
-Per-Host Resource Estimates: Memory=269MB
+Max Per-Host Resource Reservation: Memory=48.38MB Threads=9
+Per-Host Resource Estimates: Memory=273MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN sum(ws_net_paid) WHEN 4 THEN sum(ws_net_paid) WHEN 5 THEN sum(ws_net_paid) END), CASE valid_tid(3,4,5) WHEN 3 THEN i_category WHEN 4 THEN i_category WHEN 5 THEN NULL END, CASE valid_tid(3,4,5) WHEN 3 THEN i_class WHEN 4 THEN NULL WHEN 5 THEN NULL END, CAST(aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) AS SMALLINT) + CAST(aggif(valid_tid(3,4,5 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) + aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 1 WHEN 5 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) + aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 1 WHEN 5 THEN 1 END) = 0 THEN CASE valid_tid(3,4,5) WHEN [...]
@@ -276,13 +276,13 @@ Per-Host Resources: mem-estimate=132.77MB mem-reservation=19.88MB thread-reserva
    tuple-ids=0 row-size=16B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=50.25MB Threads=8
-Per-Host Resource Estimates: Memory=163MB
+Max Per-Host Resource Reservation: Memory=54.25MB Threads=8
+Per-Host Resource Estimates: Memory=167MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN sum(ws_net_paid) WHEN 4 THEN sum(ws_net_paid) WHEN 5 THEN sum(ws_net_paid) END), CASE valid_tid(3,4,5) WHEN 3 THEN i_category WHEN 4 THEN i_category WHEN 5 THEN NULL END, CASE valid_tid(3,4,5) WHEN 3 THEN i_class WHEN 4 THEN NULL WHEN 5 THEN NULL END, CAST(aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) AS SMALLINT) + CAST(aggif(valid_tid(3,4,5 [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) + aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 1 WHEN 5 THEN 1 END) DESC, CASE WHEN aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 0 WHEN 5 THEN 1 END) + aggif(valid_tid(3,4,5) IN (3, 4, 5), CASE valid_tid(3,4,5) WHEN 3 THEN 0 WHEN 4 THEN 1 WHEN 5 THEN 1 END) = 0 THEN CASE valid_tid(3,4,5) WHEN [...]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test
index 22db382..3d5006c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q87.test
@@ -36,7 +36,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=569.21MB mem-reservation=183.19MB thread-reservation=10 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 20:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -225,13 +225,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=8B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=298.56MB Threads=22
-Per-Host Resource Estimates: Memory=945MB
+Max Per-Host Resource Reservation: Memory=302.56MB Threads=22
+Per-Host Resource Estimates: Memory=949MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 36:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
@@ -528,13 +528,13 @@ Per-Host Resources: mem-estimate=116.36MB mem-reservation=43.19MB thread-reserva
    tuple-ids=0 row-size=8B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=393.12MB Threads=25
-Per-Host Resource Estimates: Memory=746MB
+Max Per-Host Resource Reservation: Memory=397.12MB Threads=25
+Per-Host Resource Estimates: Memory=750MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 36:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test
index a267c1e..e21107e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q88.test
@@ -96,7 +96,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.43GB mem-reservation=69.12MB thread-reservation=33 runtime-filters-memory=10.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*), count(*), count(*), count(*), count(*), count(*), count(*), count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 70:NESTED LOOP JOIN [CROSS JOIN]
 |  mem-estimate=8B mem-reservation=0B thread-reservation=0
@@ -765,13 +765,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=69.12MB Threads=72
+Max Per-Host Resource Reservation: Memory=73.12MB Threads=72
 Per-Host Resource Estimates: Memory=1.59GB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.13MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.11MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*), count(*), count(*), count(*), count(*), count(*), count(*), count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 70:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  mem-estimate=8B mem-reservation=0B thread-reservation=0
@@ -1761,13 +1761,13 @@ Per-Host Resources: mem-estimate=65.81MB mem-reservation=8.81MB thread-reservati
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=133.62MB Threads=79
-Per-Host Resource Estimates: Memory=994MB
+Max Per-Host Resource Reservation: Memory=137.62MB Threads=79
+Per-Host Resource Estimates: Memory=998MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*), count(*), count(*), count(*), count(*), count(*), count(*), count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 70:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  join table id: 00
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test
index 04bae43..73a18cf 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q89.test
@@ -31,7 +31,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=220.81MB mem-reservation=21.88MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: i_category, i_class, i_brand, s_store_name, s_company_name, d_moy, sum_sales, avg_monthly_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:TOP-N [LIMIT=100]
 |  order by: sum_sales - avg_monthly_sales ASC, s_store_name ASC
@@ -135,13 +135,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=30.52MB Threads=10
-Per-Host Resource Estimates: Memory=253MB
+Max Per-Host Resource Reservation: Memory=34.52MB Threads=10
+Per-Host Resource Estimates: Memory=257MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=45.81KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_category, i_class, i_brand, s_store_name, s_company_name, d_moy, sum_sales, avg_monthly_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC, s_store_name ASC
@@ -289,13 +289,13 @@ Per-Host Resources: mem-estimate=66.89MB mem-reservation=11.81MB thread-reservat
    tuple-ids=1 row-size=20B cardinality=2.88M
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=60.27MB Threads=11
-Per-Host Resource Estimates: Memory=170MB
+Max Per-Host Resource Reservation: Memory=64.27MB Threads=11
+Per-Host Resource Estimates: Memory=174MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=87.15KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.09MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_category, i_class, i_brand, s_store_name, s_company_name, d_moy, sum_sales, avg_monthly_sales
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC, s_store_name ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test
index 530765a..2a03384 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q90.test
@@ -33,7 +33,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=401.62MB mem-reservation=26.78MB thread-reservation=9 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: CAST(amc AS DECIMAL(15,4)) / CAST(pmc AS DECIMAL(15,4))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:TOP-N [LIMIT=100]
 |  order by: CAST(amc AS DECIMAL(15,4)) / CAST(pmc AS DECIMAL(15,4)) ASC
@@ -207,13 +207,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=26.78MB Threads=18
+Max Per-Host Resource Reservation: Memory=30.78MB Threads=18
 Per-Host Resource Estimates: Memory=442MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.03MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CAST(amc AS DECIMAL(15,4)) / CAST(pmc AS DECIMAL(15,4))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:TOP-N [LIMIT=100]
 |  order by: CAST(amc AS DECIMAL(15,4)) / CAST(pmc AS DECIMAL(15,4)) ASC
@@ -462,13 +462,13 @@ Per-Host Resources: mem-estimate=114.81MB mem-reservation=12.81MB thread-reserva
    tuple-ids=0 row-size=12B cardinality=719.38K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=44.41MB Threads=17
+Max Per-Host Resource Reservation: Memory=48.41MB Threads=17
 Per-Host Resource Estimates: Memory=236MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: CAST(amc AS DECIMAL(15,4)) / CAST(pmc AS DECIMAL(15,4))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:TOP-N [LIMIT=100]
 |  order by: CAST(amc AS DECIMAL(15,4)) / CAST(pmc AS DECIMAL(15,4)) ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
index 103036d..056d9e2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q91.test
@@ -37,7 +37,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=400.62MB mem-reservation=29.47MB thread-reservation=8 runtime-filters-memory=6.00MB
 PLAN-ROOT SINK
 |  output exprs: cc_call_center_id, cc_name, cc_manager, sum(cr_net_loss)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 14:SORT
 |  order by: sum(cr_net_loss) DESC
@@ -182,13 +182,13 @@ PLAN-ROOT SINK
    tuple-ids=1 row-size=16B cardinality=144.07K
    in pipelines: 01(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=49.34MB Threads=16
-Per-Host Resource Estimates: Memory=439MB
+Max Per-Host Resource Reservation: Memory=53.34MB Threads=16
+Per-Host Resource Estimates: Memory=443MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=35.19KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: cc_call_center_id, cc_name, cc_manager, sum(cr_net_loss)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(cr_net_loss) DESC
@@ -397,13 +397,13 @@ Per-Host Resources: mem-estimate=115.10MB mem-reservation=12.81MB thread-reserva
    tuple-ids=1 row-size=16B cardinality=144.07K
    in pipelines: 01(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=65.91MB Threads=15
-Per-Host Resource Estimates: Memory=191MB
+Max Per-Host Resource Reservation: Memory=69.91MB Threads=15
+Per-Host Resource Estimates: Memory=195MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=35.19KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.03MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: cc_call_center_id, cc_name, cc_manager, sum(cr_net_loss)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(cr_net_loss) DESC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test
index 4788fb1..3925da2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q92.test
@@ -24,7 +24,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=295.88MB mem-reservation=28.75MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: sum(ws_ext_discount_amt)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:TOP-N [LIMIT=100]
 |  order by: sum(ws_ext_discount_amt) ASC
@@ -139,13 +139,13 @@ PLAN-ROOT SINK
    tuple-ids=3 row-size=16B cardinality=719.38K
    in pipelines: 03(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=35.94MB Threads=13
+Max Per-Host Resource Reservation: Memory=39.94MB Threads=13
 Per-Host Resource Estimates: Memory=342MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(ws_ext_discount_amt)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:TOP-N [LIMIT=100]
 |  order by: sum(ws_ext_discount_amt) ASC
@@ -321,13 +321,13 @@ Per-Host Resources: mem-estimate=110.15MB mem-reservation=13.94MB thread-reserva
    tuple-ids=3 row-size=16B cardinality=719.38K
    in pipelines: 03(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=41.81MB Threads=12
+Max Per-Host Resource Reservation: Memory=45.81MB Threads=12
 Per-Host Resource Estimates: Memory=172MB
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(ws_ext_discount_amt)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:TOP-N [LIMIT=100]
 |  order by: sum(ws_ext_discount_amt) ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test
index 2fa83f5..7b5acc2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q93.test
@@ -26,7 +26,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=219.44MB mem-reservation=19.45MB thread-reservation=4 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: ss_customer_sk, sum(act_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 06:TOP-N [LIMIT=100]
 |  order by: sum(act_sales) ASC, ss_customer_sk ASC
@@ -91,13 +91,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=24.39MB Threads=8
-Per-Host Resource Estimates: Memory=248MB
+Max Per-Host Resource Reservation: Memory=28.39MB Threads=8
+Per-Host Resource Estimates: Memory=252MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ss_customer_sk, sum(act_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(act_sales) ASC, ss_customer_sk ASC
@@ -199,13 +199,13 @@ Per-Host Resources: mem-estimate=108.06MB mem-reservation=17.44MB thread-reserva
    tuple-ids=0 row-size=28B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=42.77MB Threads=9
-Per-Host Resource Estimates: Memory=143MB
+Max Per-Host Resource Reservation: Memory=46.77MB Threads=9
+Per-Host Resource Estimates: Memory=147MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: ss_customer_sk, sum(act_sales)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(act_sales) ASC, ss_customer_sk ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
index e1bfc7d..11ad01b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q94.test
@@ -32,7 +32,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=409.81MB mem-reservation=30.25MB thread-reservation=7 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: count(ws_order_number), sum(ws_ext_ship_cost), sum(ws_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:TOP-N [LIMIT=100]
 |  order by: count(ws_order_number) ASC
@@ -162,13 +162,13 @@ PLAN-ROOT SINK
    tuple-ids=6 row-size=8B cardinality=71.76K
    in pipelines: 05(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=37.39MB Threads=15
+Max Per-Host Resource Reservation: Memory=41.39MB Threads=15
 Per-Host Resource Estimates: Memory=442MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(ws_order_number), sum(ws_ext_ship_cost), sum(ws_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:TOP-N [LIMIT=100]
 |  order by: count(ws_order_number) ASC
@@ -360,13 +360,13 @@ Per-Host Resources: mem-estimate=65.00MB mem-reservation=3.00MB thread-reservati
    tuple-ids=4 row-size=12B cardinality=719.38K
    in pipelines: 04(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=43.27MB Threads=14
+Max Per-Host Resource Reservation: Memory=47.27MB Threads=14
 Per-Host Resource Estimates: Memory=181MB
 F08:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(ws_order_number), sum(ws_ext_ship_cost), sum(ws_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:TOP-N [LIMIT=100]
 |  order by: count(ws_order_number) ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
index 7072b36..1091a1e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q95.test
@@ -35,7 +35,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=609.75MB mem-reservation=76.19MB thread-reservation=10 runtime-filters-memory=8.00MB
 PLAN-ROOT SINK
 |  output exprs: count(ws_order_number), sum(ws_ext_ship_cost), sum(ws_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 21:TOP-N [LIMIT=100]
 |  order by: count(ws_order_number) ASC
@@ -236,13 +236,13 @@ PLAN-ROOT SINK
    tuple-ids=9 row-size=12B cardinality=719.38K
    in pipelines: 08(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=74.33MB Threads=21
+Max Per-Host Resource Reservation: Memory=78.33MB Threads=21
 Per-Host Resource Estimates: Memory=644MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(ws_order_number), sum(ws_ext_ship_cost), sum(ws_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 21:TOP-N [LIMIT=100]
 |  order by: count(ws_order_number) ASC
@@ -526,13 +526,13 @@ Per-Host Resources: mem-estimate=67.00MB mem-reservation=5.00MB thread-reservati
    tuple-ids=9 row-size=12B cardinality=719.38K
    in pipelines: 08(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=84.08MB Threads=20
+Max Per-Host Resource Reservation: Memory=88.08MB Threads=20
 Per-Host Resource Estimates: Memory=328MB
 F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(ws_order_number), sum(ws_ext_ship_cost), sum(ws_net_profit)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 21:TOP-N [LIMIT=100]
 |  order by: count(ws_order_number) ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test
index 554360f..25590d4 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q96.test
@@ -20,7 +20,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=168.81MB mem-reservation=10.39MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 07:AGGREGATE [FINALIZE]
 |  output: count(*)
@@ -103,13 +103,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=10.39MB Threads=9
-Per-Host Resource Estimates: Memory=189MB
+Max Per-Host Resource Reservation: Memory=14.39MB Threads=9
+Per-Host Resource Estimates: Memory=193MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 12:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
@@ -226,13 +226,13 @@ Per-Host Resources: mem-estimate=66.81MB mem-reservation=9.81MB thread-reservati
    tuple-ids=0 row-size=12B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=20.20MB Threads=9
-Per-Host Resource Estimates: Memory=128MB
+Max Per-Host Resource Reservation: Memory=24.20MB Threads=9
+Per-Host Resource Estimates: Memory=132MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: count(*)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 12:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test
index 5290ec5..8a9ee13 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q97.test
@@ -41,7 +41,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=252.13MB mem-reservation=106.94MB thread-reservation=5 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL THEN 1 ELSE 0 END), sum(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END), sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 09:AGGREGATE [FINALIZE]
 |  output: sum(CAST(CASE WHEN ss_customer_sk IS NOT NULL AND cs_bill_customer_sk IS NULL THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN ss_customer_sk IS NULL AND cs_bill_customer_sk IS NOT NULL THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT)), sum(CAST(CASE WHEN ss_customer_sk IS NOT NULL AND cs_bill_customer_sk IS NOT NULL THEN CAST(1 AS TINYINT) ELSE CAST(0 AS TINYINT) END AS BIGINT))
@@ -134,13 +134,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=122.38MB Threads=10
-Per-Host Resource Estimates: Memory=377MB
+Max Per-Host Resource Reservation: Memory=126.38MB Threads=10
+Per-Host Resource Estimates: Memory=381MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL THEN 1 ELSE 0 END), sum(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END), sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:AGGREGATE [FINALIZE]
 |  output: sum:merge(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL THEN 1 ELSE 0 END), sum:merge(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END), sum:merge(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END)
@@ -286,13 +286,13 @@ Per-Host Resources: mem-estimate=69.00MB mem-reservation=37.94MB thread-reservat
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=130.25MB Threads=12
-Per-Host Resource Estimates: Memory=270MB
+Max Per-Host Resource Reservation: Memory=134.25MB Threads=12
+Per-Host Resource Estimates: Memory=274MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL THEN 1 ELSE 0 END), sum(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END), sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END)
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:AGGREGATE [FINALIZE]
 |  output: sum:merge(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL THEN 1 ELSE 0 END), sum:merge(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END), sum:merge(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL THEN 1 ELSE 0 END)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test
index bdba375..8c3855b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q98.test
@@ -36,7 +36,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=221.46MB mem-reservation=52.00MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: i_item_desc, i_category, i_class, i_current_price, sum(ss_ext_sales_price), sum(ss_ext_sales_price) * CAST(100 AS DECIMAL(3,0)) / sum(sum(ss_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 08:TOP-N [LIMIT=1000]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) ASC
@@ -117,13 +117,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=93.88MB Threads=8
-Per-Host Resource Estimates: Memory=366MB
+Max Per-Host Resource Reservation: Memory=97.88MB Threads=8
+Per-Host Resource Estimates: Memory=370MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=708.68KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_desc, i_category, i_class, i_current_price, sum(ss_ext_sales_price), sum(ss_ext_sales_price) * CAST(100 AS DECIMAL(3,0)) / sum(sum(ss_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) ASC
@@ -241,13 +241,13 @@ Per-Host Resources: mem-estimate=129.05MB mem-reservation=40.88MB thread-reserva
    tuple-ids=0 row-size=16B cardinality=2.88M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=184.75MB Threads=9
-Per-Host Resource Estimates: Memory=294MB
+Max Per-Host Resource Reservation: Memory=188.75MB Threads=9
+Per-Host Resource Estimates: Memory=298MB
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=1.32MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=5.32MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: i_item_desc, i_category, i_class, i_current_price, sum(ss_ext_sales_price), sum(ss_ext_sales_price) * CAST(100 AS DECIMAL(3,0)) / sum(sum(ss_ext_sales_price))
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test
index d3a547b..4773e65 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q99.test
@@ -51,7 +51,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=379.75MB mem-reservation=17.69MB thread-reservation=6 runtime-filters-memory=4.00MB
 PLAN-ROOT SINK
 |  output exprs: w_substr, sm_type, lower(cc_name), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 30) AND (cs_ship_date_sk - cs_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 60) AND (cs_ship_date_sk - cs_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 90) AND (cs_ship_date_sk - cs_sold_date_sk <= 120) THEN 1 ELSE 0 END),  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 10:TOP-N [LIMIT=100]
 |  order by: w_substr ASC, sm_type ASC, lower(cc_name) ASC
@@ -153,13 +153,13 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=20B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=20.23MB Threads=12
-Per-Host Resource Estimates: Memory=400MB
+Max Per-Host Resource Reservation: Memory=24.23MB Threads=12
+Per-Host Resource Estimates: Memory=404MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=20.41KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_substr, sm_type, lower(cc_name), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 30) AND (cs_ship_date_sk - cs_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 60) AND (cs_ship_date_sk - cs_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 90) AND (cs_ship_date_sk - cs_sold_date_sk <= 120) THEN 1 ELSE 0 END),  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_substr ASC, sm_type ASC, lower(cc_name) ASC
@@ -312,13 +312,13 @@ Per-Host Resources: mem-estimate=261.86MB mem-reservation=17.75MB thread-reserva
    tuple-ids=0 row-size=20B cardinality=1.44M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=31.98MB Threads=11
-Per-Host Resource Estimates: Memory=156MB
+Max Per-Host Resource Reservation: Memory=35.98MB Threads=11
+Per-Host Resource Estimates: Memory=160MB
 F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=20.41KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: w_substr, sm_type, lower(cc_name), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 30) AND (cs_ship_date_sk - cs_sold_date_sk <= 60) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 60) AND (cs_ship_date_sk - cs_sold_date_sk <= 90) THEN 1 ELSE 0 END), sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 90) AND (cs_ship_date_sk - cs_sold_date_sk <= 120) THEN 1 ELSE 0 END),  [...]
-|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: w_substr ASC, sm_type ASC, lower(cc_name) ASC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
index 53dda28..a5add55 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
@@ -22,7 +22,7 @@ order by
   l_returnflag,
   l_linestatus
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=13.94MB Threads=2
+Max Per-Host Resource Reservation: Memory=16.00MB Threads=2
 Per-Host Resource Estimates: Memory=274MB
 PLAN-ROOT SINK
 |
@@ -40,8 +40,8 @@ PLAN-ROOT SINK
    predicates: l_shipdate <= '1998-09-02'
    row-size=80B cardinality=600.12K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=23.94MB Threads=4
-Per-Host Resource Estimates: Memory=296MB
+Max Per-Host Resource Reservation: Memory=27.94MB Threads=4
+Per-Host Resource Estimates: Memory=300MB
 PLAN-ROOT SINK
 |
 05:MERGING-EXCHANGE [UNPARTITIONED]
@@ -68,8 +68,8 @@ PLAN-ROOT SINK
    predicates: l_shipdate <= '1998-09-02'
    row-size=80B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=47.88MB Threads=5
-Per-Host Resource Estimates: Memory=240MB
+Max Per-Host Resource Reservation: Memory=51.88MB Threads=5
+Per-Host Resource Estimates: Memory=244MB
 PLAN-ROOT SINK
 |
 05:MERGING-EXCHANGE [UNPARTITIONED]
@@ -241,8 +241,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpch.partsupp.ps_partkey, RF008 -> ps_suppkey
    row-size=24B cardinality=800.00K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=65.28MB Threads=21
-Per-Host Resource Estimates: Memory=662MB
+Max Per-Host Resource Reservation: Memory=69.28MB Threads=21
+Per-Host Resource Estimates: Memory=666MB
 PLAN-ROOT SINK
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
@@ -368,8 +368,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpch.partsupp.ps_partkey, RF008 -> ps_suppkey
    row-size=24B cardinality=800.00K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=82.84MB Threads=20
-Per-Host Resource Estimates: Memory=376MB
+Max Per-Host Resource Reservation: Memory=86.84MB Threads=20
+Per-Host Resource Estimates: Memory=380MB
 PLAN-ROOT SINK
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
@@ -593,8 +593,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> l_orderkey
    row-size=46B cardinality=600.12K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=41.38MB Threads=8
-Per-Host Resource Estimates: Memory=545MB
+Max Per-Host Resource Reservation: Memory=45.38MB Threads=8
+Per-Host Resource Estimates: Memory=549MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -648,8 +648,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> l_orderkey
    row-size=46B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=64.75MB Threads=9
-Per-Host Resource Estimates: Memory=369MB
+Max Per-Host Resource Reservation: Memory=68.75MB Threads=9
+Per-Host Resource Estimates: Memory=373MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -764,8 +764,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> l_orderkey
    row-size=52B cardinality=600.12K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=38.69MB Threads=7
-Per-Host Resource Estimates: Memory=493MB
+Max Per-Host Resource Reservation: Memory=42.69MB Threads=7
+Per-Host Resource Estimates: Memory=497MB
 PLAN-ROOT SINK
 |
 09:MERGING-EXCHANGE [UNPARTITIONED]
@@ -807,8 +807,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> l_orderkey
    row-size=52B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=64.62MB Threads=10
-Per-Host Resource Estimates: Memory=365MB
+Max Per-Host Resource Reservation: Memory=68.62MB Threads=10
+Per-Host Resource Estimates: Memory=369MB
 PLAN-ROOT SINK
 |
 09:MERGING-EXCHANGE [UNPARTITIONED]
@@ -950,8 +950,8 @@ PLAN-ROOT SINK
    runtime filters: RF005 -> l_suppkey, RF010 -> l_orderkey
    row-size=32B cardinality=6.00M
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=70.14MB Threads=14
-Per-Host Resource Estimates: Memory=667MB
+Max Per-Host Resource Reservation: Memory=74.14MB Threads=14
+Per-Host Resource Estimates: Memory=671MB
 PLAN-ROOT SINK
 |
 20:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1039,8 +1039,8 @@ PLAN-ROOT SINK
    runtime filters: RF005 -> l_suppkey, RF010 -> l_orderkey
    row-size=32B cardinality=6.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=113.27MB Threads=15
-Per-Host Resource Estimates: Memory=462MB
+Max Per-Host Resource Reservation: Memory=117.27MB Threads=15
+Per-Host Resource Estimates: Memory=466MB
 PLAN-ROOT SINK
 |
 20:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1173,8 +1173,8 @@ PLAN-ROOT SINK
    predicates: l_discount <= 0.07, l_discount >= 0.05, l_quantity < 24, l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
    row-size=46B cardinality=600.12K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=284MB
+Max Per-Host Resource Reservation: Memory=12.00MB Threads=3
+Per-Host Resource Estimates: Memory=288MB
 PLAN-ROOT SINK
 |
 03:AGGREGATE [FINALIZE]
@@ -1192,8 +1192,8 @@ PLAN-ROOT SINK
    predicates: l_discount <= 0.07, l_discount >= 0.05, l_quantity < 24, l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
    row-size=46B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=16.00MB Threads=3
-Per-Host Resource Estimates: Memory=206MB
+Max Per-Host Resource Reservation: Memory=20.00MB Threads=3
+Per-Host Resource Estimates: Memory=210MB
 PLAN-ROOT SINK
 |
 03:AGGREGATE [FINALIZE]
@@ -1322,8 +1322,8 @@ PLAN-ROOT SINK
    runtime filters: RF006 -> l_suppkey, RF008 -> l_orderkey
    row-size=54B cardinality=600.12K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=68.20MB Threads=16
-Per-Host Resource Estimates: Memory=682MB
+Max Per-Host Resource Reservation: Memory=72.20MB Threads=16
+Per-Host Resource Estimates: Memory=686MB
 PLAN-ROOT SINK
 |
 22:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1416,8 +1416,8 @@ PLAN-ROOT SINK
    runtime filters: RF006 -> l_suppkey, RF008 -> l_orderkey
    row-size=54B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=102.89MB Threads=21
-Per-Host Resource Estimates: Memory=494MB
+Max Per-Host Resource Reservation: Memory=106.89MB Threads=21
+Per-Host Resource Estimates: Memory=498MB
 PLAN-ROOT SINK
 |
 22:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1658,8 +1658,8 @@ PLAN-ROOT SINK
    runtime filters: RF004 -> c_nationkey, RF006 -> c_custkey
    row-size=10B cardinality=150.00K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=71.46MB Threads=21
-Per-Host Resource Estimates: Memory=756MB
+Max Per-Host Resource Reservation: Memory=75.46MB Threads=21
+Per-Host Resource Estimates: Memory=760MB
 PLAN-ROOT SINK
 |
 29:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1776,8 +1776,8 @@ PLAN-ROOT SINK
    runtime filters: RF008 -> l_suppkey, RF010 -> l_orderkey, RF012 -> l_partkey
    row-size=40B cardinality=6.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=106.02MB Threads=28
-Per-Host Resource Estimates: Memory=504MB
+Max Per-Host Resource Reservation: Memory=110.02MB Threads=28
+Per-Host Resource Estimates: Memory=508MB
 PLAN-ROOT SINK
 |
 29:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2022,8 +2022,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> l_partkey, RF003 -> l_suppkey, RF006 -> l_suppkey, RF008 -> l_orderkey, RF010 -> l_partkey
    row-size=48B cardinality=6.00M
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=123.70MB Threads=15
-Per-Host Resource Estimates: Memory=879MB
+Max Per-Host Resource Reservation: Memory=127.70MB Threads=15
+Per-Host Resource Estimates: Memory=883MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2110,8 +2110,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> l_partkey, RF003 -> l_suppkey, RF006 -> l_suppkey, RF008 -> l_orderkey, RF010 -> l_partkey
    row-size=48B cardinality=6.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=193.38MB Threads=18
-Per-Host Resource Estimates: Memory=647MB
+Max Per-Host Resource Reservation: Memory=197.38MB Threads=18
+Per-Host Resource Estimates: Memory=651MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2302,8 +2302,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> c_nationkey, RF002 -> c_custkey
    row-size=197B cardinality=150.00K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=124.45MB Threads=11
-Per-Host Resource Estimates: Memory=668MB
+Max Per-Host Resource Reservation: Memory=128.45MB Threads=11
+Per-Host Resource Estimates: Memory=672MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2370,8 +2370,8 @@ PLAN-ROOT SINK
    runtime filters: RF004 -> l_orderkey
    row-size=37B cardinality=2.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=144.88MB Threads=14
-Per-Host Resource Estimates: Memory=484MB
+Max Per-Host Resource Reservation: Memory=148.88MB Threads=14
+Per-Host Resource Estimates: Memory=488MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2557,8 +2557,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> ps_suppkey
    row-size=28B cardinality=800.00K
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=43.70MB Threads=15
-Per-Host Resource Estimates: Memory=541MB
+Max Per-Host Resource Reservation: Memory=47.70MB Threads=15
+Per-Host Resource Estimates: Memory=545MB
 PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2654,8 +2654,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> ps_suppkey
    row-size=28B cardinality=800.00K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=53.45MB Threads=14
-Per-Host Resource Estimates: Memory=311MB
+Max Per-Host Resource Reservation: Memory=57.45MB Threads=14
+Per-Host Resource Estimates: Memory=315MB
 PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2830,8 +2830,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> o_orderkey
    row-size=28B cardinality=1.50M
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=50.94MB Threads=7
-Per-Host Resource Estimates: Memory=511MB
+Max Per-Host Resource Reservation: Memory=54.94MB Threads=7
+Per-Host Resource Estimates: Memory=515MB
 PLAN-ROOT SINK
 |
 09:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2872,8 +2872,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> l_orderkey
    row-size=90B cardinality=320.78K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=75.88MB Threads=10
... 1369 lines suppressed ...