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

[impala] branch master updated: IMPALA-9156: share broadcast join builds

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 08acccf  IMPALA-9156: share broadcast join builds
08acccf is described below

commit 08acccf9ebb13445ab7e3d5d2cd6ceb122bde1f3
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Sat Dec 28 16:12:05 2019 -0800

    IMPALA-9156: share broadcast join builds
    
    The scheduler will only create one join build finstance per
    backend in cases where this is supported.
    
    The builder is aware of the number of finstances executing the
    probe and hands off the build data structures to the builders.
    
    Nested loop join requires minimal modifications because the
    build data structures are read-only after initial construction.
    The only significant change is that memory can't be transferred
    to the multiple consumers, so MarkNeedsDeepCopy() needs to be
    used instead.
    
    Hash join requires additional synchronisation because the
    spilling algorithm mutates build-side data structures. This
    patch adds synchronisation so that rebuilding spilled
    partitions is done in a thread-safe manner, using a single
    thread. This uses the CyclicBarrier added in an earlier patch.
    
    Threads blocked on CyclicBarrier need to be cancellable,
    which is handled by cancelling the barrier when cancelling
    fragments on the backend.
    
    BufferPool now correctly handles multiple threads calling
    CleanPages() concurrently, which makes other methods thread-safe.
    
    Update planner to cost broadcast join and estimate memory
    consumption based on a single instance per node.
    
    Planner estimates of number of instances are improved. Instead of
    assuming mt_dop instances per node, use the total number of input
    splits (also called scan ranges in places) as an upper bound on
    the number of instances generated by scans. These instance
    estimates from the scan nodes are then propagated up the
    plan tree in the same way as the numNodes estimates. The instance
    estimate for the join build fragment is fixed to be based on
    the destination fragment.
    
    The profile now correctly accounts for time waiting for the
    builder, counting it in inactive time and showing it in the
    node timeline. Additional improvements/cleanup to the time
    accounting are deferring until IMPALA-9422.
    
    Testing:
    * Updated planner tests
    * Ran a single node stress test with TPC-H and TPC-DS
    * Add a targeted test for spilling broadcast joins, both repartitioning
      and not repartitioning.
    * Add a targeted test for a spilling broadcast join with empty probe
    * Add a targeted test for spilling broadcast join with empty build
      partitions.
    * Add a broadcast join to test_cancellation and test_failpoints.
    
    Perf:
    
    I did a single node run on my desktop:
    +----------+-----------------------+---------+------------+------------+----------------+
    | Workload | File Format           | Avg (s) | Delta(Avg) | GeoMean(s) | Delta(GeoMean) |
    +----------+-----------------------+---------+------------+------------+----------------+
    | TPCH(30) | parquet / none / none | 6.26    | -15.70%    | 4.63       | -16.16%        |
    +----------+-----------------------+---------+------------+------------+----------------+
    
    +----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------+----------------+---------+---------+
    | Workload | Query    | File Format           | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%) | Base StdDev(%) | Iters | Median Diff(%) | MW Zval | Tval    |
    +----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------+----------------+---------+---------+
    | TPCH(30) | TPCH-Q21 | parquet / none / none | 24.97  | 23.25       | R +7.38%   |   0.51%   |   0.22%        | 5     | R +6.95%       | 2.31    | 27.93   |
    | TPCH(30) | TPCH-Q4  | parquet / none / none | 2.83   | 2.79        |   +1.31%   |   1.86%   |   0.36%        | 5     |   +1.88%       | 1.15    | 1.53    |
    | TPCH(30) | TPCH-Q6  | parquet / none / none | 1.28   | 1.28        |   -0.01%   |   1.64%   |   1.63%        | 5     |   -0.11%       | -0.58   | -0.01   |
    | TPCH(30) | TPCH-Q22 | parquet / none / none | 2.65   | 2.68        |   -0.94%   |   0.84%   |   1.46%        | 5     |   -0.21%       | -0.87   | -1.25   |
    | TPCH(30) | TPCH-Q1  | parquet / none / none | 4.69   | 4.72        |   -0.56%   |   1.29%   |   0.52%        | 5     |   -1.04%       | -1.15   | -0.89   |
    | TPCH(30) | TPCH-Q13 | parquet / none / none | 10.64  | 10.80       |   -1.48%   |   0.61%   |   0.60%        | 5     |   -1.39%       | -1.73   | -3.91   |
    | TPCH(30) | TPCH-Q15 | parquet / none / none | 4.11   | 4.32        |   -4.92%   |   0.05%   |   0.40%        | 5     |   -4.93%       | -2.31   | -27.46  |
    | TPCH(30) | TPCH-Q20 | parquet / none / none | 3.47   | 3.67        | I -5.41%   |   0.81%   |   0.03%        | 5     | I -5.70%       | -2.31   | -15.75  |
    | TPCH(30) | TPCH-Q17 | parquet / none / none | 7.58   | 8.14        | I -6.93%   |   3.13%   |   2.62%        | 5     | I -9.31%       | -2.02   | -3.96   |
    | TPCH(30) | TPCH-Q9  | parquet / none / none | 15.59  | 17.02       | I -8.38%   |   0.95%   |   0.43%        | 5     | I -8.92%       | -2.31   | -19.37  |
    | TPCH(30) | TPCH-Q14 | parquet / none / none | 2.90   | 3.25        | I -10.93%  |   1.42%   |   4.41%        | 5     | I -10.28%      | -2.31   | -5.33   |
    | TPCH(30) | TPCH-Q12 | parquet / none / none | 2.69   | 3.13        | I -14.31%  |   4.50%   |   1.40%        | 5     | I -17.79%      | -2.31   | -7.80   |
    | TPCH(30) | TPCH-Q16 | parquet / none / none | 2.50   | 3.03        | I -17.54%  |   0.10%   |   0.79%        | 5     | I -20.55%      | -2.31   | -49.31  |
    | TPCH(30) | TPCH-Q10 | parquet / none / none | 4.76   | 5.92        | I -19.52%  |   0.78%   |   0.33%        | 5     | I -24.31%      | -2.31   | -61.63  |
    | TPCH(30) | TPCH-Q2  | parquet / none / none | 2.56   | 3.33        | I -23.18%  |   2.13%   |   0.85%        | 5     | I -30.39%      | -2.31   | -28.14  |
    | TPCH(30) | TPCH-Q18 | parquet / none / none | 12.59  | 16.41       | I -23.26%  |   1.73%   |   0.90%        | 5     | I -30.43%      | -2.31   | -32.36  |
    | TPCH(30) | TPCH-Q11 | parquet / none / none | 1.83   | 2.41        | I -24.04%  |   1.83%   |   2.22%        | 5     | I -30.48%      | -2.31   | -20.54  |
    | TPCH(30) | TPCH-Q8  | parquet / none / none | 4.43   | 5.94        | I -25.33%  |   0.96%   |   0.54%        | 5     | I -34.54%      | -2.31   | -63.01  |
    | TPCH(30) | TPCH-Q5  | parquet / none / none | 3.81   | 5.37        | I -29.08%  |   1.43%   |   0.69%        | 5     | I -41.47%      | -2.31   | -53.11  |
    | TPCH(30) | TPCH-Q7  | parquet / none / none | 13.34  | 21.49       | I -37.92%  |   0.46%   |   0.30%        | 5     | I -60.69%      | -2.31   | -203.08 |
    | TPCH(30) | TPCH-Q3  | parquet / none / none | 4.73   | 7.73        | I -38.81%  |   4.90%   |   1.35%        | 5     | I -61.68%      | -2.31   | -26.40  |
    | TPCH(30) | TPCH-Q19 | parquet / none / none | 3.71   | 6.61        | I -43.83%  |   1.63%   |   0.09%        | 5     | I -77.12%      | -2.31   | -106.61 |
    +----------+----------+-----------------------+--------+-------------+------------+-----------+----------------+-------+----------------+---------+---------+
    
    Change-Id: I4c67e4b2c87ed0fba648f1e1710addb885d66dc7
    Reviewed-on: http://gerrit.cloudera.org:8080/15096
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exec/blocking-join-node.cc                  |  10 +-
 be/src/exec/join-builder.cc                        |  42 +-
 be/src/exec/join-builder.h                         |   8 +-
 be/src/exec/join-op.h                              |   8 +
 be/src/exec/nested-loop-join-node.cc               |  11 +-
 be/src/exec/partitioned-hash-join-builder.cc       | 160 ++++++--
 be/src/exec/partitioned-hash-join-builder.h        |  86 +++-
 be/src/exec/partitioned-hash-join-node.cc          |  40 +-
 be/src/exec/partitioned-hash-join-node.h           |  17 +-
 be/src/runtime/bufferpool/buffer-pool-internal.h   |   4 +
 be/src/runtime/bufferpool/buffer-pool.cc           |  21 +-
 be/src/runtime/bufferpool/buffer-pool.h            |   6 +
 be/src/runtime/coordinator-backend-state.cc        |   3 +
 be/src/runtime/runtime-state.cc                    |  13 +
 be/src/runtime/runtime-state.h                     |  14 +
 be/src/scheduling/query-schedule.h                 |   7 +-
 be/src/scheduling/scheduler.cc                     |  21 +-
 be/src/util/cyclic-barrier-test.cc                 |  27 +-
 be/src/util/cyclic-barrier.h                       |  17 +-
 common/thrift/DataSinks.thrift                     |   4 +
 common/thrift/ImpalaInternalService.thrift         |   4 +
 .../org/apache/impala/planner/AggregationNode.java |   2 +-
 .../apache/impala/planner/DataSourceScanNode.java  |   2 +-
 .../apache/impala/planner/DistributedPlanner.java  |  18 +-
 .../org/apache/impala/planner/EmptySetNode.java    |   2 +-
 .../org/apache/impala/planner/ExchangeNode.java    |   4 +-
 .../org/apache/impala/planner/HBaseScanNode.java   |  16 +-
 .../org/apache/impala/planner/HashJoinNode.java    |  35 +-
 .../org/apache/impala/planner/HdfsScanNode.java    |  71 +++-
 .../org/apache/impala/planner/HdfsTableSink.java   |   2 +-
 .../org/apache/impala/planner/JoinBuildSink.java   |  25 ++
 .../java/org/apache/impala/planner/JoinNode.java   |   8 +
 .../org/apache/impala/planner/KuduScanNode.java    |   9 +-
 .../org/apache/impala/planner/ParallelPlanner.java |   2 +-
 .../org/apache/impala/planner/PlanFragment.java    |  61 ++-
 .../java/org/apache/impala/planner/PlanNode.java   |  25 +-
 .../apache/impala/planner/SingularRowSrcNode.java  |   1 +
 .../java/org/apache/impala/planner/UnionNode.java  |   6 +-
 .../java/org/apache/impala/planner/UnnestNode.java |   1 +
 .../queries/PlannerTest/kudu-selectivity.test      |  12 +-
 .../queries/PlannerTest/max-row-size.test          |  20 +-
 .../PlannerTest/mem-limit-broadcast-join.test      |   8 +-
 ...t-dop-validation-hdfs-num-rows-est-enabled.test |  34 +-
 .../queries/PlannerTest/mt-dop-validation.test     |  54 +--
 .../queries/PlannerTest/resource-requirements.test | 437 ++++++++++-----------
 .../PlannerTest/spillable-buffer-sizing.test       | 233 ++++++-----
 .../queries/PlannerTest/tpcds-all.test             | 184 ++++-----
 .../queries/PlannerTest/tpch-all.test              | 112 +++---
 .../QueryTest/spilling-broadcast-joins.test        | 127 ++++++
 tests/common/test_dimensions.py                    |   6 +
 tests/failure/test_failpoints.py                   |   6 +-
 tests/query_test/test_cancellation.py              |  18 +-
 tests/query_test/test_spilling.py                  |  28 +-
 53 files changed, 1320 insertions(+), 772 deletions(-)

diff --git a/be/src/exec/blocking-join-node.cc b/be/src/exec/blocking-join-node.cc
index ffc8a90..2912e63 100644
--- a/be/src/exec/blocking-join-node.cc
+++ b/be/src/exec/blocking-join-node.cc
@@ -207,7 +207,8 @@ Status BlockingJoinNode::OpenImpl(RuntimeState* state, JoinBuilder** separate_bu
   } else {
     // The integrated join build requires some tricky time accounting because two
     // threads execute concurrently with the time from the left and right child
-    // overlapping. The separate join build does not have this problem, because
+    // overlapping. We also want to count the builder profile as local time.
+    // The separate join build does not have this problem, because
     // the build is executed in a separate fragment with a separate profile tree.
     runtime_profile_->AddLocalTimeCounter(bind<int64_t>(
         &BlockingJoinNode::LocalTimeCounterFn, runtime_profile_->total_time_counter(),
@@ -242,7 +243,12 @@ Status BlockingJoinNode::ProcessBuildInputAndOpenProbe(
     // AcquireResourcesForBuild() opens the buffer pool client, so that probe reservation
     // can be transferred.
     RETURN_IF_ERROR(AcquireResourcesForBuild(state));
-    RETURN_IF_ERROR(build_sink->WaitForInitialBuild(state));
+    {
+      SCOPED_TIMER(runtime_profile_->inactive_timer());
+      events_->MarkEvent("Waiting for builder");
+      RETURN_IF_ERROR(build_sink->WaitForInitialBuild(state));
+      events_->MarkEvent("Initial build available");
+    }
     waited_for_build_ = true;
   } else if (!IsInSubplan() && state->resource_pool()->TryAcquireThreadToken()) {
     // The build is integrated into the join node and we got a thread token. Do the hash
diff --git a/be/src/exec/join-builder.cc b/be/src/exec/join-builder.cc
index 18a0a38..a85a210 100644
--- a/be/src/exec/join-builder.cc
+++ b/be/src/exec/join-builder.cc
@@ -34,7 +34,9 @@ JoinBuilder::JoinBuilder(TDataSinkId sink_id, const JoinBuilderConfig& sink_conf
   : DataSink(sink_id, sink_config, name, state),
     join_node_id_(sink_config.join_node_id_),
     join_op_(sink_config.join_op_),
-    is_separate_build_(sink_id != -1) {}
+    is_separate_build_(sink_id != -1),
+    num_probe_threads_(
+        is_separate_build_ ? state->instance_ctx().num_join_build_outputs : 1) {}
 
 JoinBuilder::~JoinBuilder() {
   DCHECK_EQ(0, probe_refcount_);
@@ -93,20 +95,30 @@ void JoinBuilder::HandoffToProbesAndWait(RuntimeState* build_side_state) {
   DCHECK(is_separate_build_) << "Doesn't make sense for embedded builder.";
   VLOG(2) << "Initial build ready JoinBuilder (id=" << join_node_id_ << ")";
   build_side_state->AddCancellationCV(&build_wakeup_cv_);
-  unique_lock<mutex> l(separate_build_lock_);
-  ready_to_probe_ = true;
-  outstanding_probes_ = 1; // TODO: IMPALA-9156: this will be the number of join nodes.
-  probe_wakeup_cv_.NotifyAll();
-  while (probe_refcount_ > 0
-      || (outstanding_probes_ > 0 && !build_side_state->is_cancelled())) {
-    SCOPED_TIMER(profile_->inactive_timer());
-    build_wakeup_cv_.Wait(l);
+  {
+    unique_lock<mutex> l(separate_build_lock_);
+    ready_to_probe_ = true;
+    outstanding_probes_ = num_probe_threads_;
+    DCHECK_GE(outstanding_probes_, 1);
+    VLOG(3) << "JoinBuilder (id=" << join_node_id_ << ")"
+            << " waiting for " << outstanding_probes_ << " probes.";
+    probe_wakeup_cv_.NotifyAll();
+    while (probe_refcount_ > 0
+        || (outstanding_probes_ > 0 && !build_side_state->is_cancelled())) {
+      SCOPED_TIMER(profile_->inactive_timer());
+      VLOG(3) << "JoinBuilder (id=" << join_node_id_ << ") waiting"
+              << " probe_refcount_=" << probe_refcount_
+              << " outstanding_probes_=" << outstanding_probes_
+              << " cancelled=" << build_side_state->is_cancelled();
+      build_wakeup_cv_.Wait(l);
+    }
+    // Don't let probe side pick up the builder when we're going to clean it up.
+    // Query cancellation will propagate to the probe finstance.
+    ready_to_probe_ = !build_side_state->is_cancelled();
+    VLOG(2) << "JoinBuilder (id=" << join_node_id_ << ") all probes complete. "
+            << " probe_refcount_=" << probe_refcount_
+            << " outstanding_probes_=" << outstanding_probes_
+            << " cancelled=" << build_side_state->is_cancelled();
   }
-  // Don't let probe side pick up the builder when we're going to clean it up.
-  // Query cancellation will propagate to the probe finstance.
-  ready_to_probe_ = !build_side_state->is_cancelled();
-  VLOG(2) << "JoinBuilder (id=" << join_node_id_ << ")"
-          << " all probes complete. cancelled=" << build_side_state->is_cancelled()
-          << " outstanding_probes_=" << outstanding_probes_;
 }
 } // namespace impala
diff --git a/be/src/exec/join-builder.h b/be/src/exec/join-builder.h
index b9e742e..65e63f0 100644
--- a/be/src/exec/join-builder.h
+++ b/be/src/exec/join-builder.h
@@ -136,10 +136,10 @@ class JoinBuilder : public DataSink {
   /// a separate join build, each BlockingJoinNode that called WaitForInitialBuild() needs
   /// to call CloseFromProbe(). The builder is closed when the last BlockingJoinNode using
   /// this builder calls CloseFromProbe(). BlockingJoinNode never calls Close() directly.
-  /// TODO: IMPALA-9156: for now, we only have a 1:1 relationship between builders and
-  /// nodes. This will change for broadcast joins.
   void CloseFromProbe(RuntimeState* join_node_state);
 
+  int num_probe_threads() const { return num_probe_threads_; }
+
  protected:
   /// ID of the join node that this builder is associated with.
   const int join_node_id_;
@@ -173,6 +173,10 @@ class JoinBuilder : public DataSink {
   // Protected by 'separate_build_lock_'.
   bool ready_to_probe_ = false;
 
+  // Total number of probe-side fragment instances, i.e. threads.
+  // Always 1 if 'is_separate_build_' is false.
+  const int num_probe_threads_;
+
   // Number of probe-side threads that are expected to call WaitForInitialBuild()
   // but have not yet called it.
   // Protected by 'separate_build_lock_'.
diff --git a/be/src/exec/join-op.h b/be/src/exec/join-op.h
index ce857a0..cd0c992 100644
--- a/be/src/exec/join-op.h
+++ b/be/src/exec/join-op.h
@@ -45,4 +45,12 @@ inline bool NeedToProcessUnmatchedBuildRows(TJoinOp::type join_op) {
   return join_op == TJoinOp::RIGHT_ANTI_JOIN || join_op == TJoinOp::RIGHT_OUTER_JOIN
       || join_op == TJoinOp::FULL_OUTER_JOIN;
 }
+
+/// Returns true if the join returns references to the build-side data.
+inline bool ReturnsBuildData(TJoinOp::type join_op) {
+  return join_op == TJoinOp::INNER_JOIN || join_op == TJoinOp::LEFT_OUTER_JOIN
+      || join_op == TJoinOp::RIGHT_OUTER_JOIN || join_op == TJoinOp::RIGHT_ANTI_JOIN
+      || join_op == TJoinOp::RIGHT_SEMI_JOIN || join_op == TJoinOp::FULL_OUTER_JOIN
+      || join_op == TJoinOp::CROSS_JOIN;
+}
 } // namespace impala
diff --git a/be/src/exec/nested-loop-join-node.cc b/be/src/exec/nested-loop-join-node.cc
index f01d18c..dc8cc60 100644
--- a/be/src/exec/nested-loop-join-node.cc
+++ b/be/src/exec/nested-loop-join-node.cc
@@ -22,6 +22,7 @@
 
 #include "common/names.h"
 #include "exec/exec-node-util.h"
+#include "exec/join-op.h"
 #include "exprs/scalar-expr-evaluator.h"
 #include "exprs/scalar-expr.h"
 #include "gen-cpp/PlanNodes_types.h"
@@ -283,7 +284,15 @@ end:
   if (eos_) {
     *eos = true;
     probe_batch_->TransferResourceOwnership(output_batch);
-    build_batches_->TransferResourceOwnership(output_batch);
+    if (ReturnsBuildData(join_op_)) {
+      if (builder_->num_probe_threads() == 1) {
+        build_batches_->TransferResourceOwnership(output_batch);
+      } else {
+        // Resources can't be transferred to all NestedLoopJoinNodes sharing the builder,
+        // each finstance must copy out data it needs to hold onto.
+        output_batch->MarkNeedsDeepCopy();
+      }
+    }
   }
   COUNTER_SET(rows_returned_counter_, rows_returned());
   return Status::OK();
diff --git a/be/src/exec/partitioned-hash-join-builder.cc b/be/src/exec/partitioned-hash-join-builder.cc
index 29874f0..2c41545 100644
--- a/be/src/exec/partitioned-hash-join-builder.cc
+++ b/be/src/exec/partitioned-hash-join-builder.cc
@@ -23,8 +23,8 @@
 
 #include "codegen/llvm-codegen.h"
 #include "exec/hash-table.inline.h"
-#include "exprs/scalar-expr.h"
 #include "exprs/scalar-expr-evaluator.h"
+#include "exprs/scalar-expr.h"
 #include "runtime/buffered-tuple-stream.h"
 #include "runtime/exec-env.h"
 #include "runtime/mem-tracker.h"
@@ -34,6 +34,7 @@
 #include "runtime/runtime-filter.h"
 #include "runtime/runtime-state.h"
 #include "util/bloom-filter.h"
+#include "util/cyclic-barrier.h"
 #include "util/min-max-filter.h"
 #include "util/runtime-profile-counters.h"
 
@@ -167,6 +168,9 @@ PhjBuilder::PhjBuilder(
     is_not_distinct_from_(sink_config.is_not_distinct_from_),
     filter_exprs_(sink_config.filter_exprs_),
     hash_table_config_(*sink_config.hash_table_config_),
+    probe_barrier_(num_probe_threads_ <= 1 ?
+            nullptr :
+            make_unique<CyclicBarrier>(state->instance_ctx().num_join_build_outputs)),
     process_build_batch_fn_(sink_config.process_build_batch_fn_),
     process_build_batch_fn_level0_(sink_config.process_build_batch_fn_level0_),
     insert_batch_fn_(
@@ -174,10 +178,18 @@ PhjBuilder::PhjBuilder(
     insert_batch_fn_level0_(reinterpret_cast<const Partition::InsertBatchFn&>(
         sink_config.insert_batch_fn_level0_)) {
   DCHECK_GT(sink_config.hash_seed_, 0);
+  DCHECK(num_probe_threads_ <= 1 || !NeedToProcessUnmatchedBuildRows(join_op_))
+      << "Returning rows with build partitions is not supported with shared builds";
   for (const TRuntimeFilterDesc& filter_desc : sink_config.filter_descs_) {
     filter_ctxs_.emplace_back();
     filter_ctxs_.back().filter = state->filter_bank()->RegisterFilter(filter_desc, true);
   }
+  // Ensure threads get unblocked from probe_barrier_ when the query is cancelled. Using
+  // the AddBarrierToCancel() mechanism ensures that cancellation happens after the
+  // overall error for this backend has already been set in QueryState. Otherwise this
+  // status and the original status could race with each other to become this backend's
+  // status.
+  if (probe_barrier_ != nullptr) state->AddBarrierToCancel(probe_barrier_.get());
 }
 
 PhjBuilder::PhjBuilder(const PhjBuilderConfig& sink_config,
@@ -195,6 +207,7 @@ PhjBuilder::PhjBuilder(const PhjBuilderConfig& sink_config,
     is_not_distinct_from_(sink_config.is_not_distinct_from_),
     filter_exprs_(sink_config.filter_exprs_),
     hash_table_config_(*sink_config.hash_table_config_),
+    probe_barrier_(nullptr),
     process_build_batch_fn_(sink_config.process_build_batch_fn_),
     process_build_batch_fn_level0_(sink_config.process_build_batch_fn_level0_),
     insert_batch_fn_(
@@ -202,6 +215,7 @@ PhjBuilder::PhjBuilder(const PhjBuilderConfig& sink_config,
     insert_batch_fn_level0_(reinterpret_cast<const Partition::InsertBatchFn&>(
         sink_config.insert_batch_fn_level0_)) {
   DCHECK_GT(sink_config.hash_seed_, 0);
+  DCHECK_EQ(1, num_probe_threads_) << "Embedded builders cannot be shared";
   for (const TRuntimeFilterDesc& filter_desc : sink_config.filter_descs_) {
     filter_ctxs_.emplace_back();
     filter_ctxs_.back().filter = state->filter_bank()->RegisterFilter(filter_desc, true);
@@ -419,8 +433,8 @@ void PhjBuilder::UpdateState(HashJoinState next_state) {
   VLOG(2) << "Transitioned State:" << endl << DebugString();
 }
 
-string PhjBuilder::PrintState() const {
-  switch (state_) {
+string PhjBuilder::PrintState(HashJoinState state) {
+  switch (state) {
     case HashJoinState::PARTITIONING_BUILD:
       return "PartitioningBuild";
     case HashJoinState::PARTITIONING_PROBE:
@@ -586,8 +600,9 @@ Status PhjBuilder::BuildHashTablesAndReserveProbeBuffers(HashJoinState next_stat
 
 Status PhjBuilder::ReserveProbeBuffers(HashJoinState next_state) {
   DCHECK_EQ(PARTITION_FANOUT, hash_partitions_.size());
-  int64_t addtl_reservation = CalcProbeStreamReservation(next_state)
-      - probe_stream_reservation_.GetReservation();
+  int64_t curr_reservation = probe_stream_reservation_.GetReservation();
+  int64_t addtl_reservation =
+      CalcProbeStreamReservation(next_state) * num_probe_threads_ - curr_reservation;
 
   // Loop until either we get enough reservation or all partitions are spilled (in which
   // case SpillPartition() returns an error).
@@ -595,9 +610,10 @@ Status PhjBuilder::ReserveProbeBuffers(HashJoinState next_state) {
     Partition* spilled_partition;
     RETURN_IF_ERROR(SpillPartition(
           BufferedTupleStream::UNPIN_ALL, &spilled_partition));
+    // Increase reservation to reflect the additional spilled partition.
     // Don't need to create a probe stream for the null-aware partition.
     if (spilled_partition != null_aware_partition_.get()) {
-      addtl_reservation += spillable_buffer_size_;
+      addtl_reservation += spillable_buffer_size_ * num_probe_threads_;
     }
   }
   VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") saved reservation "
@@ -620,11 +636,10 @@ Status PhjBuilder::TransferProbeStreamReservation(
     BufferPool::ClientHandle* probe_client) {
   DCHECK_EQ(is_separate_build_, probe_client != buffer_pool_client_);
   int64_t probe_reservation = CalcProbeStreamReservation(state_);
-  VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") transfer " << probe_reservation
+  VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") will transfer " << probe_reservation
           << " to probe client.";
   if (probe_reservation == 0) return Status::OK();
-  int64_t saved_reservation = probe_stream_reservation_.GetReservation();
-  DCHECK_GE(saved_reservation, probe_reservation);
+  DCHECK_GE(probe_stream_reservation_.GetReservation(), probe_reservation);
 
   buffer_pool_client_->RestoreReservation(&probe_stream_reservation_, probe_reservation);
   if (is_separate_build_) {
@@ -681,6 +696,37 @@ Status PhjBuilder::DoneProbingHashPartitions(
   // 'spilled_partitions_', which the calculation depends on.
   int64_t probe_reservation = CalcProbeStreamReservation(state_);
   DCHECK_GE(probe_client->GetUnusedReservation(), probe_reservation);
+
+  // Merge together num_spilled_probe_rows to include info from all threads.
+  for (int i = 0; i < PARTITION_FANOUT; ++i) {
+    PhjBuilder::Partition* partition = hash_partitions_[i].get();
+    if (partition->IsClosed()) continue;
+    partition->IncrementNumSpilledProbeRows(num_spilled_probe_rows[i]);
+  }
+
+  if (num_probe_threads_ > 1) {
+    // TODO: IMPALA-9411: consider reworking to attach buffers to all output batches.
+    RETURN_IF_ERROR(probe_barrier_->Wait([&]() {
+      CleanUpHashPartitions(output_partitions, nullptr);
+      DCHECK_EQ(0, output_partitions->size())
+          << "Cannot share build for join modes that return rows from build partitions";
+      return Status::OK();
+    }));
+  } else {
+    CleanUpHashPartitions(output_partitions, batch);
+  }
+
+  if (is_separate_build_) {
+    bool success;
+    RETURN_IF_ERROR(probe_client->TransferReservationTo(
+        buffer_pool_client_, probe_reservation, &success));
+    DCHECK(success) << "Transferring within query shouldn't violate reservation limits.";
+  }
+  return Status::OK();
+}
+
+void PhjBuilder::CleanUpHashPartitions(
+    deque<unique_ptr<Partition>>* output_partitions, RowBatch* batch) {
   if (state_ == HashJoinState::REPARTITIONING_PROBE) {
     // Finished repartitioning this partition. Discard before pushing more spilled
     // partitions onto 'spilled_partitions_'.
@@ -691,16 +737,18 @@ Status PhjBuilder::DoneProbingHashPartitions(
   for (int i = 0; i < PARTITION_FANOUT; ++i) {
     unique_ptr<PhjBuilder::Partition> partition = std::move(hash_partitions_[i]);
     if (partition->IsClosed()) continue;
-    partition->IncrementNumSpilledProbeRows(num_spilled_probe_rows[i]);
     if (partition->is_spilled()) {
       DCHECK(partition->hash_tbl() == nullptr) << DebugString();
       DCHECK_EQ(partition->build_rows()->BytesPinned(false), 0)
           << "Build was fully unpinned in BuildHashTablesAndPrepareProbeStreams()";
       if (partition->num_spilled_probe_rows() == 0
-          && !NeedToProcessUnmatchedBuildRows(join_op_)) {
+          && !NeedToProcessUnmatchedBuildRows(join_op_)
+          && num_probe_threads_ == 1) {
         COUNTER_ADD(num_hash_table_builds_skipped_, 1);
         partition->Close(nullptr);
       } else {
+        // For shared builds, always add the partition to keep the spilled partitions
+        // in sync across all the builders and join nodes.
         spilled_partitions_.push_back(std::move(partition));
       }
     } else if (NeedToProcessUnmatchedBuildRows(join_op_)) {
@@ -710,23 +758,41 @@ Status PhjBuilder::DoneProbingHashPartitions(
       partition->Close(batch);
     }
   }
+  hash_partitions_.clear();
+}
+
+Status PhjBuilder::DoneProbingSinglePartition(BufferPool::ClientHandle* probe_client,
+    deque<unique_ptr<Partition>>* output_partitions, RowBatch* batch) {
+  VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") done probing single partition.";
+  DCHECK_EQ(is_separate_build_, probe_client != buffer_pool_client_);
+  // Calculate before popping off the last 'spilled_partition_'.
+  int64_t probe_reservation = CalcProbeStreamReservation(state_);
+  DCHECK_GE(probe_client->GetUnusedReservation(), probe_reservation);
+  if (num_probe_threads_ > 1) {
+    // TODO: IMPALA-9411: consider reworking to attach buffers to all output batches.
+    RETURN_IF_ERROR(probe_barrier_->Wait([&]() {
+      CleanUpSinglePartition(output_partitions, nullptr);
+      DCHECK_EQ(0, output_partitions->size())
+          << "Cannot share build for join modes that return rows from build partitions";
+      return Status::OK();
+    }));
+  } else {
+    CleanUpSinglePartition(output_partitions, batch);
+  }
   if (is_separate_build_) {
     bool success;
     RETURN_IF_ERROR(probe_client->TransferReservationTo(
           buffer_pool_client_, probe_reservation, &success));
     DCHECK(success) << "Transferring within query shouldn't violate reservation limits.";
   }
-  hash_partitions_.clear();
   return Status::OK();
 }
 
-Status PhjBuilder::DoneProbingSinglePartition(BufferPool::ClientHandle* probe_client,
+void PhjBuilder::CleanUpSinglePartition(
     deque<unique_ptr<Partition>>* output_partitions, RowBatch* batch) {
-  VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") done probing single partition.";
-  DCHECK_EQ(is_separate_build_, probe_client != buffer_pool_client_);
-  // Calculate before popping off the last 'spilled_partition_'.
-  int64_t probe_reservation = CalcProbeStreamReservation(state_);
   if (NeedToProcessUnmatchedBuildRows(join_op_)) {
+    DCHECK_LE(num_probe_threads_, 1)
+        << "Don't support returning build partitions with shared build";
     // If the build partition was in memory, we are done probing this partition.
     // In case of right-outer, right-anti and full-outer joins, we move this partition
     // to the list of partitions that we need to output their unmatched build rows.
@@ -736,14 +802,6 @@ Status PhjBuilder::DoneProbingSinglePartition(BufferPool::ClientHandle* probe_cl
     spilled_partitions_.back()->Close(IsLeftSemiJoin(join_op_) ? nullptr : batch);
   }
   spilled_partitions_.pop_back();
-  DCHECK_GE(probe_client->GetUnusedReservation(), probe_reservation);
-  if (is_separate_build_) {
-    bool success;
-    RETURN_IF_ERROR(probe_client->TransferReservationTo(
-          buffer_pool_client_, probe_reservation, &success));
-    DCHECK(success) << "Transferring within query shouldn't violate reservation limits.";
-  }
-  return Status::OK();
 }
 
 void PhjBuilder::CloseAndDeletePartitions(RowBatch* row_batch) {
@@ -836,15 +894,42 @@ Status PhjBuilder::BeginSpilledProbe(
   DCHECK_EQ(is_separate_build_, probe_client != buffer_pool_client_);
   DCHECK(!spilled_partitions_.empty());
   DCHECK_EQ(0, hash_partitions_.size());
+  if (num_probe_threads_ > 1) {
+    RETURN_IF_ERROR(probe_barrier_->Wait([&]() { return BeginSpilledProbeSerial(); }));
+  } else {
+    RETURN_IF_ERROR(BeginSpilledProbeSerial());
+  }
+
+  RETURN_IF_ERROR(TransferProbeStreamReservation(probe_client));
+  *input_partition = spilled_partitions_.back().get();
+  if (state_ == HashJoinState::PROBING_SPILLED_PARTITION) {
+    *repartitioned = false;
+  } else {
+    DCHECK_ENUM_EQ(HashJoinState::REPARTITIONING_PROBE, state_);
+    *repartitioned = true;
+    *new_partitions =
+        HashPartitions(ht_ctx_->level(), &hash_partitions_, non_empty_build_);
+  }
+  return Status::OK();
+}
+
+Status PhjBuilder::BeginSpilledProbeSerial() {
+  DCHECK_EQ(0, probe_stream_reservation_.GetReservation());
+  if (is_separate_build_ || join_op_ != TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) {
+    DCHECK_EQ(0, buffer_pool_client_->GetUsedReservation())
+        << "All memory should be available to bring the spilled partition into memory: "
+        << "all build and probe data shuld be spilled to disk. THe only exception is "
+        << "NAAJ probe streams, which are accounted for in the PHJ node separately.";
+  }
+
   // Pick the next spilled partition to process. The partition will stay in
   // 'spilled_partitions_' until we are done probing it or repartitioning its probe.
   // Thus it will remain valid as long as it's needed and always get cleaned up in
   // Close(), even if an error occurs.
   Partition* partition = spilled_partitions_.back().get();
-  *input_partition = partition;
   DCHECK(partition->is_spilled()) << partition->DebugString();
 
-  if (partition->num_spilled_probe_rows() == 0) {
+  if (partition->num_spilled_probe_rows() == 0 && num_probe_threads_ == 1) {
     // If there are no probe rows, there's no need to build the hash table, and
     // only partitions with NeedToProcessUnmatchedBuildRows() will have been added
     // to 'spilled_partitions_' in DoneProbingHashPartitions().
@@ -857,22 +942,24 @@ Status PhjBuilder::BeginSpilledProbe(
     }
     COUNTER_ADD(num_hash_table_builds_skipped_, 1);
     UpdateState(HashJoinState::PROBING_SPILLED_PARTITION);
-    *repartitioned = false;
     return Status::OK();
   }
 
   // Set aside memory required for reading the probe stream, so that we don't use
   // it for the hash table.
+  bool need_probe_buffer = partition->num_spilled_probe_rows() > 0;
+  int64_t saved_probe_reservation =
+      need_probe_buffer ? max_row_buffer_size_ * num_probe_threads_ : 0;
   buffer_pool_client_->SaveReservation(
-      &probe_stream_reservation_, max_row_buffer_size_);
+      &probe_stream_reservation_, saved_probe_reservation);
+  VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") saved " << saved_probe_reservation
+          << " for probe clients.";
 
   // Try to build a hash table for the spilled build partition.
   bool built;
   RETURN_IF_ERROR(partition->BuildHashTable(&built));
   if (built) {
     UpdateState(HashJoinState::PROBING_SPILLED_PARTITION);
-    *repartitioned = false;
-    RETURN_IF_ERROR(TransferProbeStreamReservation(probe_client));
     return Status::OK();
   }
   // This build partition still does not fit in memory, repartition.
@@ -889,7 +976,9 @@ Status PhjBuilder::BeginSpilledProbe(
   // Temporarily free up the probe reservation to use when repartitioning. Repartitioning
   // will reserve as much memory as needed for the probe streams.
   buffer_pool_client_->RestoreReservation(
-      &probe_stream_reservation_, max_row_buffer_size_);
+      &probe_stream_reservation_, saved_probe_reservation);
+  VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") unsave " << saved_probe_reservation
+          << " for probe clients.";
   // All reservation should be available for repartitioning.
   DCHECK_EQ(0, probe_stream_reservation_.GetReservation());
   DCHECK_EQ(0, buffer_pool_client_->GetUsedReservation());
@@ -907,9 +996,6 @@ Status PhjBuilder::BeginSpilledProbe(
         next_partition_level, num_input_rows, DebugString(),
         buffer_pool_client_->DebugString());
   }
-  RETURN_IF_ERROR(TransferProbeStreamReservation(probe_client));
-  *repartitioned = true;
-  *new_partitions = HashPartitions(ht_ctx_->level(), &hash_partitions_, non_empty_build_);
   return Status::OK();
 }
 
@@ -1135,7 +1221,7 @@ std::string PhjBuilder::Partition::DebugString() {
   if (hash_tbl_ != nullptr) {
     ss << "    Hash Table Rows: " << hash_tbl_->size();
   }
-  ss << "    Spilled Probe Rows: " << num_spilled_probe_rows_ << endl;
+  ss << "    Spilled Probe Rows: " << num_spilled_probe_rows_.Load() << endl;
   return ss.str();
 }
 
@@ -1186,7 +1272,7 @@ void PhjBuilderConfig::Codegen(RuntimeState* state, RuntimeProfile* profile) {
 
 string PhjBuilder::DebugString() const {
   stringstream ss;
-  ss << " PhjBuilder state=" << PrintState()
+  ss << " PhjBuilder state=" << PrintState(state_)
      << " Hash partitions: " << hash_partitions_.size() << ":" << endl;
   for (int i = 0; i < hash_partitions_.size(); ++i) {
     ss << " Hash partition " << i << " " << hash_partitions_[i]->DebugString() << endl;
diff --git a/be/src/exec/partitioned-hash-join-builder.h b/be/src/exec/partitioned-hash-join-builder.h
index 5a7dc33..c009211 100644
--- a/be/src/exec/partitioned-hash-join-builder.h
+++ b/be/src/exec/partitioned-hash-join-builder.h
@@ -24,6 +24,7 @@
 #include <vector>
 #include <boost/scoped_ptr.hpp>
 
+#include "common/atomic.h"
 #include "common/object-pool.h"
 #include "common/status.h"
 #include "exec/filter-context.h"
@@ -37,6 +38,7 @@
 
 namespace impala {
 
+class CyclicBarrier;
 class PhjBuilder;
 class RowDescriptor;
 class RuntimeState;
@@ -207,6 +209,28 @@ enum class HashJoinState {
 /// be a different client.
 ///
 /// The full hash join algorithm is documented in PartitionedHashJoinNode.
+///
+/// Shared Build
+/// ------------
+/// A separate builder can be shared between multiple PartitionedHashJoinNodes. The
+/// spilling hash join algorithm mutates the state of the builder between phases, so
+/// requires synchronization between the probe threads executing PartitionedHashJoinNode
+/// that are reading that state.
+///
+/// The algorithm (specifically the HashJoinState state machine) is executed in lockstep
+/// across all probe threads with each probe thread working on the same set of partitions
+/// at the same time. A CyclicBarrier, 'probe_barrier_', is used for synchronization.
+/// At each state transition where the builder state needs to be mutated, all probe
+/// threads must arrive at the barrier before proceeding. The state transition is executed
+/// serially by a single thread before all threads proceed. All probe threads go through
+/// the same state transitions in lockstep, even if they have no work to do. E.g. if a
+/// probe thread has zero rows remaining in its spilled partitions, it still needs to
+/// wait for the other probe threads.
+///
+/// Not all join ops can be used with a shared build. For example, RIGHT_OUTER_JOIN is
+/// not supported currently, in part because it mutates the hash table during probing to
+/// track matches, but also because hash table matches would need to be broadcast across
+/// all instances within the query, not just the backend.
 class PhjBuilder : public JoinBuilder {
  public:
   /// Number of initial partitions to create. Must be a power of two.
@@ -312,7 +336,9 @@ class PhjBuilder : public JoinBuilder {
   /// PARTITION_FANOUT new partitions with level input_partition->level() + 1. The
   /// previous hash partitions must have been cleared with DoneProbingHashPartitions().
   /// The new hash partitions are returned in 'new_partitions'.
-  /// TODO: IMPALA-9156: this will be a synchronization point for shared join build.
+  ///
+  /// This is a synchronization point for shared join build. All probe threads must
+  /// call this function before continuing the next phase of the hash join algorithm.
   Status BeginSpilledProbe(BufferPool::ClientHandle* probe_client, bool* repartitioned,
       Partition** input_partition, HashPartitions* new_partitions);
 
@@ -330,7 +356,9 @@ class PhjBuilder : public JoinBuilder {
   /// transferred back to the builder.
   ///
   /// Returns an error if an error was encountered or if the query was cancelled.
-  /// TODO: IMPALA-9156: this will be a synchronization point for shared join build.
+  ///
+  /// This is a synchronization point for shared join build. All probe threads must
+  /// call this function before continuing the next phase of the hash join algorithm.
   Status DoneProbingHashPartitions(const int64_t num_spilled_probe_rows[PARTITION_FANOUT],
       BufferPool::ClientHandle* probe_client,
       std::deque<std::unique_ptr<Partition>>* output_partitions, RowBatch* batch);
@@ -351,7 +379,9 @@ class PhjBuilder : public JoinBuilder {
   /// transferred back to the builder.
   ///
   /// Returns an error if an error was encountered or if the query was cancelled.
-  /// TODO: IMPALA-9156: this will be a synchronization point for shared join build.
+  ///
+  /// This is a synchronization point for shared join build. All probe threads must
+  /// call this function before continuing the next phase of the hash join algorithm.
   Status DoneProbingSinglePartition(BufferPool::ClientHandle* probe_client,
       std::deque<std::unique_ptr<Partition>>* output_partitions, RowBatch* batch);
 
@@ -373,10 +403,6 @@ class PhjBuilder : public JoinBuilder {
   static bool HashTableStoresNulls(
       TJoinOp::type join_op, const std::vector<bool>& is_not_distinct_from);
 
-  /// TODO: IMPALA-9156: document thread safety for accessing this from
-  /// multiple PartitionedHashJoinNodes.
-  void AddHashTableStatsToProfile(RuntimeProfile* profile);
-
   /// Returns 'bytes' of reservation to the builder from 'probe_client'.
   /// Called by the probe side to return surplus reservation. This is usually handled by
   /// the above methods, but if an error occured during execution, the probe may still
@@ -384,16 +410,14 @@ class PhjBuilder : public JoinBuilder {
   /// Must only be called if this is a separate build.
   void ReturnReservation(BufferPool::ClientHandle* probe_client, int64_t bytes);
 
-  /// TODO: IMPALA-9156: document thread safety for accessing this from
-  /// multiple PartitionedHashJoinNodes.
+  /// Safe to call from PartitionedHashJoinNode threads during the probe phase.
   HashJoinState state() const { return state_; }
 
   /// Accessor to allow PartitionedHashJoinNode to access null_aware_partition_.
   /// TODO: IMPALA-9176: improve the encapsulation of the null-aware partition.
   inline Partition* null_aware_partition() const { return null_aware_partition_.get(); }
 
-  /// TODO: IMPALA-9156: document thread safety for accessing this from
-  /// multiple PartitionedHashJoinNodes.
+  /// Thread-safe.
   HashTableStatsProfile* ht_stats_profile() const { return ht_stats_profile_.get(); }
 
   std::string DebugString() const;
@@ -453,8 +477,12 @@ class PhjBuilder : public JoinBuilder {
     int ALWAYS_INLINE level() const { return level_; }
     /// Return true if the partition can be spilled - is not closed and is not spilled.
     bool CanSpill() const { return !IsClosed() && !is_spilled(); }
-    int64_t num_spilled_probe_rows() const { return num_spilled_probe_rows_; }
-    void IncrementNumSpilledProbeRows(int64_t count) { num_spilled_probe_rows_ += count; }
+    int64_t num_spilled_probe_rows() const { return num_spilled_probe_rows_.Load(); }
+
+    /// Increment the number of spilled probe rows. Thread-safe.
+    void IncrementNumSpilledProbeRows(int64_t count) {
+      num_spilled_probe_rows_.Add(count);
+    }
 
     /// Method signature of the codegened version of InsertBatch().
     typedef bool (*InsertBatchFn)(Partition*, TPrefetchMode::type, HashTableCtx*,
@@ -497,7 +525,7 @@ class PhjBuilder : public JoinBuilder {
 
     /// The number of spilled probe rows associated with this partition. Updated in
     /// DoneProbingHashPartitions().
-    int64_t num_spilled_probe_rows_ = 0;
+    AtomicInt64 num_spilled_probe_rows_{0};
   };
 
   /// Computes the minimum reservation required to execute the spilling partitioned
@@ -536,8 +564,8 @@ class PhjBuilder : public JoinBuilder {
   /// Updates 'state_' to 'next_state', logging the transition.
   void UpdateState(HashJoinState next_state);
 
-  /// Returns the current 'state_' as a string.
-  std::string PrintState() const;
+  /// Returns the string represenvation of 'state'.
+  static std::string PrintState(HashJoinState state);
 
   /// Create and initialize a set of hash partitions for partitioning level 'level'.
   /// The previous hash partitions must have been cleared with DoneProbing().
@@ -601,7 +629,8 @@ class PhjBuilder : public JoinBuilder {
   /// Ensures that 'probe_stream_reservation_' has enough reservation for a stream per
   /// spilled partition in 'hash_partitions_', plus for the input stream if the input
   /// is a spilled partition (determined by 'next_state' - either PARTITIONING_PROBE or
-  /// REPARTITIONING_PROBE). May spill additional partitions until it can free enough
+  /// REPARTITIONING_PROBE). If num_probe_threads_ is > 1, reserves this amount for each
+  /// probe thread. May spill additional partitions until it can free enough
   /// reservation. Returns an error if an error is encountered or if it runs out of
   /// partitions to spill.
   Status ReserveProbeBuffers(HashJoinState next_state);
@@ -613,12 +642,17 @@ class PhjBuilder : public JoinBuilder {
   /// Transfer reservation for probe streams to 'probe_client'. Memory for one stream was
   /// reserved per spilled partition in FlushFinal(), plus the input stream if the input
   /// partition was spilled.
+  /// This is safe to call from multiple probe threads concurrently.
   Status TransferProbeStreamReservation(BufferPool::ClientHandle* probe_client);
 
-  /// Calculates the amount of memory to be transferred for probe streams when probing
-  /// in the given 'state'. Depends on 'hash_partitions_' and 'spillable_buffer_size_'.
+  /// Calculates the amount of memory per probe thread/join node instance to be
+  /// transferred for probe streams when probing in the given 'state'. Depends on
+  /// 'hash_partitions_', 'spilled_partitions_' and 'spillable_buffer_size_'.
   int64_t CalcProbeStreamReservation(HashJoinState state) const;
 
+  /// The serial part of BeginSpilledProbe() that is executed by a single thread.
+  Status BeginSpilledProbeSerial();
+
   /// Creates new hash partitions and repartitions 'input_partition' into PARTITION_FANOUT
   /// new partitions with level input_partition->level() + 1. The previous hash partitions
   /// must have been cleared with ClearHashPartitions(). This function reserves enough
@@ -629,6 +663,16 @@ class PhjBuilder : public JoinBuilder {
   /// Returns the largest build row count out of the current hash partitions.
   int64_t LargestPartitionRows() const;
 
+  /// Helper for DoneProbingHashPartitions() that processes and cleans up the hash
+  /// partitions.
+  void CleanUpHashPartitions(
+      std::deque<std::unique_ptr<Partition>>* output_partitions, RowBatch* batch);
+
+  /// Helper for DoneProbingSinglePartition() that processes and cleans up the current
+  /// spilled partition.
+  void CleanUpSinglePartition(
+      std::deque<std::unique_ptr<Partition>>* output_partitions, RowBatch* batch);
+
   /// Calls Close() on every Partition, deletes them, and cleans up any pointers that
   /// may reference them. If 'row_batch' if not NULL, transfers the ownership of all
   /// row-backing resources to it.
@@ -742,6 +786,10 @@ class PhjBuilder : public JoinBuilder {
   /// that were not spilled.
   RuntimeProfile::Counter* repartition_timer_ = nullptr;
 
+  // Barrier used to synchronize the probe-side threads at synchronization points in the
+  // partitioned hash join algorithm. Used only when 'num_probe_threads_' > 1.
+  std::unique_ptr<CyclicBarrier> probe_barrier_;
+
   /////////////////////////////////////////
   /// BEGIN: Members that must be Reset()
 
diff --git a/be/src/exec/partitioned-hash-join-node.cc b/be/src/exec/partitioned-hash-join-node.cc
index 05d4cb6..5f53f1d 100644
--- a/be/src/exec/partitioned-hash-join-node.cc
+++ b/be/src/exec/partitioned-hash-join-node.cc
@@ -267,6 +267,7 @@ Status PartitionedHashJoinNode::Reset(RuntimeState* state, RowBatch* row_batch)
   }
   output_unmatched_batch_.reset();
   output_unmatched_batch_iter_.reset();
+  flushed_unattachable_build_buffers_ = false;
   return BlockingJoinNode::Reset(state, row_batch);
 }
 
@@ -588,12 +589,23 @@ Status PartitionedHashJoinNode::GetNext(
           probe_state_ = ProbeState::PROBING_IN_BATCH;
         } else if (probe_eos) {
           DCHECK_EQ(probe_batch_pos_, -1);
-          // Finished processing all the probe rows for the current hash partitions.
-          // There may be some partitions that need to outpt their unmatched build rows.
-          RETURN_IF_ERROR(DoneProbing(state, out_batch));
-          probe_state_ = output_build_partitions_.empty() ?
-              ProbeState::PROBE_COMPLETE :
-              ProbeState::OUTPUTTING_UNMATCHED;
+          if (UseSeparateBuild(state->query_options())
+              && !flushed_unattachable_build_buffers_ && ReturnsBuildData(join_op_)) {
+            // Can't attach build-side data because it may be referenced by multiple
+            // finstances. Note that this makes the batch AtCapacity(), so we will exit
+            // the loop below.
+            // TODO: IMPALA-9411: implement shared ownership of buffers to avoid this.
+            flushed_unattachable_build_buffers_ = true;
+            out_batch->MarkNeedsDeepCopy();
+          } else {
+            // Finished processing all the probe rows for the current hash partitions.
+            // There may be some partitions that need to outpt their unmatched build rows.
+            RETURN_IF_ERROR(DoneProbing(state, out_batch));
+            probe_state_ = output_build_partitions_.empty() ?
+                ProbeState::PROBE_COMPLETE :
+                ProbeState::OUTPUTTING_UNMATCHED;
+            flushed_unattachable_build_buffers_ = false;
+          }
         } else {
           // Got an empty batch with resources that we need to flush before getting the
           // next batch.
@@ -1187,7 +1199,8 @@ Status PartitionedHashJoinNode::DoneProbing(RuntimeState* state, RowBatch* batch
           }
         }
       } else if (probe_partition->probe_rows()->num_rows() != 0
-          || NeedToProcessUnmatchedBuildRows(join_op_)) {
+          || NeedToProcessUnmatchedBuildRows(join_op_)
+          || builder_->num_probe_threads() > 1) {
         num_spilled_probe_rows[i] = probe_partition->probe_rows()->num_rows();
         // Unpin the probe stream to free up more memory. We need to free all memory so we
         // can recurse the algorithm and create new hash partitions from spilled
@@ -1200,6 +1213,8 @@ Status PartitionedHashJoinNode::DoneProbing(RuntimeState* state, RowBatch* batch
         // There's no more processing to do for this partition, and since there were no
         // probe rows we didn't return any rows that reference memory from these
         // partitions, so just free the resources.
+        // Avoid doing this for shared builds so that all probe threads have the same
+        // number of partitions, which simplifies logic.
         probe_partition->Close(nullptr);
       }
     }
@@ -1265,12 +1280,13 @@ string PartitionedHashJoinNode::NodeDebugString() const {
       PhjBuilder::Partition* build_partition = probe_partition->build_partition();
       DCHECK(build_partition->is_spilled());
       DCHECK(build_partition->hash_tbl() == NULL);
-      DCHECK(build_partition->build_rows() != NULL);
-      DCHECK(probe_partition->probe_rows() != NULL);
+      int build_rows = build_partition->build_rows() == nullptr ?  -1 :
+          build_partition->build_rows()->num_rows();
+      int probe_rows = probe_partition->probe_rows() == nullptr ?  -1 :
+          probe_partition->probe_rows()->num_rows();
       ss << "  ProbePartition (id=" << entry.first << "):" << probe_partition << endl
-         << "   Spilled Build Rows: " << build_partition->build_rows()->num_rows() << endl
-         << "   Spilled Probe Rows: " << probe_partition->probe_rows()->num_rows()
-         << endl;
+         << "   Spilled Build Rows: " << build_rows << endl
+         << "   Spilled Probe Rows: " << probe_rows << endl;
     }
   }
   if (input_partition_ != NULL) {
diff --git a/be/src/exec/partitioned-hash-join-node.h b/be/src/exec/partitioned-hash-join-node.h
index 8721d35..b6bcca9 100644
--- a/be/src/exec/partitioned-hash-join-node.h
+++ b/be/src/exec/partitioned-hash-join-node.h
@@ -152,11 +152,11 @@ class PartitionedHashJoinPlanNode : public BlockingJoinPlanNode {
 ///      progress.
 ///
 ///
-/// TODO: when IMPALA-9156 is implemented, HashJoinState of the builder will drive the
-/// hash join algorithm across all the PartitionedHashJoinNode implementations sharing
-/// the builder. Each PartitionedHashJoinNode implementation will independently execute
-/// its ProbeState state machine, synchronizing via the builder for transitions of the
-/// HashJoinState state machine.
+/// When the PhjBuilder is shared by multiple PartitionedHashJoinNodes, HashJoinState of
+/// the builder will drive the hash join algorithm across all the PartitionedHashJoinNode
+/// implementations sharing the builder. Each PartitionedHashJoinNode implementation will
+/// independently execute its ProbeState state machine, synchronizing via the builder for
+/// transitions of the HashJoinState state machine.
 ///
 /// Null aware anti-join (NAAJ) extends the above algorithm by accumulating rows with
 /// NULLs into several different streams, which are processed in a separate step to
@@ -589,6 +589,11 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
   /// State of the probing algorithm. Used to drive the state machine in GetNext().
   ProbeState probe_state_ = ProbeState::PROBE_COMPLETE;
 
+  /// Additional state for flushing build-side data. Needed for
+  /// separate build.
+  /// TODO: IMPALA-9411: this could be removed if we attached the buffers.
+  bool flushed_unattachable_build_buffers_ = false;
+
   /// The build-side rows of the join. Initialized in Prepare() if the build is embedded
   /// in the join, otherwise looked up in Open() if it's a separate build. Owned by an
   /// object pool with query lifetime in either case.
@@ -614,6 +619,8 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
 
   /// Probe partitions that have been spilled and still need more processing. Each of
   /// these has a corresponding build partition in 'builder_' with the same PartitionId.
+  /// For shared broadcast join builds, the set of keys in this map will be the same
+  /// across all of the instances of the join builder.
   /// This list is populated at DoneProbing().
   std::unordered_map<PhjBuilder::PartitionId, std::unique_ptr<ProbePartition>>
       spilled_partitions_;
diff --git a/be/src/runtime/bufferpool/buffer-pool-internal.h b/be/src/runtime/bufferpool/buffer-pool-internal.h
index 36fa631..2b2d31c 100644
--- a/be/src/runtime/bufferpool/buffer-pool-internal.h
+++ b/be/src/runtime/bufferpool/buffer-pool-internal.h
@@ -364,6 +364,10 @@ class BufferPool::Client {
   /// Condition variable signalled when a write for this client completes.
   ConditionVariable write_complete_cv_;
 
+  /// Used to ensure that only one thread at a time is active in CleanPages().
+  bool cleaning_pages_ = false;
+  ConditionVariable clean_pages_done_cv_;
+
   /// All non-OK statuses returned by write operations are merged into this status.
   /// All operations that depend on pages being written to disk successfully (e.g.
   /// reading pages back from disk) must check 'write_status_' before proceeding, so
diff --git a/be/src/runtime/bufferpool/buffer-pool.cc b/be/src/runtime/bufferpool/buffer-pool.cc
index 9636c21..c56b6cd 100644
--- a/be/src/runtime/bufferpool/buffer-pool.cc
+++ b/be/src/runtime/bufferpool/buffer-pool.cc
@@ -29,6 +29,7 @@
 #include "util/debug-util.h"
 #include "util/metrics.h"
 #include "util/runtime-profile-counters.h"
+#include "util/scope-exit-trigger.h"
 #include "util/time.h"
 
 DEFINE_int32(concurrent_scratch_ios_per_device, 2,
@@ -646,18 +647,22 @@ Status BufferPool::Client::CleanPages(
   DCHECK_GE(len, 0);
   DCHECK_LE(len, reservation_.GetReservation());
   DCheckHoldsLock(*client_lock);
-  DCHECK_CONSISTENCY();
+  // If another thread is in CleanPages() for this client (and has dropped the lock while
+  // waiting on 'write_complete_cv_', wait for it to flush its pages before proceeding so
+  // that we don't overcommit memory.
+  while (cleaning_pages_) clean_pages_done_cv_.Wait(*client_lock);
 
+  DCHECK_CONSISTENCY();
   // Work out what we need to get bytes of dirty unpinned + in flight pages down to
   // in order to satisfy the eviction policy.
   int64_t target_dirty_bytes = reservation_.GetReservation() - buffers_allocated_bytes_
       - pinned_pages_.bytes() - len;
   if (VLOG_IS_ON(3)) {
     VLOG(3)   << "target_dirty_bytes=" << target_dirty_bytes
-              << "reservation=" << reservation_.GetReservation()
-              << "buffers_allocated_bytes_=" << buffers_allocated_bytes_
-              << "pinned_pages_.bytes()=" << pinned_pages_.bytes()
-              << "len=" << len << "\n"
+              << " reservation=" << reservation_.GetReservation()
+              << " buffers_allocated_bytes_=" << buffers_allocated_bytes_
+              << " pinned_pages_.bytes()=" << pinned_pages_.bytes()
+              << " len=" << len << "\n"
               << DebugStringLocked();
   }
   // Start enough writes to ensure that the loop condition below will eventually become
@@ -669,6 +674,12 @@ Status BufferPool::Client::CleanPages(
           <= target_dirty_bytes) {
     return Status::OK();
   }
+  cleaning_pages_ = true;
+  auto exit_trigger = MakeScopeExitTrigger([this, client_lock]() {
+    DCheckHoldsLock(*client_lock);
+    cleaning_pages_ = false;
+    clean_pages_done_cv_.NotifyAll();
+  });
   WriteDirtyPagesAsync(min_bytes_to_write);
 
   // One of the writes we initiated, or an earlier in-flight write may have hit an error.
diff --git a/be/src/runtime/bufferpool/buffer-pool.h b/be/src/runtime/bufferpool/buffer-pool.h
index 7d44905..ffa97a9 100644
--- a/be/src/runtime/bufferpool/buffer-pool.h
+++ b/be/src/runtime/bufferpool/buffer-pool.h
@@ -366,6 +366,9 @@ class BufferPool::ClientHandle {
 
   /// Move some of src's reservation to this client. 'bytes' of unused reservation must be
   /// available in 'src'.
+  ///
+  /// This is safe to call concurrently from multiple threads, as long as those threads
+  /// coordinate to ensure there is sufficient unused reservation.
   void RestoreReservation(SubReservation* src, int64_t bytes);
 
   /// Accessors for this client's reservation corresponding to the identically-named
@@ -385,6 +388,9 @@ class BufferPool::ClientHandle {
   /// unpinned pages to disk and a write to disk fails, in which case it returns an error
   /// status. May also fail if a reservation limit on 'dst' would be exceeded as a result
   /// of the transfer, in which case *transferred is false but Status::OK is returned.
+  ///
+  /// This is safe to call concurrently from multiple threads, as long as those threads
+  /// coordinate to ensure there is sufficient unused reservation.
   Status TransferReservationTo(ReservationTracker* dst, int64_t bytes, bool* transferred);
   Status TransferReservationTo(ClientHandle* dst, int64_t bytes, bool* transferred);
 
diff --git a/be/src/runtime/coordinator-backend-state.cc b/be/src/runtime/coordinator-backend-state.cc
index 3d53700..5d2223d 100644
--- a/be/src/runtime/coordinator-backend-state.cc
+++ b/be/src/runtime/coordinator-backend-state.cc
@@ -144,6 +144,9 @@ void Coordinator::BackendState::SetRpcParams(const DebugOptions& debug_options,
         params.fragment_exec_params.per_exch_num_senders);
     instance_ctx.__set_sender_id(params.sender_id);
     instance_ctx.__set_join_build_inputs(params.join_build_inputs);
+    if (params.num_join_build_outputs != -1) {
+      instance_ctx.__set_num_join_build_outputs(params.num_join_build_outputs);
+    }
     if (debug_options.enabled()
         && (debug_options.instance_idx() == -1
             || debug_options.instance_idx() == GetInstanceIdx(params.instance_id))) {
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index 89fa6f7..8da7259 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -46,6 +46,7 @@
 #include "util/auth-util.h" // for GetEffectiveUser()
 #include "util/bitmap.h"
 #include "util/cpu-info.h"
+#include "util/cyclic-barrier.h"
 #include "util/debug-util.h"
 #include "util/disk-info.h"
 #include "util/error-util.h"
@@ -247,6 +248,9 @@ void RuntimeState::Cancel() {
   {
     lock_guard<SpinLock> l(cancellation_cvs_lock_);
     for (ConditionVariable* cv : cancellation_cvs_) cv->NotifyAll();
+    for (CyclicBarrier* cb : cancellation_cbs_) {
+      cb->Cancel(Status::CancelledInternal("RuntimeState::Cancel()"));
+    }
   }
 }
 
@@ -259,6 +263,15 @@ void RuntimeState::AddCancellationCV(ConditionVariable* cv) {
   cancellation_cvs_.push_back(cv);
 }
 
+void RuntimeState::AddBarrierToCancel(CyclicBarrier* cb) {
+  lock_guard<SpinLock> l(cancellation_cvs_lock_);
+  for (CyclicBarrier* cb2 : cancellation_cbs_) {
+    // Don't add if already present.
+    if (cb == cb2) return;
+  }
+  cancellation_cbs_.push_back(cb);
+}
+
 double RuntimeState::ComputeExchangeScanRatio() const {
   int64_t bytes_read = 0;
   for (const auto& c : bytes_read_counters_) bytes_read += c->value();
diff --git a/be/src/runtime/runtime-state.h b/be/src/runtime/runtime-state.h
index 095163c..2735d50 100644
--- a/be/src/runtime/runtime-state.h
+++ b/be/src/runtime/runtime-state.h
@@ -56,6 +56,7 @@ class TPlanFragmentCtx;
 class TPlanFragmentInstanceCtx;
 class QueryState;
 class ConditionVariable;
+class CyclicBarrier;
 
 namespace io {
   class DiskIoMgr;
@@ -249,6 +250,12 @@ class RuntimeState {
   /// The condition variable must have query lifetime.
   void AddCancellationCV(ConditionVariable* cv);
 
+  /// Add a barrier to be cancelled when this RuntimeState is cancelled. Adding a barrier
+  /// multiple times is a no-op. Each distinct 'cb' will be cancelled with status code
+  /// CANCELLED_INTERNALLY when is_cancelled() becomes true. 'cb' must have query
+  /// lifetime.
+  void AddBarrierToCancel(CyclicBarrier* cb);
+
   RuntimeProfile::Counter* total_storage_wait_timer() {
     return total_storage_wait_timer_;
   }
@@ -436,6 +443,13 @@ class RuntimeState {
   /// Condition variables that will be signalled by Cancel(). Protected by
   /// 'cancellation_cvs_lock_'.
   std::vector<ConditionVariable*> cancellation_cvs_;
+
+  /// Cyclic barriers that will be signalled by Cancel(). Protected by
+  /// 'cancellation_cvs_lock_'.
+  std::vector<CyclicBarrier*> cancellation_cbs_;
+
+  /// Condition variables that will be signalled by Cancel(). Protected by
+  /// 'cancellation_cvs_lock_'.
   SpinLock cancellation_cvs_lock_;
 
   /// if true, ReleaseResources() was called.
diff --git a/be/src/scheduling/query-schedule.h b/be/src/scheduling/query-schedule.h
index 9cbbfa9..b2d5d65 100644
--- a/be/src/scheduling/query-schedule.h
+++ b/be/src/scheduling/query-schedule.h
@@ -104,11 +104,15 @@ struct FInstanceExecParams {
 
   /// In its role as a data sender, a fragment instance is assigned a "sender id" to
   /// uniquely identify it to a receiver. -1 = invalid.
-  int sender_id;
+  int sender_id = -1;
 
   // List of input join build finstances for joins in this finstance.
   std::vector<TJoinBuildInput> join_build_inputs;
 
+  // If this is a join build fragment, the number of fragment instances that consume the
+  // join build. -1 = invalid.
+  int num_join_build_outputs = -1;
+
   /// The parent FragmentExecParams
   const FragmentExecParams& fragment_exec_params;
   const TPlanFragment& fragment() const;
@@ -120,7 +124,6 @@ struct FInstanceExecParams {
       host(host),
       krpc_host(krpc_host),
       per_fragment_instance_idx(per_fragment_instance_idx),
-      sender_id(-1),
       fragment_exec_params(fragment_exec_params) {}
 };
 
diff --git a/be/src/scheduling/scheduler.cc b/be/src/scheduling/scheduler.cc
index ffa393e..00af384 100644
--- a/be/src/scheduling/scheduler.cc
+++ b/be/src/scheduling/scheduler.cc
@@ -535,20 +535,29 @@ void Scheduler::CreateCollocatedJoinBuildInstances(
       schedule->GetFragmentExecParams(join_fragment_idx);
   DCHECK(!join_fragment_params->instance_exec_params.empty())
       << "Parent fragment instances must already be created.";
+  vector<FInstanceExecParams>* instance_exec_params =
+      &fragment_params->instance_exec_params;
+  bool share_build = fragment.output_sink.join_build_sink.share_build;
   int per_fragment_instance_idx = 0;
   for (FInstanceExecParams& parent_exec_params :
       join_fragment_params->instance_exec_params) {
-    TUniqueId instance_id = schedule->GetNextInstanceId();
-    fragment_params->instance_exec_params.emplace_back(instance_id,
-        parent_exec_params.host, parent_exec_params.krpc_host,
-        per_fragment_instance_idx++, *fragment_params);
+    // Share the build if join build sharing is enabled for this fragment and the previous
+    // instance was on the same host (instances for a backend are clustered together).
+    if (!share_build || instance_exec_params->empty()
+        || instance_exec_params->back().krpc_host != parent_exec_params.krpc_host) {
+      TUniqueId instance_id = schedule->GetNextInstanceId();
+      instance_exec_params->emplace_back(instance_id, parent_exec_params.host,
+          parent_exec_params.krpc_host, per_fragment_instance_idx++, *fragment_params);
+      instance_exec_params->back().num_join_build_outputs = 0;
+    }
     TJoinBuildInput build_input;
     build_input.__set_join_node_id(sink.dest_node_id);
-    build_input.__set_input_finstance_id(instance_id);
+    build_input.__set_input_finstance_id(instance_exec_params->back().instance_id);
     parent_exec_params.join_build_inputs.emplace_back(build_input);
     VLOG(3) << "Linked join build for node id=" << sink.dest_node_id
-            << " build finstance=" << PrintId(instance_id)
+            << " build finstance=" << PrintId(instance_exec_params->back().instance_id)
             << " dst finstance=" << PrintId(parent_exec_params.instance_id);
+    ++instance_exec_params->back().num_join_build_outputs;
   }
 }
 
diff --git a/be/src/util/cyclic-barrier-test.cc b/be/src/util/cyclic-barrier-test.cc
index 2617263..cafacd4 100644
--- a/be/src/util/cyclic-barrier-test.cc
+++ b/be/src/util/cyclic-barrier-test.cc
@@ -40,7 +40,10 @@ void BasicTest(int num_threads, int num_iters) {
         // Add some randomness to test so that threads don't always join in predictable
         // order.
         SleepForMs(rand() % 5);
-        EXPECT_OK(barrier.Wait([&counter]() { ++counter; }));
+        EXPECT_OK(barrier.Wait([&counter]() {
+          ++counter;
+          return Status::OK();
+        }));
       }));
     }
     threads.join_all();
@@ -82,7 +85,10 @@ void OverlapTest(int num_threads, int num_iters) {
   for (int i = 0; i < num_threads; ++i) {
     threads.add_thread(new thread([&]() {
       for (int j = 0; j < num_iters; ++j) {
-        EXPECT_OK(barrier.Wait([&counter]() { ++counter; }));
+        EXPECT_OK(barrier.Wait([&counter]() {
+          ++counter;
+          return Status::OK();
+        }));
       }
     }));
   }
@@ -110,7 +116,10 @@ TEST(CyclicBarrierTest, Cancellation) {
     threads.add_thread(new thread([&barrier, &waits_complete, &counter, i]() {
       // Add some jitter so that not all threads will be waiting when cancelled.
       if (i % 2 == 0) SleepForMs(rand() % 5);
-      Status status = barrier.Wait([&counter]() { ++counter; });
+      Status status = barrier.Wait([&counter]() {
+        ++counter;
+        return Status::OK();
+      });
       EXPECT_FALSE(status.ok());
       EXPECT_EQ(status.code(), TErrorCode::CANCELLED);
       waits_complete.Add(1);
@@ -124,7 +133,10 @@ TEST(CyclicBarrierTest, Cancellation) {
 
   // Subsequent calls to Wait() return immediately.
   for (int i = 0; i < NUM_THREADS; ++i) {
-    Status status = barrier.Wait([&counter]() { ++counter; });
+    Status status = barrier.Wait([&counter]() {
+      ++counter;
+      return Status::OK();
+    });
     EXPECT_FALSE(status.ok());
     EXPECT_EQ(status.code(), TErrorCode::CANCELLED);
     EXPECT_EQ(0, counter) << "The callback should not have run.";
@@ -132,7 +144,10 @@ TEST(CyclicBarrierTest, Cancellation) {
 
   // First status is not overwritten by later Cancel() calls.
   barrier.Cancel(Status("Different status"));
-  Status status = barrier.Wait([&counter]() { ++counter; });
+  Status status = barrier.Wait([&counter]() {
+    ++counter;
+    return Status::OK();
+  });
   EXPECT_FALSE(status.ok());
   EXPECT_EQ(status.code(), TErrorCode::CANCELLED);
 
@@ -143,7 +158,7 @@ TEST(CyclicBarrierTest, Cancellation) {
 // Passing an empty/null function to Wait() is not supported.
 TEST(CyclicBarrierTest, NullFunction) {
   CyclicBarrier barrier(1);
-  typedef void (*fn_ptr_t)();
+  typedef Status (*fn_ptr_t)();
   IMPALA_ASSERT_DEBUG_DEATH(barrier.Wait(static_cast<fn_ptr_t>(nullptr)), "");
 }
 } // namespace impala
diff --git a/be/src/util/cyclic-barrier.h b/be/src/util/cyclic-barrier.h
index 7b2aee5..065c1ae 100644
--- a/be/src/util/cyclic-barrier.h
+++ b/be/src/util/cyclic-barrier.h
@@ -38,9 +38,14 @@ class CyclicBarrier {
   /// Waits until all threads have joined the barrier. Then the last thread executes 'fn'
   /// and once that is completed, all threads return. Note that 'fn' executes serially,
   /// so can be used to implement a serial phase of a parallel algorithm.
+  ///
+  /// 'fn' must return a Status object and have no arguments. If the call to 'fn' returns
+  /// an error status, the barrier will be cancelled with that status.
+  ///
   /// Returns OK if all threads joined the barrier or an error status if cancelled.
   template <typename F>
   Status Wait(const F& fn) {
+    Status fn_status;
     {
       std::unique_lock<std::mutex> l(lock_);
       RETURN_IF_ERROR(cancel_status_);
@@ -56,12 +61,16 @@ class CyclicBarrier {
       }
       // This is the last thread and barrier isn't cancelled. We can proceed by
       // resetting state for the next cycle.
-      fn();
-      num_waiting_threads_ = 0;
-      ++cycle_num_;
+      fn_status = fn();
+      if (fn_status.ok()) {
+        num_waiting_threads_ = 0;
+        ++cycle_num_;
+      } else {
+        cancel_status_ = fn_status;
+      }
     }
     barrier_cv_.NotifyAll();
-    return Status::OK();
+    return fn_status;
   }
 
   // Cancels the barrier. All blocked and future calls to cancel will return immediately
diff --git a/common/thrift/DataSinks.thrift b/common/thrift/DataSinks.thrift
index 81c1a67..858921e 100644
--- a/common/thrift/DataSinks.thrift
+++ b/common/thrift/DataSinks.thrift
@@ -117,6 +117,10 @@ struct TJoinBuildSink {
   // Hash seed to use. Only set for hash join builds. Must be the same as the join node's
   // hash seed. Must be positive.
   5: optional i32 hash_seed
+
+  // If true, join build sharing is enabled and, if multiple instances of a join node are
+  // scheduled on the same backend, they will share the join build on that backend.
+  6: optional bool share_build
 }
 
 struct TPlanRootSink {
diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index cbe3f50..24112c4 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -659,6 +659,10 @@ struct TPlanFragmentInstanceCtx {
 
   // List of input join build finstances for joins in this finstance.
   9: optional list<TJoinBuildInput> join_build_inputs
+
+  // If this is a join build fragment, the number of fragment instances that consume the
+  // join build. -1 = invalid.
+  10: optional i32 num_join_build_outputs
 }
 
 
diff --git a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
index 625e832..239afc6 100644
--- a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
@@ -517,7 +517,7 @@ public class AggregationNode extends PlanNode {
       long inputCardinality = getChild(0).getCardinality();
       if (inputCardinality != -1) {
         // Calculate the input cardinality distributed across fragment instances.
-        long numInstances = fragment_.getNumInstances(queryOptions.getMt_dop());
+        long numInstances = fragment_.getNumInstances();
         long perInstanceInputCardinality;
         if (numInstances > 1) {
           perInstanceInputCardinality =
diff --git a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
index ee7e4ee..90a9e90 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
@@ -275,7 +275,7 @@ public class DataSourceScanNode extends ScanNode {
       LOG.trace("computeStats DataSourceScan: cardinality=" + Long.toString(cardinality_));
     }
 
-    numNodes_ = table_.getNumNodes();
+    numInstances_ = numNodes_ = table_.getNumNodes();
     if (LOG.isTraceEnabled()) {
       LOG.trace("computeStats DataSourceScan: #nodes=" + Integer.toString(numNodes_));
     }
diff --git a/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java b/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
index cfd238b..f8fdd60 100644
--- a/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
@@ -449,22 +449,23 @@ public class DistributedPlanner {
     PlanNode rhsTree = rightChildFragment.getPlanRoot();
     long rhsDataSize = -1;
     long broadcastCost = -1;
-    // TODO: IMPALA-4224: update this once we can share the broadcast join data between
-    // finstances.
     int mt_dop = ctx_.getQueryOptions().mt_dop;
-    int leftChildInstances = leftChildFragment.getNumInstances(mt_dop);
+    int leftChildNodes = leftChildFragment.getNumNodes();
     if (rhsTree.getCardinality() != -1) {
       rhsDataSize = Math.round(
           rhsTree.getCardinality() * ExchangeNode.getAvgSerializedRowSize(rhsTree));
-      if (leftChildInstances != -1) {
-        broadcastCost = 2 * rhsDataSize * leftChildInstances;
+      if (leftChildNodes != -1) {
+        // RHS data must be broadcast once to each node.
+        // TODO: IMPALA-9176: this is inaccurate for NAAJ until IMPALA-9176 is fixed
+        // because it must be broadcast once per instance.
+        broadcastCost = 2 * rhsDataSize * leftChildNodes;
       }
     }
     if (LOG.isTraceEnabled()) {
       LOG.trace("broadcast: cost=" + Long.toString(broadcastCost));
       LOG.trace("card=" + Long.toString(rhsTree.getCardinality()) + " row_size="
-          + Float.toString(rhsTree.getAvgRowSize()) + " #instances="
-          + Integer.toString(leftChildInstances));
+          + Float.toString(rhsTree.getAvgRowSize()) + " #nodes="
+          + Integer.toString(leftChildNodes));
     }
 
     // repartition: both left- and rightChildFragment are partitioned on the
@@ -563,9 +564,6 @@ public class DistributedPlanner {
    // don't broadcast very large inputs (for example in case the broadcast cost was
    // not computed correctly and the query mem limit has not been set or set too high)
    long htSize = Math.round(rhsDataSize * PlannerContext.HASH_TBL_SPACE_OVERHEAD);
-   // TODO: IMPALA-4224: update this once we can share the broadcast join data between
-   // finstances.
-   if (mt_dop > 1) htSize *= mt_dop;
    long memLimit = ctx_.getQueryOptions().mem_limit;
    long broadcast_bytes_limit = ctx_.getQueryOptions().getBroadcast_bytes_limit();
 
diff --git a/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java b/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
index 55830c2..b39a2ae 100644
--- a/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
@@ -44,7 +44,7 @@ public class EmptySetNode extends PlanNode {
   public void computeStats(Analyzer analyzer) {
     avgRowSize_ = 0;
     cardinality_ = 0;
-    numNodes_ = 1;
+    numInstances_ = numNodes_ = 1;
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
index b1ad34a..ce16d1d 100644
--- a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
@@ -17,8 +17,6 @@
 
 package org.apache.impala.planner;
 
-import java.util.List;
-
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.SortInfo;
@@ -194,7 +192,7 @@ public class ExchangeNode extends PlanNode {
     // lean towards the soft limits.
     Preconditions.checkState(!children_.isEmpty());
     Preconditions.checkNotNull(children_.get(0).getFragment());
-    int numSenders = children_.get(0).getFragment().getNumInstances(queryOptions.mt_dop);
+    int numSenders = children_.get(0).getFragment().getNumInstances();
     long estimatedTotalQueueByteSize = estimateTotalQueueByteSize(numSenders);
     long estimatedDeferredRPCQueueSize = estimateDeferredRPCQueueSize(queryOptions,
         numSenders);
diff --git a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
index c7fadc7..99dfb56 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
@@ -303,6 +303,7 @@ public class HBaseScanNode extends ScanNode {
       // We have run computeStats successfully. Don't need to estimate cardinality again
       // (IMPALA-8912). Check some invariants if computeStats has been called.
       Preconditions.checkState(numNodes_ > 0);
+      Preconditions.checkState(numInstances_ > 0);
       Preconditions.checkState(cardinality_ >= 0);
       cardinality_ = inputCardinality_;
       if (LOG.isTraceEnabled()) {
@@ -358,12 +359,17 @@ public class HBaseScanNode extends ScanNode {
       LOG.trace("computeStats HbaseScan: cardinality=" + cardinality_);
     }
 
-    // Assume that each executor in the cluster gets a scan range, unless there are fewer
-    // scan ranges than nodes.
-    numNodes_ = Math.max(1, Math.min(scanRangeSpecs_.getConcrete_rangesSize(),
-                                ExecutorMembershipSnapshot.getCluster().numExecutors()));
+    // Assume that each node/instance in the cluster gets a scan range, unless there are
+    // fewer scan ranges than nodes/instances.
+    int numExecutors = ExecutorMembershipSnapshot.getCluster().numExecutors();
+    numNodes_ =
+        Math.max(1, Math.min(scanRangeSpecs_.getConcrete_rangesSize(), numExecutors));
+    int maxInstances = numNodes_ * getMaxInstancesPerNode(analyzer);
+    numInstances_ =
+        Math.max(1, Math.min(scanRangeSpecs_.getConcrete_rangesSize(), maxInstances));
     if (LOG.isTraceEnabled()) {
-      LOG.trace("computeStats HbaseScan: #nodes=" + numNodes_);
+      LOG.trace(
+          "computeStats HbaseScan: #nodes=" + numNodes_ + " #instances=" + numInstances_);
     }
   }
 
diff --git a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
index 9e568ad..211cb13 100644
--- a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
@@ -217,25 +217,22 @@ public class HashJoinNode extends JoinNode {
   @Override
   public Pair<ResourceProfile, ResourceProfile> computeJoinResourceProfile(
       TQueryOptions queryOptions) {
-    long perInstanceMemEstimate;
-    long perInstanceDataBytes;
-    int numInstances = fragment_.getNumInstances(queryOptions.getMt_dop());
+    long perBuildInstanceMemEstimate;
+    long perBuildInstanceDataBytes;
+    int numInstances = fragment_.getNumInstances();
     if (getChild(1).getCardinality() == -1 || getChild(1).getAvgRowSize() == -1
         || numInstances <= 0) {
-      perInstanceMemEstimate = DEFAULT_PER_INSTANCE_MEM;
-      perInstanceDataBytes = -1;
+      perBuildInstanceMemEstimate = DEFAULT_PER_INSTANCE_MEM;
+      perBuildInstanceDataBytes = -1;
     } else {
-      // TODO: IMPALA-4224: update this once we can share the broadcast join data between
-      // finstances. Currently this implicitly assumes that each instance has a copy of
-      // the hash tables.
-      perInstanceDataBytes = (long) Math.ceil(getChild(1).cardinality_
+      perBuildInstanceDataBytes = (long) Math.ceil(getChild(1).cardinality_
           * getChild(1).avgRowSize_);
       // Assume the rows are evenly divided among instances.
       if (distrMode_ == DistributionMode.PARTITIONED) {
-        perInstanceDataBytes /= numInstances;
+        perBuildInstanceDataBytes /= numInstances;
       }
-      perInstanceMemEstimate = (long) Math.ceil(
-          perInstanceDataBytes * PlannerContext.HASH_TBL_SPACE_OVERHEAD);
+      perBuildInstanceMemEstimate = (long) Math.ceil(
+          perBuildInstanceDataBytes * PlannerContext.HASH_TBL_SPACE_OVERHEAD);
     }
 
     // Must be kept in sync with PartitionedHashJoinBuilder::MinReservation() in be.
@@ -244,8 +241,8 @@ public class HashJoinNode extends JoinNode {
         + (joinOp_ == JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN ? 1 : 0);
 
     long bufferSize = queryOptions.getDefault_spillable_buffer_size();
-    if (perInstanceDataBytes != -1) {
-      long bytesPerBuffer = perInstanceDataBytes / PARTITION_FANOUT;
+    if (perBuildInstanceDataBytes != -1) {
+      long bytesPerBuffer = perBuildInstanceDataBytes / PARTITION_FANOUT;
       // Scale down the buffer size if we think there will be excess free space with the
       // default buffer size, e.g. if the right side is a small dimension table.
       bufferSize = Math.min(bufferSize, Math.max(
@@ -259,6 +256,14 @@ public class HashJoinNode extends JoinNode {
         computeMaxSpillableBufferSize(bufferSize, queryOptions.getMax_row_size());
     long perInstanceBuildMinMemReservation =
         bufferSize * (minBuildBuffers - 2) + maxRowBufferSize * 2;
+    if (queryOptions.mt_dop > 0 && canShareBuild()) {
+      // Ensure we reserve enough memory to hand off to the PartitionedHashJoinNodes for
+      // probe streams when spilling. mt_dop is an upper bound on the number of
+      // PartitionedHashJoinNodes per builder.
+      // TODO: IMPALA-9416: be less conservative here
+      // TODO: how did we not detect the reservation bug here with spilling?
+      perInstanceBuildMinMemReservation *= queryOptions.mt_dop;
+    }
     // Most reservation for probe buffers is obtained from the join builder when
     // spilling. However, for NAAJ, two additional probe streams are needed that
     // are used exclusively by the probe side.
@@ -279,7 +284,7 @@ public class HashJoinNode extends JoinNode {
         .setSpillableBufferBytes(bufferSize)
         .setMaxRowBufferBytes(maxRowBufferSize).build();
     ResourceProfile buildProfile = new ResourceProfileBuilder()
-        .setMemEstimateBytes(perInstanceMemEstimate)
+        .setMemEstimateBytes(perBuildInstanceMemEstimate)
         .setMinMemReservationBytes(perInstanceBuildMinMemReservation)
         .setSpillableBufferBytes(bufferSize)
         .setMaxRowBufferBytes(maxRowBufferSize).build();
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
index 22dbc74..4ef381c 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -20,6 +20,7 @@ package org.apache.impala.planner;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -1237,10 +1238,16 @@ public class HdfsScanNode extends ScanNode {
   protected void computeNumNodes(Analyzer analyzer, long cardinality) {
     Preconditions.checkNotNull(scanRangeSpecs_);
     ExecutorMembershipSnapshot cluster = ExecutorMembershipSnapshot.getCluster();
-    Set<TNetworkAddress> localHostSet = new HashSet<>();
+    final int maxInstancesPerNode = getMaxInstancesPerNode(analyzer);
+    final int maxPossibleInstances = cluster.numExecutors() * maxInstancesPerNode;
     int totalNodes = 0;
+    int totalInstances = 0;
     int numLocalRanges = 0;
     int numRemoteRanges = 0;
+    // Counts the number of local ranges, capped at maxInstancesPerNode.
+    Map<TNetworkAddress, Integer> localRangeCounts = new HashMap<>();
+    // Sum of the counter values in localRangeCounts.
+    int totalLocalParallelism = 0;
     if (scanRangeSpecs_.isSetConcrete_ranges()) {
       if (analyzer.getQueryOptions().planner_testcase_mode) {
         // TODO: Have a separate scan node implementation that mocks an HDFS scan
@@ -1255,11 +1262,13 @@ public class HdfsScanNode extends ScanNode {
             ++numLocalRanges;
           }
         }
-        totalNodes = Math.min(
-            scanRangeSpecs_.concrete_ranges.size(), dummyHostIndex.size());
-        LOG.info(String.format("Planner running in DEBUG mode. ScanNode: %s, " +
-            "TotalNodes %d, Local Ranges %d", tbl_.getFullName(), totalNodes,
-            numLocalRanges));
+        totalNodes =
+            Math.min(scanRangeSpecs_.concrete_ranges.size(), dummyHostIndex.size());
+        totalInstances = Math.min(scanRangeSpecs_.concrete_ranges.size(),
+            totalNodes * maxInstancesPerNode);
+        LOG.info(String.format("Planner running in DEBUG mode. ScanNode: %s, "
+                + "TotalNodes %d, TotalInstances %d Local Ranges %d",
+            tbl_.getFullName(), totalNodes, totalInstances, numLocalRanges));
       } else {
         for (TScanRangeLocationList range : scanRangeSpecs_.concrete_ranges) {
           boolean anyLocal = false;
@@ -1274,7 +1283,11 @@ public class HdfsScanNode extends ScanNode {
                 // host.  This assumes that when an impalad is colocated with a datanode,
                 // there are the same number of impalads as datanodes on this host in this
                 // cluster.
-                localHostSet.add(dataNode);
+                int count = localRangeCounts.getOrDefault(dataNode, 0);
+                if (count < maxInstancesPerNode) {
+                  ++totalLocalParallelism;
+                  localRangeCounts.put(dataNode, count + 1);
+                }
               }
             }
           }
@@ -1288,37 +1301,63 @@ public class HdfsScanNode extends ScanNode {
           // Approximate the number of nodes that will execute locally assigned ranges to
           // be the smaller of the number of locally assigned ranges and the number of
           // hosts that hold block replica for those ranges.
-          int numLocalNodes = Math.min(numLocalRanges, localHostSet.size());
+          int numLocalNodes = Math.min(numLocalRanges, localRangeCounts.size());
           // The remote ranges are round-robined across all the impalads.
           int numRemoteNodes = Math.min(numRemoteRanges, cluster.numExecutors());
           // The local and remote assignments may overlap, but we don't know by how much
           // so conservatively assume no overlap.
           totalNodes = Math.min(numLocalNodes + numRemoteNodes, cluster.numExecutors());
-          // Exit early if all hosts have a scan range assignment, to avoid extraneous
-          // work in case the number of scan ranges dominates the number of nodes.
-          if (totalNodes == cluster.numExecutors()) break;
+          totalInstances = computeNumInstances(numLocalRanges, numRemoteRanges,
+              totalNodes, maxInstancesPerNode, totalLocalParallelism);
+          // Exit early if we have maxed out our estimate of hosts/instances, to avoid
+          // extraneous work in case the number of scan ranges dominates the number of
+          // nodes.
+          if (totalInstances == maxPossibleInstances) break;
         }
       }
     }
-    // Handle the generated range specifications.
-    if (totalNodes < cluster.numExecutors() && scanRangeSpecs_.isSetSplit_specs()) {
+    // Handle the generated range specifications, which may increase our estimates of
+    // number of nodes.
+    if (totalInstances < maxPossibleInstances && scanRangeSpecs_.isSetSplit_specs()) {
       Preconditions.checkState(
           generatedScanRangeCount_ >= scanRangeSpecs_.getSplit_specsSize());
       numRemoteRanges += generatedScanRangeCount_;
-      totalNodes = Math.min(numRemoteRanges, cluster.numExecutors());
+      int numLocalNodes = Math.min(numLocalRanges, localRangeCounts.size());
+      totalNodes = Math.min(numLocalNodes + numRemoteRanges, cluster.numExecutors());
+      totalInstances = computeNumInstances(numLocalRanges, numRemoteRanges,
+          totalNodes, maxInstancesPerNode, totalLocalParallelism);
     }
     // Tables can reside on 0 nodes (empty table), but a plan node must always be
     // executed on at least one node.
     numNodes_ = (cardinality == 0 || totalNodes == 0) ? 1 : totalNodes;
+    numInstances_ = (cardinality == 0 || totalInstances == 0) ? 1 : totalInstances;
     if (LOG.isTraceEnabled()) {
       LOG.trace("computeNumNodes totalRanges="
           + (scanRangeSpecs_.getConcrete_rangesSize() + generatedScanRangeCount_)
           + " localRanges=" + numLocalRanges + " remoteRanges=" + numRemoteRanges
-          + " localHostSet.size=" + localHostSet.size()
-          + " executorNodes=" + cluster.numExecutors());
+          + " localRangeCounts.size=" + localRangeCounts.size()
+          + " totalLocalParallelism=" + totalLocalParallelism
+          + " executorNodes=" + cluster.numExecutors() + " "
+          + " numNodes=" + numNodes_ + " numInstances=" + numInstances_);
     }
   }
 
+  /**
+   * Compute an estimate of the number of instances based on the total number of local
+   * and remote ranges, the estimated number of nodes, the maximum possible number
+   * of instances we can schedule on a node, and the total number of local instances
+   * across nodes.
+   */
+  private int computeNumInstances(int numLocalRanges, int numRemoteRanges, int numNodes,
+      int maxInstancesPerNode, int totalLocalParallelism) {
+    // Estimate the total number of instances, based on two upper bounds:
+    // * The number of scan ranges to process, excluding local ranges in excess of
+    //   maxInstancesPerNode.
+    // * The maximum parallelism allowed across all the nodes that will participate.
+    int numLocalInstances = Math.min(numLocalRanges, totalLocalParallelism);
+    return Math.min(numLocalInstances + numRemoteRanges, numNodes * maxInstancesPerNode);
+  }
+
   @Override
   protected void toThrift(TPlanNode msg) {
     msg.hdfs_scan_node = new THdfsScanNode(desc_.getId().asInt());
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
index 5ddc63d..970b088 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
@@ -92,7 +92,7 @@ public class HdfsTableSink extends TableSink {
   @Override
   public void computeResourceProfile(TQueryOptions queryOptions) {
     PlanNode inputNode = fragment_.getPlanRoot();
-    int numInstances = fragment_.getNumInstances(queryOptions.getMt_dop());
+    int numInstances = fragment_.getNumInstances();
     // Compute the number of partitions buffered in memory at the same time, taking into
     // account the number of nodes and the data partition of the fragment executing this
     // sink.
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
index 75af109..19032c7 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
@@ -28,6 +28,9 @@ import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TJoinBuildSink;
 import org.apache.impala.thrift.TQueryOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import com.google.common.base.Preconditions;
 
 import static org.apache.impala.analysis.ToSqlOptions.DEFAULT;
@@ -36,6 +39,8 @@ import static org.apache.impala.analysis.ToSqlOptions.DEFAULT;
  * Sink to materialize the build side of a join.
  */
 public class JoinBuildSink extends DataSink {
+  private final static Logger LOG = LoggerFactory.getLogger(JoinBuildSink.class);
+
   // id of join's build-side table assigned during planning
   private final JoinTableId joinTableId_;
 
@@ -81,6 +86,7 @@ public class JoinBuildSink extends DataSink {
     for (RuntimeFilter filter : runtimeFilters_) {
       tBuildSink.addToRuntime_filters(filter.toThrift());
     }
+    tBuildSink.setShare_build(joinNode_.canShareBuild());
     tsink.setJoin_build_sink(tBuildSink);
   }
 
@@ -115,6 +121,25 @@ public class JoinBuildSink extends DataSink {
     }
   }
 
+  /**
+   * Return an estimate of the number of nodes the fragment with this sink will run
+   * on. This is based on the number of nodes of the join node, since they are
+   * co-located.
+   */
+  public int getNumNodes() {
+    return joinNode_.getFragment().getNumNodes();
+  }
+
+  /**
+   * Return an estimate of the number of instances the fragment with this sink will run
+   * on. This is based on the number of instances or nodes of the join node, since they
+   * are co-located, but the build may be shared.
+   */
+  public int getNumInstances() {
+    return joinNode_.canShareBuild() ? joinNode_.getFragment().getNumNodes() :
+                                       joinNode_.getFragment().getNumInstances();
+  }
+
   @Override
   protected String getLabel() {
     return "JOIN BUILD";
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinNode.java b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
index a3d6042..903e492 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
@@ -190,6 +190,14 @@ public abstract class JoinNode extends PlanNode {
     return true;
   }
 
+  // Returns true if we can share a join build between multiple consuming fragment
+  // instances.
+  public boolean canShareBuild() {
+    // TODO: IMPALA-9176: null-aware anti-join doesn't support join build sharing.
+    if (joinOp_ == JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN) return false;
+    return distrMode_ == JoinNode.DistributionMode.BROADCAST;
+  }
+
   public JoinOperator getJoinOp() { return joinOp_; }
   public List<BinaryPredicate> getEqJoinConjuncts() { return eqJoinConjuncts_; }
   public List<Expr> getOtherJoinConjuncts() { return otherJoinConjuncts_; }
diff --git a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
index c00df7d..c9d158d 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -30,9 +30,6 @@ import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.BoolLiteral;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.ExprSubstitutionMap;
-import org.apache.impala.analysis.FunctionCallExpr;
-import org.apache.impala.analysis.FunctionName;
-import org.apache.impala.analysis.FunctionParams;
 import org.apache.impala.analysis.InPredicate;
 import org.apache.impala.analysis.IsNullPredicate;
 import org.apache.impala.analysis.LiteralExpr;
@@ -42,7 +39,6 @@ import org.apache.impala.analysis.SlotRef;
 import org.apache.impala.analysis.StringLiteral;
 import org.apache.impala.analysis.TupleDescriptor;
 import org.apache.impala.catalog.FeKuduTable;
-import org.apache.impala.catalog.HdfsFileFormat;
 import org.apache.impala.catalog.KuduColumn;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.ImpalaRuntimeException;
@@ -283,6 +279,11 @@ public class KuduScanNode extends ScanNode {
     super.computeStats(analyzer);
     // Update the number of nodes to reflect the hosts that have relevant data.
     numNodes_ = Math.max(1, hostIndexSet_.size());
+    // Estimate the total number of instances, based on two upper bounds:
+    // * The number of scan ranges to process.
+    // * The maximum parallelism allowed across all the hosts.
+    numInstances_ = Math.min(scanRangeSpecs_.getConcrete_rangesSize(),
+        numNodes_ * getMaxInstancesPerNode(analyzer));
 
     // Update the cardinality
     inputCardinality_ = cardinality_ = kuduTable_.getNumRows();
diff --git a/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java b/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
index d0b83e9..b72ec03 100644
--- a/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
@@ -178,7 +178,7 @@ public class ParallelPlanner {
       join.getFragment().removeChild(inputFragments.get(i));
       buildFragment.getChildren().add(inputFragments.get(i));
     }
-    join.getFragment().addChild(buildFragment);
+    buildFragment.setDestination(join);
 
     // assign plan and cohort id
     buildFragment.setPlanId(planIdGenerator_.getNextId());
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
index fb23f0e..e816edd 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -84,8 +84,8 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   // root of plan tree executed by this fragment
   private PlanNode planRoot_;
 
-  // exchange node to which this fragment sends its output
-  private ExchangeNode destNode_;
+  // exchange node or join node to which this fragment sends its output
+  private PlanNode destNode_;
 
   // created in finalize() or set in setSink()
   private DataSink sink_;
@@ -195,10 +195,11 @@ public class PlanFragment extends TreeNode<PlanFragment> {
    * among senders if the partition-expr types are not identical.
    */
   public void finalizeExchanges(Analyzer analyzer) throws InternalException {
-    if (destNode_ != null) {
+    if (destNode_ != null && destNode_ instanceof ExchangeNode) {
       Preconditions.checkState(sink_ == null);
       // we're streaming to an exchange node
-      DataStreamSink streamSink = new DataStreamSink(destNode_, outputPartition_);
+      DataStreamSink streamSink =
+          new DataStreamSink((ExchangeNode)destNode_, outputPartition_);
       streamSink.setFragment(this);
       sink_ = streamSink;
     }
@@ -364,32 +365,49 @@ public class PlanFragment extends TreeNode<PlanFragment> {
    * invalid: -1
    */
   public int getNumNodes() {
-    return dataPartition_ == DataPartition.UNPARTITIONED ? 1 : planRoot_.getNumNodes();
+    if (dataPartition_ == DataPartition.UNPARTITIONED) {
+      return 1;
+    } else if (sink_ instanceof JoinBuildSink) {
+      // One instance is scheduled per node, for all instances of the fragment containing
+      // the destination join node. ParallelPlanner sets the destination fragment when
+      // adding the JoinBuildSink.
+      return ((JoinBuildSink)sink_).getNumNodes();
+    } else {
+      return planRoot_.getNumNodes();
+    }
   }
 
   /**
-   * Return the number of instances of this fragment per host that it executes on.
-   * invalid: -1
+   * Return an estimate of the number of instances of this fragment per host that it
+   * executes on.
    */
   public int getNumInstancesPerHost(int mt_dop) {
-    Preconditions.checkState(mt_dop >= 0);
-    if (dataPartition_ == DataPartition.UNPARTITIONED) return 1;
-    return mt_dop == 0 ? 1 : mt_dop;
+    // Assume that instances are evenly divided across hosts.
+    int numNodes = getNumNodes();
+    int numInstances = getNumInstances();
+    // Fall back to assuming that all mt_dop instances will be generated.
+    if (numNodes == -1 || numInstances == -1) return Math.max(1, mt_dop);
+    return (int) Math.ceil((double)numInstances / (double)numNodes);
   }
 
   /**
    * Return the total number of instances of this fragment across all hosts.
    * invalid: -1
    */
-  public int getNumInstances(int mt_dop) {
-    if (dataPartition_ == DataPartition.UNPARTITIONED) return 1;
-    int numNodes = planRoot_.getNumNodes();
-    if (numNodes == -1) return -1;
-    return getNumInstancesPerHost(mt_dop) * numNodes;
+  public int getNumInstances() {
+    if (dataPartition_ == DataPartition.UNPARTITIONED) {
+      return 1;
+    } else if (sink_ instanceof JoinBuildSink) {
+      // One instance is scheduled per instance of the fragment containing the destination
+      // join. ParallelPlanner sets the destination fragment when adding the
+      // JoinBuildSink.
+      return ((JoinBuildSink)sink_).getNumInstances();
+    } else {
+      return planRoot_.getNumInstances();
+    }
   }
 
   /**
-    * Estimates the number of distinct values of exprs per fragment instance based on the
     * data partition of this fragment, the number of nodes, and the degree of parallelism.
     * Returns -1 for an invalid estimate, e.g., because getNumDistinctValues() failed on
     * one of the exprs.
@@ -397,7 +415,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   public long getPerInstanceNdv(int mt_dop, List<Expr> exprs) {
     Preconditions.checkNotNull(dataPartition_);
     long result = 1;
-    int numInstances = getNumInstances(mt_dop);
+    int numInstances = getNumInstances();
     Preconditions.checkState(numInstances >= 0);
     // The number of nodes is zero for empty tables.
     if (numInstances == 0) return 0;
@@ -509,7 +527,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     builder.append(String.format("%s%s:PLAN FRAGMENT [%s]", firstLinePrefix,
         fragmentId_.toString(), dataPartition_.getExplainString()));
     builder.append(PrintUtils.printNumHosts(" ", getNumNodes()));
-    builder.append(PrintUtils.printNumInstances(" ", getNumInstances(mt_dop)));
+    builder.append(PrintUtils.printNumInstances(" ", getNumInstances()));
     builder.append("\n");
     String perHostPrefix = mt_dop == 0 ?
         "Per-Host Resources: " : "Per-Host Shared Resources: ";
@@ -580,7 +598,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     if (destNode_ == null) return null;
     return destNode_.getFragment();
   }
-  public ExchangeNode getDestNode() { return destNode_; }
+  public PlanNode getDestNode() { return destNode_; }
   public DataPartition getDataPartition() { return dataPartition_; }
   public void setDataPartition(DataPartition dataPartition) {
     this.dataPartition_ = dataPartition;
@@ -595,7 +613,10 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     setFragmentInPlanTree(planRoot_);
   }
 
-  public void setDestination(ExchangeNode destNode) {
+  /**
+   * Set the destination node of this fragment's sink, i.e. an ExchangeNode or a JoinNode.
+   */
+  public void setDestination(PlanNode destNode) {
     destNode_ = destNode;
     PlanFragment dest = getDestFragment();
     Preconditions.checkNotNull(dest);
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanNode.java b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
index 2fc883e..ce492dd 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -122,10 +122,16 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   // invalid: -1
   protected long cardinality_;
 
-  // number of nodes on which the plan tree rooted at this node would execute;
+  // Estimated number of nodes on which the plan tree rooted at this node would be
+  // scheduled;
   // set in computeStats(); invalid: -1
   protected int numNodes_;
 
+  // Estimated number of instances across all nodes that the scheduler would generate for
+  // a fragment with the plan tree rooted at this node;
+  // set in computeStats(); invalid: -1
+  protected int numInstances_;
+
   // resource requirements and estimates for this plan node.
   // Initialized with a dummy value. Gets set correctly in
   // computeResourceProfile().
@@ -163,6 +169,7 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     tblRefIds_ = new ArrayList<>();
     cardinality_ = -1;
     numNodes_ = -1;
+    numInstances_ = -1;
     displayName_ = displayName;
     disableCodegen_ = false;
   }
@@ -179,6 +186,7 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     conjuncts_ = Expr.cloneList(node.conjuncts_);
     cardinality_ = -1;
     numNodes_ = -1;
+    numInstances_ = -1;
     displayName_ = displayName;
     disableCodegen_ = node.disableCodegen_;
   }
@@ -210,6 +218,7 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   public boolean hasLimit() { return limit_ > -1; }
   public long getCardinality() { return cardinality_; }
   public int getNumNodes() { return numNodes_; }
+  public int getNumInstances() { return numInstances_; }
   public ResourceProfile getNodeResourceProfile() { return nodeResourceProfile_; }
   public float getAvgRowSize() { return avgRowSize_; }
   public void setFragment(PlanFragment fragment) { fragment_ = fragment; }
@@ -547,7 +556,10 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
       TupleDescriptor desc = analyzer.getTupleDesc(tid);
       avgRowSize_ += desc.getAvgSerializedSize();
     }
-    if (!children_.isEmpty()) numNodes_ = getChild(0).numNodes_;
+    if (!children_.isEmpty()) {
+      numNodes_ = getChild(0).numNodes_;
+      numInstances_ = getChild(0).numInstances_;
+    }
   }
 
   protected long capCardinalityAtLimit(long cardinality) {
@@ -686,6 +698,7 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
    */
   protected boolean hasValidStats() {
     return (numNodes_ == -1 || numNodes_ >= 0) &&
+           (numInstances_ == -1 || numInstances_ >= 0) &&
            (cardinality_ == -1 || cardinality_ >= 0);
   }
 
@@ -860,6 +873,14 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     return sum;
   }
 
+  /**
+   * Returns the max number of instances of a fragment that can be scheduled on a single
+   * node - the mt_dop value if mt_dop > 0, or 1 otherwise.
+   */
+  protected int getMaxInstancesPerNode(Analyzer analyzer) {
+    return Math.max(1, analyzer.getQueryOptions().getMt_dop());
+  }
+
   protected void addRuntimeFilter(RuntimeFilter filter) { runtimeFilters_.add(filter); }
 
   protected Collection<RuntimeFilter> getRuntimeFilters() { return runtimeFilters_; }
diff --git a/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java b/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
index bdf3a01..e9c4984 100644
--- a/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
@@ -63,6 +63,7 @@ public class SingularRowSrcNode extends PlanNode {
     // The containing SubplanNode has not yet been initialized, so get the number
     // of nodes from the SubplanNode's input.
     numNodes_ = containingSubplanNode_.getChild(0).getNumNodes();
+    numInstances_ = containingSubplanNode_.getChild(0).getNumInstances();
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/planner/UnionNode.java b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
index e4148b0..0ec2457 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnionNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
@@ -124,6 +124,7 @@ public class UnionNode extends PlanNode {
       // correct iff the child fragments run on sets of nodes that are supersets or
       // subsets of each other, i.e. not just partly overlapping.
       numNodes_ = Math.max(child.getNumNodes(), numNodes_);
+      numInstances_ = Math.max(child.getNumInstances(), numInstances_);
     }
     // Consider estimate valid if we have at least one child with known cardinality, or
     // only constant values.
@@ -135,7 +136,10 @@ public class UnionNode extends PlanNode {
     // The number of nodes of a union node is -1 (invalid) if all the referenced tables
     // are inline views (e.g. select 1 FROM (VALUES(1 x, 1 y)) a FULL OUTER JOIN
     // (VALUES(1 x, 1 y)) b ON (a.x = b.y)). We need to set the correct value.
-    if (numNodes_ == -1) numNodes_ = 1;
+    if (numNodes_ == -1) {
+      numNodes_ = 1;
+      numInstances_ = 1;
+    }
     cardinality_ = capCardinalityAtLimit(cardinality_);
     if (LOG.isTraceEnabled()) {
       LOG.trace("stats Union: cardinality=" + Long.toString(cardinality_));
diff --git a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
index 4866961..496268d 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
@@ -70,6 +70,7 @@ public class UnnestNode extends PlanNode {
     // The containing SubplanNode has not yet been initialized, so get the number
     // of nodes from the SubplanNode's input.
     numNodes_ = containingSubplanNode_.getChild(0).getNumNodes();
+    numInstances_ = containingSubplanNode_.getChild(0).getNumInstances();
     cardinality_ = capCardinalityAtLimit(cardinality_);
   }
 
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
index cd08942..321d7bd 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
@@ -23,7 +23,7 @@ Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
      tuple-ids=0 row-size=124B cardinality=1
      in pipelines: 00(GETNEXT)
 
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=1
 Per-Host Resources: mem-estimate=1.88MB mem-reservation=0B thread-reservation=2
   DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -86,13 +86,13 @@ Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
      in pipelines: 00(GETNEXT)
 ---- DISTRIBUTEDPLAN
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-Per-Host Resources: mem-estimate=517.89KB mem-reservation=0B thread-reservation=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
   |
   01:EXCHANGE [UNPARTITIONED]
-     mem-estimate=517.89KB mem-reservation=0B thread-reservation=0
+     mem-estimate=517.93KB mem-reservation=0B thread-reservation=0
      tuple-ids=0 row-size=124B cardinality=3.32K
      in pipelines: 00(GETNEXT)
 
@@ -176,7 +176,7 @@ Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
      predicates: id IN (int_col), 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(int_col AS BIGINT) IN (CAST(9999999999 AS BIGINT)), 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')
      kudu predicates: double_col IN (CAST(0 AS DOUBLE)), float_col IN (CAST(0 AS FLOAT)), bigint_col IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT)), int_col IN (CAST(1 AS INT), CAST(2 AS INT)), smallint_col IN (CAST(0 AS SMALLINT), CAST(2 AS SMALLINT)), string_col IN ('foo', 'foo       '), tinyint_col IN (CAST(1 AS TINYINT), CAST(2 AS TINYINT)), bool_col IN (TRUE)
      mem-estimate=9.75MB mem-reservation=0B thread-reservation=1
-     tuple-ids=0 row-size=97B cardinality=5
+     tuple-ids=0 row-size=97B cardinality=6
      in pipelines: 00(GETNEXT)
 ====
 # Test decimal in-list predicates.
@@ -191,7 +191,7 @@ Per-Host Resources: mem-estimate=4.50MB mem-reservation=0B thread-reservation=2
   00:SCAN KUDU [functional_kudu.decimal_tbl]
      kudu predicates: d1 IN (CAST(1234 AS DECIMAL(9,0)), CAST(12345 AS DECIMAL(9,0)))
      mem-estimate=4.50MB mem-reservation=0B thread-reservation=1
-     tuple-ids=0 row-size=56B cardinality=2
+     tuple-ids=0 row-size=56B cardinality=3
      in pipelines: 00(GETNEXT)
 ====
 select * from functional_kudu.alltypes where
@@ -227,7 +227,7 @@ Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   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')
      mem-estimate=9.75MB mem-reservation=0B thread-reservation=1
-     tuple-ids=0 row-size=97B cardinality=1
+     tuple-ids=0 row-size=97B cardinality=2
      in pipelines: 00(GETNEXT)
 ====
 select * from functional_kudu.alltypes where
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 f36b5e2..d00afb8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
@@ -107,9 +107,9 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=86.00MB thread-reserva
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -164,9 +164,9 @@ Per-Host Resources: mem-estimate=124.02MB mem-reservation=74.00MB thread-reserva
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -256,10 +256,10 @@ Per-Host Resources: mem-estimate=99.65MB mem-reservation=66.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
@@ -308,9 +308,9 @@ Per-Host Resources: mem-estimate=806.43MB mem-reservation=74.00MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -363,9 +363,9 @@ Per-Host Resources: mem-estimate=168.14MB mem-reservation=50.00MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=16.00MB thread-reservation=1
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test b/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test
index 6d097f2..5cdf66c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test
@@ -54,9 +54,9 @@ PLAN-ROOT SINK
 ---- PARALLELPLANS
 PLAN-ROOT SINK
 |
-05:EXCHANGE [UNPARTITIONED]
+04:EXCHANGE [UNPARTITIONED]
 |
-02:HASH JOIN [INNER JOIN, PARTITIONED]
+02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c_nationkey = n_nationkey
 |  row-size=327B cardinality=150.00K
 |
@@ -65,14 +65,12 @@ PLAN-ROOT SINK
 |  |  build expressions: n_nationkey
 |  |  runtime filters: RF000 <- n_nationkey
 |  |
-|  04:EXCHANGE [HASH(n_nationkey)]
+|  03:EXCHANGE [BROADCAST]
 |  |
 |  01:SCAN HDFS [tpch.nation]
 |     HDFS partitions=1/1 files=1 size=2.15KB
 |     row-size=109B cardinality=25
 |
-03:EXCHANGE [HASH(c_nationkey)]
-|
 00:SCAN HDFS [tpch.customer]
    HDFS partitions=1/1 files=1 size=23.08MB
    runtime filters: RF000 -> c_nationkey
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation-hdfs-num-rows-est-enabled.test b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation-hdfs-num-rows-est-enabled.test
index ee04783..d493c4a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation-hdfs-num-rows-est-enabled.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation-hdfs-num-rows-est-enabled.test
@@ -22,29 +22,29 @@ PLAN-ROOT SINK
 |
 02:NESTED LOOP JOIN [CROSS JOIN]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=0,1 row-size=0B cardinality=550.56K
+|  tuple-ids=0,1 row-size=0B cardinality=574.56K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional_parquet.alltypestiny b]
-|     HDFS partitions=4/4 files=4 size=11.67KB
+|     HDFS partitions=4/4 files=4 size=11.92KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=unavailable
 |     mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=0
-|     tuple-ids=1 row-size=0B cardinality=742
+|     tuple-ids=1 row-size=0B cardinality=758
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny a]
-   HDFS partitions=4/4 files=4 size=11.67KB
+   HDFS partitions=4/4 files=4 size=11.92KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/4 rows=unavailable
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=0
-   tuple-ids=0 row-size=0B cardinality=742
+   tuple-ids=0 row-size=0B cardinality=758
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 NotImplementedException: MT_DOP not supported for plans with base table joins or table sinks.
@@ -72,22 +72,22 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF000[bloom] <- b.id
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=0,1 row-size=8B cardinality=742
+|  tuple-ids=0,1 row-size=8B cardinality=758
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional_parquet.alltypestiny b]
-|     HDFS partitions=4/4 files=4 size=11.67KB
+|     HDFS partitions=4/4 files=4 size=11.92KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
 |       columns: unavailable
 |     extrapolated-rows=disabled max-scan-range-rows=unavailable
 |     mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=0
-|     tuple-ids=1 row-size=4B cardinality=742
+|     tuple-ids=1 row-size=4B cardinality=758
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny a]
-   HDFS partitions=4/4 files=4 size=11.67KB
+   HDFS partitions=4/4 files=4 size=11.92KB
    runtime filters: RF000[bloom] -> a.id
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -95,7 +95,7 @@ PLAN-ROOT SINK
      columns: unavailable
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=0
-   tuple-ids=0 row-size=4B cardinality=742
+   tuple-ids=0 row-size=4B cardinality=758
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 NotImplementedException: MT_DOP not supported for plans with base table joins or table sinks.
@@ -127,7 +127,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -169,7 +169,7 @@ Per-Instance Resources: mem-estimate=128.18MB mem-reservation=34.00MB thread-res
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 03:EXCHANGE [HASH(bigint_col)]
-|  mem-estimate=186.69KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=186.68KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=16B cardinality=1.28K
 |  in pipelines: 00(GETNEXT)
 |
@@ -183,7 +183,7 @@ Per-Instance Resources: mem-estimate=144.00MB mem-reservation=34.02MB thread-res
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -223,7 +223,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -237,13 +237,13 @@ PLAN-ROOT SINK
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=222.69KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=222.68KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=222.69KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=222.68KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=4,3 row-size=16B cardinality=1.28K
 |  in pipelines: 01(GETNEXT)
 |
@@ -272,7 +272,7 @@ Per-Instance Resources: mem-estimate=10.11MB mem-reservation=10.00MB thread-rese
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
 Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
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 4e53644..2e69e8c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
@@ -21,7 +21,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional_parquet.alltypestiny b]
-|     HDFS partitions=4/4 files=4 size=11.67KB
+|     HDFS partitions=4/4 files=4 size=11.92KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
@@ -32,7 +32,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny a]
-   HDFS partitions=4/4 files=4 size=11.67KB
+   HDFS partitions=4/4 files=4 size=11.92KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/4 rows=unavailable
@@ -71,7 +71,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional_parquet.alltypestiny b]
-|     HDFS partitions=4/4 files=4 size=11.67KB
+|     HDFS partitions=4/4 files=4 size=11.92KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
@@ -82,7 +82,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny a]
-   HDFS partitions=4/4 files=4 size=11.67KB
+   HDFS partitions=4/4 files=4 size=11.92KB
    runtime filters: RF000[bloom] -> a.id
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -137,7 +137,7 @@ WRITE TO HDFS [default.ctas_mt_dop_test, OVERWRITE=false]
 |  mem-estimate=100.00KB mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/24 rows=unavailable
@@ -176,7 +176,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -232,7 +232,7 @@ Per-Instance Resources: mem-estimate=144.00MB mem-reservation=34.02MB thread-res
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -272,7 +272,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -321,7 +321,7 @@ Per-Instance Resources: mem-estimate=10.11MB mem-reservation=10.00MB thread-rese
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
 Per-Instance Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.71KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -390,14 +390,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   HDFS partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=289.17MB
    predicates: c_custkey < CAST(10 AS BIGINT), !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems), o_orderkey < CAST(5 AS BIGINT)
    predicates on o_lineitems: l_linenumber < CAST(3 AS INT)
    stored statistics:
-     table: rows=150.00K size=288.99MB
+     table: rows=150.00K size=289.17MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=50.12K
+   extrapolated-rows=disabled max-scan-range-rows=50.11K
    parquet statistics predicates: c_custkey < CAST(10 AS BIGINT)
    parquet statistics predicates on o: o_orderkey < CAST(5 AS BIGINT)
    parquet statistics predicates on o_lineitems: l_linenumber < CAST(3 AS INT)
@@ -409,17 +409,17 @@ PLAN-ROOT SINK
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=14.34MB mem-reservation=0B thread-reservation=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
 |
 09:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=14.34MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=11.93MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=2,1,0 row-size=482B cardinality=1.50M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4
 Per-Instance Resources: mem-estimate=104.00MB mem-reservation=104.00MB thread-reservation=1
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -466,14 +466,14 @@ Per-Instance Resources: mem-estimate=104.00MB mem-reservation=104.00MB thread-re
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM]
-   HDFS partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=289.17MB
    predicates: c_custkey < CAST(10 AS BIGINT), !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems), o_orderkey < CAST(5 AS BIGINT)
    predicates on o_lineitems: l_linenumber < CAST(3 AS INT)
    stored statistics:
-     table: rows=150.00K size=288.99MB
+     table: rows=150.00K size=289.17MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=50.12K
+   extrapolated-rows=disabled max-scan-range-rows=50.11K
    parquet statistics predicates: c_custkey < CAST(10 AS BIGINT)
    parquet statistics predicates on o: o_orderkey < CAST(5 AS BIGINT)
    parquet statistics predicates on o_lineitems: l_linenumber < CAST(3 AS INT)
@@ -531,13 +531,13 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   HDFS partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=289.17MB
    predicates: !empty(c.c_orders), !empty(c.c_orders)
    predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    stored statistics:
-     table: rows=150.00K size=288.99MB
+     table: rows=150.00K size=289.17MB
      columns missing stats: c_orders, c_orders
-   extrapolated-rows=disabled max-scan-range-rows=50.12K
+   extrapolated-rows=disabled max-scan-range-rows=50.11K
    parquet statistics predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    parquet dictionary predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    mem-estimate=88.00MB mem-reservation=16.00MB thread-reservation=0
@@ -545,17 +545,17 @@ PLAN-ROOT SINK
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=12.37MB mem-reservation=0B thread-reservation=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
 |
 07:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=12.37MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=11.05MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1,0,2 row-size=258B cardinality=1.50M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4
 Per-Instance Resources: mem-estimate=89.94MB mem-reservation=17.94MB thread-reservation=1
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -593,13 +593,13 @@ Per-Instance Resources: mem-estimate=89.94MB mem-reservation=17.94MB thread-rese
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM]
-   HDFS partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=289.17MB
    predicates: !empty(c.c_orders), !empty(c.c_orders)
    predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    stored statistics:
-     table: rows=150.00K size=288.99MB
+     table: rows=150.00K size=289.17MB
      columns missing stats: c_orders, c_orders
-   extrapolated-rows=disabled max-scan-range-rows=50.12K
+   extrapolated-rows=disabled max-scan-range-rows=50.11K
    parquet statistics predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    parquet dictionary predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    mem-estimate=88.00MB mem-reservation=16.00MB 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 acc0ccd..3cd02c5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -48,22 +48,22 @@ 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=80.00MB Threads=3
-Per-Host Resource Estimates: Memory=171MB
+Max Per-Host Resource Reservation: Memory=40.00MB Threads=2
+Per-Host Resource Estimates: Memory=91MB
 Analyzed query: SELECT * FROM tpch_parquet.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=10.69MB mem-reservation=0B 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
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=11.38MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=231B cardinality=6.00M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Instance Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
    HDFS partitions=1/1 files=3 size=193.99MB
@@ -126,22 +126,22 @@ 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=8.00MB Threads=3
-Per-Host Resource Estimates: Memory=170MB
+Max Per-Host Resource Reservation: Memory=4.00MB Threads=2
+Per-Host Resource Estimates: Memory=90MB
 Analyzed query: SELECT l_comment FROM tpch_parquet.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=10.12MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=10.25MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.12MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=38B cardinality=6.00M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Instance Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
    HDFS partitions=1/1 files=3 size=193.99MB
@@ -1181,24 +1181,24 @@ 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=16.00MB Threads=3
-Per-Host Resource Estimates: Memory=176MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
+Per-Host Resource Estimates: Memory=88MB
 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=367.96KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=183.96KB mem-reservation=0B 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
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=367.96KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=183.96KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=88B cardinality=unavailable
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
 00:SCAN HDFS [tpch_avro.orders, RANDOM]
    HDFS partitions=1/1 files=2 size=156.92MB
@@ -1320,24 +1320,24 @@ 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=16.00MB Threads=3
-Per-Host Resource Estimates: Memory=64MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
+Per-Host Resource Estimates: Memory=32MB
 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=163.92KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=81.92KB mem-reservation=0B 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
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=163.92KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=81.92KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=78B cardinality=unavailable
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
 00:SCAN HDFS [tpch_rc.customer, RANDOM]
    HDFS partitions=1/1 files=1 size=22.47MB
@@ -1403,24 +1403,24 @@ 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=16.00MB Threads=3
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=8.00MB Threads=2
+Per-Host Resource Estimates: Memory=16MB
 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=215.90KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=107.90KB mem-reservation=0B 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
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=215.90KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=107.90KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=104B cardinality=unavailable
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=1
 00:SCAN HDFS [tpcds_seq_snap.web_returns, RANDOM]
    HDFS partitions=1/1 files=1 size=6.61MB
@@ -1624,17 +1624,17 @@ functional.alltypesmixedformat
 Analyzed query: SELECT * FROM functional.alltypesmixedformat
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=503.97KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=335.97KB mem-reservation=0B 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
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=503.97KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=335.97KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=80B cardinality=unavailable
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4
 Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=1
 00:SCAN HDFS [functional.alltypesmixedformat, RANDOM]
    HDFS partitions=4/4 files=4 size=66.33KB
@@ -1697,24 +1697,24 @@ 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=3
+Max Per-Host Resource Reservation: Memory=0B 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.25MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=1.17MB mem-reservation=0B 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
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=1.25MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0 row-size=80B cardinality=14.23K
+|  mem-estimate=1.17MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0 row-size=80B cardinality=14.30K
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 Per-Instance Resources: mem-estimate=256.00KB mem-reservation=0B thread-reservation=1
 00:SCAN HBASE [functional_hbase.alltypes]
    stored statistics:
@@ -1772,23 +1772,23 @@ 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=3
+Max Per-Host Resource Reservation: Memory=0B 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=28.69KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=16.00KB mem-reservation=0B 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
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=28.69KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=89B cardinality=50
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Instance Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservation=1
 00:SCAN HBASE [functional_hbase.alltypessmall]
    stored statistics:
@@ -1842,24 +1842,24 @@ 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=3
-Per-Host Resource Estimates: Memory=2.00GB
+Max Per-Host Resource Reservation: Memory=0B 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=806.41KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=686.41KB mem-reservation=0B 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
 |
 01:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=806.41KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=686.41KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=116B cardinality=5.00K
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 Per-Instance Resources: mem-estimate=1.00GB mem-reservation=0B thread-reservation=1
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
    mem-estimate=1.00GB mem-reservation=0B thread-reservation=0
@@ -2078,37 +2078,37 @@ 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=85.50MB Threads=5
-Per-Host Resource Estimates: Memory=272MB
+Max Per-Host Resource Reservation: Memory=46.50MB Threads=3
+Per-Host Resource Estimates: Memory=140MB
 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.07MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=8.01MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=8.07MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=8.01MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=16B cardinality=1.56M
 |  in pipelines: 03(GETNEXT)
 |
-F01:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=18.07MB mem-reservation=4.75MB thread-reservation=1
+F01:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=18.01MB mem-reservation=8.50MB thread-reservation=1
 03:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |  group by: l_orderkey
-|  mem-estimate=10.00MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  tuple-ids=1 row-size=16B cardinality=1.56M
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
 02:EXCHANGE [HASH(l_orderkey)]
-|  mem-estimate=8.07MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=8.01MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=16B cardinality=1.56M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Instance Resources: mem-estimate=114.00MB mem-reservation=38.00MB thread-reservation=1
 01:AGGREGATE [STREAMING]
 |  output: count(*)
@@ -2195,8 +2195,8 @@ 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=256.00KB Threads=3
-Per-Host Resource Estimates: Memory=190MB
+Max Per-Host Resource Reservation: Memory=128.00KB Threads=2
+Per-Host Resource Estimates: Memory=100MB
 Analyzed query: SELECT count(*) FROM tpch_parquet.lineitem
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -2216,7 +2216,7 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=8B cardinality=1
 |  in pipelines: 01(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Instance Resources: mem-estimate=90.00MB mem-reservation=128.00KB thread-reservation=1
 01:AGGREGATE
 |  output: sum_init_zero(tpch_parquet.lineitem.stats: num_rows)
@@ -2299,23 +2299,23 @@ 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=104.00MB Threads=3
-Per-Host Resource Estimates: Memory=297MB
+Max Per-Host Resource Reservation: Memory=52.00MB Threads=2
+Per-Host Resource Estimates: Memory=149MB
 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=61.38MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=30.69MB mem-reservation=0B 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
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_comment ASC
-|  mem-estimate=61.38MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=30.69MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=231B cardinality=6.00M
 |  in pipelines: 01(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Instance Resources: mem-estimate=118.00MB mem-reservation=52.00MB thread-reservation=1
 01:SORT
 |  order by: l_comment ASC
@@ -2402,13 +2402,13 @@ 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=80.00MB Threads=3
-Per-Host Resource Estimates: Memory=160MB
+Max Per-Host Resource Reservation: Memory=40.00MB Threads=2
+Per-Host Resource Estimates: Memory=80MB
 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=145.08KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=76.29KB mem-reservation=0B 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
@@ -2416,11 +2416,11 @@ PLAN-ROOT SINK
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_comment ASC
 |  limit: 100
-|  mem-estimate=145.08KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=76.29KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=231B cardinality=100
 |  in pipelines: 01(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Instance Resources: mem-estimate=80.02MB mem-reservation=40.00MB thread-reservation=1
 01:TOP-N [LIMIT=100]
 |  order by: l_comment ASC
@@ -2429,7 +2429,7 @@ Per-Instance Resources: mem-estimate=80.02MB mem-reservation=40.00MB thread-rese
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
      table: rows=6.00M size=193.99MB
      columns: all
@@ -2536,25 +2536,26 @@ 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=103.00MB Threads=9
-Per-Host Resource Estimates: Memory=501MB
+Max Per-Host Resource Reservation: Memory=94.00MB Threads=5
+Per-Host Resource Estimates: Memory=558MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey
 = o_orderkey
 
-F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=12.40MB mem-reservation=0B 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
 |
-05:EXCHANGE [UNPARTITIONED]
+04:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=12.40MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1 row-size=402B cardinality=5.76M
 |  in pipelines: 00(GETNEXT)
 |
-F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
-02:HASH JOIN [INNER JOIN, PARTITIONED]
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
+02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
@@ -2562,20 +2563,20 @@ Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservati
 |  tuple-ids=0,1 row-size=402B cardinality=5.76M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F04:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4
-|  |  Per-Instance Resources: mem-estimate=56.51MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=280.29MB mem-reservation=69.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: o_orderkey
 |  |  runtime filters: RF000[bloom] <- o_orderkey
-|  |  mem-estimate=44.82MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  mem-estimate=268.94MB mem-reservation=68.00MB spill-buffer=2.00MB thread-reservation=0
 |  |
-|  04:EXCHANGE [HASH(o_orderkey)]
-|  |  mem-estimate=10.68MB mem-reservation=0B thread-reservation=0
+|  03:EXCHANGE [BROADCAST]
+|  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=171B cardinality=1.50M
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     HDFS partitions=1/1 files=1 size=162.56MB
@@ -2587,14 +2588,6 @@ Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservati
 |     tuple-ids=1 row-size=171B cardinality=1.50M
 |     in pipelines: 01(GETNEXT)
 |
-03:EXCHANGE [HASH(l_orderkey)]
-|  mem-estimate=11.38MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0 row-size=231B cardinality=6.00M
-|  in pipelines: 00(GETNEXT)
-|
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
 00:SCAN HDFS [tpch.lineitem, RANDOM]
    HDFS partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000[bloom] -> l_orderkey
@@ -2711,8 +2704,8 @@ 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=103.00MB Threads=9
-Per-Host Resource Estimates: Memory=501MB
+Max Per-Host Resource Reservation: Memory=95.00MB Threads=8
+Per-Host Resource Estimates: Memory=412MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN /* +shuffle */
 tpch.orders ON l_orderkey = o_orderkey
 
@@ -2737,8 +2730,8 @@ Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservati
 |  tuple-ids=0,1 row-size=402B cardinality=5.76M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F04:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4
-|  |  Per-Instance Resources: mem-estimate=56.51MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F04:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=56.17MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: o_orderkey
@@ -2746,11 +2739,11 @@ Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservati
 |  |  mem-estimate=44.82MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  |
 |  04:EXCHANGE [HASH(o_orderkey)]
-|  |  mem-estimate=10.68MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=171B cardinality=1.50M
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     HDFS partitions=1/1 files=1 size=162.56MB
@@ -2869,8 +2862,8 @@ 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=32.00MB Threads=7
-Per-Host Resource Estimates: Memory=875MB
+Max Per-Host Resource Reservation: Memory=24.00MB Threads=5
+Per-Host Resource Estimates: Memory=531MB
 Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -2892,18 +2885,18 @@ Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reser
 |  tuple-ids=0,1 row-size=402B cardinality=9.00T
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-|  |  Per-Instance Resources: mem-estimate=255.18MB mem-reservation=0B thread-reservation=1
+|--F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=254.84MB mem-reservation=0B thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  mem-estimate=244.49MB mem-reservation=0B thread-reservation=0
 |  |
 |  03:EXCHANGE [BROADCAST]
-|  |  mem-estimate=10.68MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=171B cardinality=1.50M
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     HDFS partitions=1/1 files=1 size=162.56MB
@@ -3259,14 +3252,14 @@ Per-Host Resources: mem-estimate=56.00MB mem-reservation=36.00MB thread-reservat
    tuple-ids=0 row-size=171B cardinality=1.50M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=104.00MB Threads=3
-Per-Host Resource Estimates: Memory=181MB
+Max Per-Host Resource Reservation: Memory=68.00MB Threads=2
+Per-Host Resource Estimates: Memory=104MB
 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=68.68MB mem-reservation=32.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=48.34MB 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
@@ -3309,11 +3302,11 @@ PLAN-ROOT SINK
 |
 07:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_totalprice ASC
-|  mem-estimate=40.68MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=20.34MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=6 row-size=171B cardinality=1.50M
 |  in pipelines: 01(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 Per-Instance Resources: mem-estimate=56.00MB mem-reservation=36.00MB thread-reservation=1
 01:SORT
 |  order by: o_totalprice ASC
@@ -3642,8 +3635,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=264.50MB Threads=21
-Per-Host Resource Estimates: Memory=879MB
+Max Per-Host Resource Reservation: Memory=186.75MB Threads=10
+Per-Host Resource Estimates: Memory=437MB
 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
@@ -3653,48 +3646,48 @@ WHERE o_orderpriority = '2-HIGH' UNION ALL SELECT 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_shipmode = 'F'
 
-F10:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=10.41MB mem-reservation=0B thread-reservation=1
+F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=10.21MB mem-reservation=0B 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
 |
-18:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=10.41MB mem-reservation=0B thread-reservation=0
+17:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=10.21MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=7 row-size=66B cardinality=2.55M
 |  in pipelines: 14(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
-F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-reservation=1
+F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=80.00MB mem-reservation=34.00MB thread-reservation=1
 00:UNION
 |  pass-through-operands: 14
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  tuple-ids=7 row-size=66B cardinality=2.55M
 |  in pipelines: 14(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
-|--10:HASH JOIN [INNER JOIN, PARTITIONED]
+|--10:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=01
 |  |  hash predicates: l_orderkey = o_orderkey
 |  |  fk/pk conjuncts: l_orderkey = o_orderkey
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=5,6 row-size=91B cardinality=822.53K
 |  |  in pipelines: 08(GETNEXT), 09(OPEN)
 |  |
-|  |--F12:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-|  |  |  Per-Instance Resources: mem-estimate=9.64MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |--F11:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=45.02MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  |  build expressions: o_orderkey
 |  |  |  runtime filters: RF004[bloom] <- o_orderkey
-|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  |
-|  |  17:EXCHANGE [HASH(o_orderkey)]
-|  |  |  mem-estimate=5.77MB mem-reservation=0B thread-reservation=0
+|  |  16:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=10.02MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=6 row-size=8B cardinality=1.50M
 |  |  |  in pipelines: 09(GETNEXT)
 |  |  |
-|  |  F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1
 |  |  09:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |  |     HDFS partitions=1/1 files=2 size=54.21MB
@@ -3706,14 +3699,6 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-rese
 |  |     tuple-ids=6 row-size=8B cardinality=1.50M
 |  |     in pipelines: 09(GETNEXT)
 |  |
-|  16:EXCHANGE [HASH(l_orderkey)]
-|  |  mem-estimate=10.51MB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=5 row-size=83B cardinality=857.32K
-|  |  in pipelines: 08(GETNEXT)
-|  |
-|  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-|  Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-reservation=1
 |  08:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
 |     HDFS partitions=1/1 files=3 size=193.99MB
 |     predicates: l_shipmode = 'F'
@@ -3736,20 +3721,20 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-rese
 |  |  tuple-ids=3,4 row-size=95B cardinality=1.15M
 |  |  in pipelines: 05(GETNEXT), 06(OPEN)
 |  |
-|  |--F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-|  |  |  Per-Instance Resources: mem-estimate=26.25MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |--F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=43.19MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  |  build expressions: o_orderkey
 |  |  |  runtime filters: RF002[bloom] <- o_orderkey
-|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  |
 |  |  15:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=8.25MB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=8.19MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=4 row-size=28B cardinality=300.00K
 |  |  |  in pipelines: 06(GETNEXT)
 |  |  |
-|  |  F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  Per-Instance Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=1
 |  |  06:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |  |     HDFS partitions=1/1 files=2 size=54.21MB
@@ -3777,20 +3762,20 @@ Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-rese
 |
 14:AGGREGATE [FINALIZE]
 |  group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=2 row-size=66B cardinality=575.77K
 |  in pipelines: 14(GETNEXT), 01(OPEN)
 |
 13:EXCHANGE [HASH(l_orderkey,l_partkey,l_suppkey,l_linenumber,l_comment)]
-|  mem-estimate=10.41MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.21MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=2 row-size=66B cardinality=575.77K
 |  in pipelines: 01(GETNEXT)
 |
-F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=27.46MB mem-reservation=17.00MB thread-reservation=1
+F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=44.23MB mem-reservation=34.00MB thread-reservation=1
 04:AGGREGATE [STREAMING]
 |  group by: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
-|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=2 row-size=66B cardinality=575.77K
 |  in pipelines: 01(GETNEXT)
 |
@@ -3798,24 +3783,24 @@ Per-Instance Resources: mem-estimate=27.46MB mem-reservation=17.00MB thread-rese
 |  hash-table-id=02
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
-|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
 |  tuple-ids=0,1 row-size=82B cardinality=575.77K
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
-|--F13:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4
-|  |  Per-Instance Resources: mem-estimate=9.64MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F12:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=11.50MB mem-reservation=5.75MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: o_orderkey
 |  |  runtime filters: RF000[bloom] <- o_orderkey
-|  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
 |  |
 |  12:EXCHANGE [HASH(o_orderkey)]
-|  |  mem-estimate=5.77MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=5.75MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=8B cardinality=1.50M
 |  |  in pipelines: 02(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1
 |  02:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     HDFS partitions=1/1 files=2 size=54.21MB
@@ -3828,11 +3813,11 @@ Per-Instance Resources: mem-estimate=27.46MB mem-reservation=17.00MB thread-rese
 |     in pipelines: 02(GETNEXT)
 |
 11:EXCHANGE [HASH(l_orderkey)]
-|  mem-estimate=10.46MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.23MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=74B cardinality=600.12K
 |  in pipelines: 01(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 Per-Instance Resources: mem-estimate=80.00MB mem-reservation=24.00MB thread-reservation=1
 01:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
@@ -4147,8 +4132,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=298.88MB Threads=19
-Per-Host Resource Estimates: Memory=924MB
+Max Per-Host Resource Reservation: Memory=281.88MB Threads=16
+Per-Host Resource Estimates: Memory=797MB
 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)
@@ -4254,20 +4239,20 @@ Per-Instance Resources: mem-estimate=27.12MB mem-reservation=17.00MB thread-rese
 |  tuple-ids=2,1,0 row-size=100B cardinality=5.76M
 |  in pipelines: 02(GETNEXT), 00(OPEN)
 |
-|--F09:PLAN FRAGMENT [HASH(l_orderkey)] hosts=1 instances=2
-|  |  Per-Instance Resources: mem-estimate=15.02MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F09:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=23.48MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: c_custkey
 |  |  runtime filters: RF002[bloom] <- c_custkey
-|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=512.00KB thread-reservation=0
 |  |
 |  12:EXCHANGE [BROADCAST]
-|  |  mem-estimate=5.52MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=5.48MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=0 row-size=38B cardinality=150.00K
 |  |  in pipelines: 00(GETNEXT)
 |  |
-|  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+|  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=1
 |  00:SCAN HDFS [tpch.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=23.08MB
@@ -4287,8 +4272,8 @@ Per-Instance Resources: mem-estimate=27.12MB mem-reservation=17.00MB thread-rese
 |  tuple-ids=2,1 row-size=62B cardinality=5.76M
 |  in pipelines: 02(GETNEXT), 01(OPEN)
 |
-|--F10:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4
-|  |  Per-Instance Resources: mem-estimate=28.20MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F10:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
+|  |  Per-Instance Resources: mem-estimate=28.10MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: o_orderkey
@@ -4296,11 +4281,11 @@ Per-Instance Resources: mem-estimate=27.12MB mem-reservation=17.00MB thread-rese
 |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |
 |  11:EXCHANGE [HASH(o_orderkey)]
-|  |  mem-estimate=10.20MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=10.10MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=46B cardinality=1.50M
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
 |  Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch.orders, RANDOM]
@@ -4394,20 +4379,20 @@ WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false]
    tuple-ids=0 row-size=81B cardinality=310
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=64.00KB Threads=2
-Per-Host Resource Estimates: Memory=32MB
+Max Per-Host Resource Reservation: Memory=32.00KB Threads=1
+Per-Host Resource Estimates: Memory=16MB
 Codegen disabled by planner
 Analyzed query: SELECT id, bool_col, tinyint_col, smallint_col, int_col,
 bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
 FROM functional.alltypes WHERE `year` = CAST(2009 AS INT) AND `month` = CAST(5
 AS INT)
 
-F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  Per-Instance Resources: mem-estimate=16.01MB mem-reservation=32.00KB thread-reservation=1
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.02MB mem-reservation=32.00KB thread-reservation=1
 WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false]
 |  partitions=1
 |  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
-|  mem-estimate=12.26KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=24.52KB mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes, RANDOM]
    partition predicates: `year` = CAST(2009 AS INT), `month` = CAST(5 AS INT)
@@ -4792,18 +4777,18 @@ 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.32MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.21MB mem-reservation=0B 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
 |
 12:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=10.32MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.21MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=6 row-size=50B cardinality=1.50M
 |  in pipelines: 11(GETNEXT)
 |
-F01:PLAN FRAGMENT [HASH(c_name,v.o_orderkey,v.o_orderstatus)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=138.32MB mem-reservation=34.00MB thread-reservation=1
+F01:PLAN FRAGMENT [HASH(c_name,v.o_orderkey,v.o_orderstatus)] hosts=3 instances=4
+Per-Instance Resources: mem-estimate=138.21MB mem-reservation=34.00MB thread-reservation=1
 11:AGGREGATE [FINALIZE]
 |  group by: c_name, v.o_orderkey, v.o_orderstatus
 |  mem-estimate=128.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
@@ -4811,11 +4796,11 @@ Per-Instance Resources: mem-estimate=138.32MB mem-reservation=34.00MB thread-res
 |  in pipelines: 11(GETNEXT), 00(OPEN)
 |
 10:EXCHANGE [HASH(c_name,v.o_orderkey,v.o_orderstatus)]
-|  mem-estimate=10.32MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.21MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=6 row-size=50B cardinality=1.50M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4
 Per-Instance Resources: mem-estimate=345.94MB mem-reservation=85.94MB thread-reservation=1
 09:AGGREGATE [STREAMING]
 |  group by: c_name, o1.o_orderkey, o2.o_orderstatus
@@ -5071,24 +5056,24 @@ Per-Host Resources: mem-estimate=136.00MB mem-reservation=104.00MB thread-reserv
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 Max Per-Host Resource Reservation: Memory=208.00MB Threads=3
-Per-Host Resource Estimates: Memory=284MB
+Per-Host Resource Estimates: Memory=283MB
 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=12.21MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=11.47MB 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, 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
 |
 11:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=12.21MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=11.47MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=12,7,0 row-size=366B cardinality=1.50M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4
 Per-Instance Resources: mem-estimate=136.00MB mem-reservation=104.00MB thread-reservation=1
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -5377,8 +5362,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=141.50MB Threads=17
-Per-Host Resource Estimates: Memory=456MB
+Max Per-Host Resource Reservation: Memory=135.50MB Threads=9
+Per-Host Resource Estimates: Memory=297MB
 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,
@@ -5387,72 +5372,80 @@ 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.82MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.41MB mem-reservation=0B 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
 |
 11:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=10.82MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.41MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1,2,3 row-size=195B cardinality=1.50M
 |  in pipelines: 00(GETNEXT)
 |
-F04:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=4
-Per-Instance Resources: mem-estimate=10.68MB mem-reservation=0B thread-reservation=1
-06:HASH JOIN [INNER JOIN, PARTITIONED]
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
+06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: t1.o_orderkey = t3.o_orderkey
 |  fk/pk conjuncts: t1.o_orderkey = t3.o_orderkey
-|  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=0,1,2,3 row-size=195B cardinality=1.50M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F06:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=4
-|  |  Per-Instance Resources: mem-estimate=23.77MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F06:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=79.07MB mem-reservation=69.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: t3.o_orderkey
 |  |  runtime filters: RF000[bloom] <- t3.o_orderkey
-|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  mem-estimate=68.00MB mem-reservation=68.00MB spill-buffer=2.00MB thread-reservation=0
 |  |
+|  10:EXCHANGE [BROADCAST]
+|  |  mem-estimate=10.07MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1,2,3 row-size=24B cardinality=1.50M
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F04:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=2
+|  Per-Instance Resources: mem-estimate=5.75MB mem-reservation=0B thread-reservation=1
 |  05:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=01
 |  |  hash predicates: t2.o_orderkey = t3.o_orderkey
 |  |  fk/pk conjuncts: t2.o_orderkey = t3.o_orderkey
-|  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
 |  |  tuple-ids=1,2,3 row-size=24B cardinality=1.50M
 |  |  in pipelines: 01(GETNEXT), 02(OPEN)
 |  |
-|  |--F07:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=4
-|  |  |  Per-Instance Resources: mem-estimate=15.27MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |--F07:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=23.75MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: t3.o_orderkey
 |  |  |  runtime filters: RF002[bloom] <- t3.o_orderkey
-|  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  |
 |  |  04:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  |  hash-table-id=02
 |  |  |  hash predicates: t3.o_orderkey = t4.o_orderkey
 |  |  |  fk/pk conjuncts: t3.o_orderkey = t4.o_orderkey
-|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
 |  |  |  tuple-ids=2,3 row-size=16B cardinality=1.50M
 |  |  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |  |
-|  |  |--F08:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=4
-|  |  |  |  Per-Instance Resources: mem-estimate=11.52MB mem-reservation=5.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |--F08:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=15.25MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: t4.o_orderkey
 |  |  |  |  runtime filters: RF004[bloom] <- t4.o_orderkey
-|  |  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |  |  |
 |  |  |  08:EXCHANGE [HASH(t4.o_orderkey)]
-|  |  |  |  mem-estimate=5.77MB mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=5.75MB mem-reservation=0B thread-reservation=0
 |  |  |  |  tuple-ids=3 row-size=8B cardinality=1.50M
 |  |  |  |  in pipelines: 03(GETNEXT)
 |  |  |  |
-|  |  |  F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  |  |  F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1
 |  |  |  03:SCAN HDFS [tpch_parquet.orders t4, RANDOM]
 |  |  |     HDFS partitions=1/1 files=2 size=54.21MB
@@ -5465,11 +5458,11 @@ Per-Instance Resources: mem-estimate=10.68MB mem-reservation=0B thread-reservati
 |  |  |     in pipelines: 03(GETNEXT)
 |  |  |
 |  |  07:EXCHANGE [HASH(t3.o_orderkey)]
-|  |  |  mem-estimate=5.77MB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=5.75MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=2 row-size=8B cardinality=1.50M
 |  |  |  in pipelines: 02(GETNEXT)
 |  |  |
-|  |  F02:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  |  F02:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1
 |  |  02:SCAN HDFS [tpch_parquet.orders t3, RANDOM]
@@ -5484,11 +5477,11 @@ Per-Instance Resources: mem-estimate=10.68MB mem-reservation=0B thread-reservati
 |  |     in pipelines: 02(GETNEXT)
 |  |
 |  09:EXCHANGE [HASH(t2.o_orderkey)]
-|  |  mem-estimate=5.77MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=5.75MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=8B cardinality=1.50M
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  Per-Instance Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch_parquet.orders t2, RANDOM]
@@ -5502,14 +5495,6 @@ Per-Instance Resources: mem-estimate=10.68MB mem-reservation=0B thread-reservati
 |     tuple-ids=1 row-size=8B cardinality=1.50M
 |     in pipelines: 01(GETNEXT)
 |
-10:EXCHANGE [HASH(t1.o_orderkey)]
-|  mem-estimate=10.68MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0 row-size=171B cardinality=1.50M
-|  in pipelines: 00(GETNEXT)
-|
-F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
-Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
 00:SCAN HDFS [tpch_parquet.orders t1, RANDOM]
    HDFS partitions=1/1 files=2 size=54.21MB
    runtime filters: RF000[bloom] -> t1.o_orderkey
@@ -5703,8 +5688,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=352.00KB Threads=15
-Per-Host Resource Estimates: Memory=311MB
+Max Per-Host Resource Reservation: Memory=176.00KB Threads=8
+Per-Host Resource Estimates: Memory=161MB
 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
@@ -5712,17 +5697,17 @@ 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.27MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.13MB mem-reservation=0B 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
 |
 10:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=10.27MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.13MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1,2,3 row-size=121B cardinality=156.25M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1
 06:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  join table id: 00
@@ -5730,18 +5715,18 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-rese
 |  tuple-ids=0,1,2,3 row-size=121B cardinality=156.25M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  |  Per-Instance Resources: mem-estimate=81.57MB mem-reservation=0B thread-reservation=1
+|--F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=81.55MB mem-reservation=0B thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  mem-estimate=71.53MB mem-reservation=0B thread-reservation=0
 |  |
 |  09:EXCHANGE [BROADCAST]
-|  |  mem-estimate=10.05MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=10.02MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1,2,3 row-size=12B cardinality=6.25M
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1
 |  05:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  |  join table id: 01
@@ -5749,18 +5734,18 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-rese
 |  |  tuple-ids=1,2,3 row-size=12B cardinality=6.25M
 |  |  in pipelines: 01(GETNEXT), 02(OPEN)
 |  |
-|  |--F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  |  |  Per-Instance Resources: mem-estimate=4.80MB mem-reservation=0B thread-reservation=1
+|  |--F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=4.79MB mem-reservation=0B thread-reservation=1
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  mem-estimate=2.38MB mem-reservation=0B thread-reservation=0
 |  |  |
 |  |  08:EXCHANGE [BROADCAST]
-|  |  |  mem-estimate=2.42MB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=2.40MB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=2,3 row-size=10B cardinality=250.00K
 |  |  |  in pipelines: 02(GETNEXT)
 |  |  |
-|  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+|  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1
 |  |  04:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  |  |  join table id: 02
@@ -5768,18 +5753,18 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-rese
 |  |  |  tuple-ids=2,3 row-size=10B cardinality=250.00K
 |  |  |  in pipelines: 02(GETNEXT), 03(OPEN)
 |  |  |
-|  |  |--F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  |  |  |  Per-Instance Resources: mem-estimate=180.25KB mem-reservation=0B thread-reservation=1
+|  |  |--F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=168.25KB mem-reservation=0B thread-reservation=1
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  mem-estimate=78.12KB mem-reservation=0B thread-reservation=0
 |  |  |  |
 |  |  |  07:EXCHANGE [BROADCAST]
-|  |  |  |  mem-estimate=102.12KB mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=90.12KB mem-reservation=0B thread-reservation=0
 |  |  |  |  tuple-ids=3 row-size=8B cardinality=10.00K
 |  |  |  |  in pipelines: 03(GETNEXT)
 |  |  |  |
-|  |  |  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+|  |  |  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=128.00KB thread-reservation=1
 |  |  |  03:SCAN HDFS [tpch_parquet.supplier t4, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=883.03KB
@@ -5997,9 +5982,9 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -6046,9 +6031,9 @@ Per-Host Resources: mem-estimate=806.43MB mem-reservation=74.00MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
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 274085c..f7969aa 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
@@ -57,23 +57,23 @@ 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=38.94MB Threads=7
-Per-Host Resource Estimates: Memory=98MB
+Max Per-Host Resource Reservation: Memory=21.91MB Threads=4
+Per-Host Resource Estimates: Memory=56MB
 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.65MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.33MB mem-reservation=0B 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
 |
 04:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=10.65MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.33MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1 row-size=327B cardinality=150.00K
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 Per-Instance Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -84,20 +84,20 @@ Per-Instance Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-rese
 |  tuple-ids=0,1 row-size=327B cardinality=150.00K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  |  Per-Instance Resources: mem-estimate=2.95MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=4.89MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: n_nationkey
 |  |  runtime filters: RF000[bloom] <- n_nationkey
-|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  03:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=109B cardinality=25
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1
 |  01:SCAN HDFS [tpch_parquet.nation, RANDOM]
 |     HDFS partitions=1/1 files=1 size=3.04KB
@@ -168,34 +168,34 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=74.00MB thread-reserva
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=196.00MB Threads=9
-Per-Host Resource Estimates: Memory=386MB
+Max Per-Host Resource Reservation: Memory=132.00MB Threads=4
+Per-Host Resource Estimates: Memory=410MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.lineitem LEFT
 OUTER JOIN tpch_parquet.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
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=11.20MB mem-reservation=0B 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
 |
-05:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=12.40MB mem-reservation=0B thread-reservation=0
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=11.20MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1N row-size=402B cardinality=6.00M
 |  in pipelines: 00(GETNEXT)
 |
-F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
-02:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
+02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
@@ -203,19 +203,19 @@ Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservati
 |  tuple-ids=0,1N row-size=402B cardinality=6.00M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F04:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4
-|  |  Per-Instance Resources: mem-estimate=55.51MB mem-reservation=34.00MB thread-reservation=1
+|--F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=279.29MB mem-reservation=68.00MB thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: o_orderkey
-|  |  mem-estimate=44.82MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  mem-estimate=268.94MB mem-reservation=68.00MB spill-buffer=2.00MB thread-reservation=0
 |  |
-|  04:EXCHANGE [HASH(o_orderkey)]
-|  |  mem-estimate=10.68MB mem-reservation=0B thread-reservation=0
+|  03:EXCHANGE [BROADCAST]
+|  |  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=171B cardinality=1.50M
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     HDFS partitions=1/1 files=2 size=54.21MB
@@ -227,17 +227,10 @@ Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservati
 |     tuple-ids=1 row-size=171B cardinality=1.50M
 |     in pipelines: 01(GETNEXT)
 |
-03:EXCHANGE [HASH(l_orderkey)]
-|  mem-estimate=11.38MB mem-reservation=0B thread-reservation=0
-|  tuple-ids=0 row-size=231B cardinality=6.00M
-|  in pipelines: 00(GETNEXT)
-|
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=0
@@ -310,46 +303,46 @@ 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=100.00MB Threads=9
-Per-Host Resource Estimates: Memory=202MB
+Max Per-Host Resource Reservation: Memory=59.00MB Threads=5
+Per-Host Resource Estimates: Memory=114MB
 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=11.55MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.77MB mem-reservation=0B 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
 |
 05:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=11.55MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.77MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1 row-size=388B cardinality=1.50M
 |  in pipelines: 00(GETNEXT)
 |
-F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=4
-Per-Instance Resources: mem-estimate=10.68MB mem-reservation=0B thread-reservation=1
+F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=2
+Per-Instance Resources: mem-estimate=10.34MB mem-reservation=0B thread-reservation=1
 02:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: o_custkey = c_custkey
 |  fk/pk conjuncts: o_custkey = c_custkey
-|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
 |  tuple-ids=0,1 row-size=388B cardinality=1.50M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F04:PLAN FRAGMENT [HASH(o_custkey)] hosts=1 instances=2
-|  |  Per-Instance Resources: mem-estimate=19.99MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F04:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=28.33MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: c_custkey
 |  |  runtime filters: RF000[bloom] <- c_custkey
-|  |  mem-estimate=8.56MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  mem-estimate=17.12MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |
 |  04:EXCHANGE [HASH(c_custkey)]
-|  |  mem-estimate=10.43MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=10.22MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=218B cardinality=150.00K
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=12.34MB
@@ -362,11 +355,11 @@ Per-Instance Resources: mem-estimate=10.68MB mem-reservation=0B thread-reservati
 |     in pipelines: 01(GETNEXT)
 |
 03:EXCHANGE [HASH(o_custkey)]
-|  mem-estimate=10.68MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.34MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=171B cardinality=1.50M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
@@ -439,23 +432,23 @@ 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=151.00MB Threads=7
-Per-Host Resource Estimates: Memory=232MB
+Max Per-Host Resource Reservation: Memory=110.00MB Threads=4
+Per-Host Resource Estimates: Memory=155MB
 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=11.55MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.77MB mem-reservation=0B 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
 |
 04:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=11.55MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.77MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1 row-size=388B cardinality=1.50M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -466,20 +459,20 @@ Per-Instance Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-rese
 |  tuple-ids=0,1 row-size=388B cardinality=1.50M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  |  Per-Instance Resources: mem-estimate=45.67MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=79.22MB mem-reservation=69.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: c_custkey
 |  |  runtime filters: RF000[bloom] <- c_custkey
-|  |  mem-estimate=34.23MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  mem-estimate=68.00MB mem-reservation=68.00MB spill-buffer=2.00MB thread-reservation=0
 |  |
 |  03:EXCHANGE [BROADCAST]
-|  |  mem-estimate=10.43MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=10.22MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=218B cardinality=150.00K
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
 |     HDFS partitions=1/1 files=1 size=12.34MB
@@ -542,7 +535,7 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.09MB thread-reservati
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=2
 |  01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
-|     HDFS partitions=4/4 files=4 size=11.67KB
+|     HDFS partitions=4/4 files=4 size=11.92KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
@@ -553,7 +546,7 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.09MB thread-reservati
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.93KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/24 rows=unavailable
@@ -563,8 +556,8 @@ 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=7
-Per-Host Resource Estimates: Memory=4.06GB
+Max Per-Host Resource Reservation: Memory=68.34MB Threads=6
+Per-Host Resource Estimates: Memory=2.06GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypestiny
 Analyzed query: SELECT /* +straight_join */ * FROM functional_parquet.alltypes
@@ -591,22 +584,22 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-rese
 |  tuple-ids=0,1N row-size=160B cardinality=unavailable
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  |  Per-Instance Resources: mem-estimate=2.00GB mem-reservation=34.00MB thread-reservation=1
+|--F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=2.00GB mem-reservation=68.00MB thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: alltypestiny.id
-|  |  mem-estimate=2.00GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  mem-estimate=2.00GB mem-reservation=68.00MB spill-buffer=2.00MB thread-reservation=0
 |  |
 |  03:EXCHANGE [BROADCAST]
-|  |  mem-estimate=503.92KB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=335.92KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=80B cardinality=unavailable
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=4
 |  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=1
 |  01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
-|     HDFS partitions=4/4 files=4 size=11.67KB
+|     HDFS partitions=4/4 files=4 size=11.92KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
@@ -617,7 +610,7 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-rese
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   HDFS partitions=24/24 files=24 size=201.80KB
+   HDFS partitions=24/24 files=24 size=201.93KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/24 rows=unavailable
@@ -681,8 +674,8 @@ 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=11.88MB Threads=5
-Per-Host Resource Estimates: Memory=88MB
+Max Per-Host Resource Reservation: Memory=5.94MB Threads=3
+Per-Host Resource Estimates: Memory=44MB
 Analyzed query: SELECT c_nationkey, avg(c_acctbal) FROM tpch_parquet.customer
 GROUP BY c_nationkey
 
@@ -697,7 +690,7 @@ PLAN-ROOT SINK
 |  tuple-ids=2 row-size=10B cardinality=25
 |  in pipelines: 03(GETNEXT)
 |
-F01:PLAN FRAGMENT [HASH(c_nationkey)] hosts=1 instances=2
+F01:PLAN FRAGMENT [HASH(c_nationkey)] hosts=1 instances=1
 Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
 03:AGGREGATE [FINALIZE]
 |  output: avg:merge(c_acctbal)
@@ -711,7 +704,7 @@ Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reser
 |  tuple-ids=1 row-size=10B cardinality=25
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 Per-Instance Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reservation=1
 01:AGGREGATE [STREAMING]
 |  output: avg(c_acctbal)
@@ -812,55 +805,55 @@ Per-Host Resources: mem-estimate=85.65MB mem-reservation=52.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
    tuple-ids=0 row-size=8B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=129.00MB Threads=11
-Per-Host Resource Estimates: Memory=419MB
+Max Per-Host Resource Reservation: Memory=82.00MB Threads=6
+Per-Host Resource Estimates: Memory=244MB
 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.19MB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=10.10MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: l_orderkey, o_orderstatus, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 08:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=10.19MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.10MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=2 row-size=29B cardinality=4.69M
 |  in pipelines: 07(GETNEXT)
 |
-F03:PLAN FRAGMENT [HASH(l_orderkey,o_orderstatus)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=20.19MB mem-reservation=8.50MB thread-reservation=1
+F03:PLAN FRAGMENT [HASH(l_orderkey,o_orderstatus)] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=27.10MB mem-reservation=17.00MB thread-reservation=1
 07:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |  group by: l_orderkey, o_orderstatus
 |  having: count(*) = CAST(1 AS BIGINT)
-|  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  tuple-ids=2 row-size=29B cardinality=4.69M
 |  in pipelines: 07(GETNEXT), 00(OPEN)
 |
 06:EXCHANGE [HASH(l_orderkey,o_orderstatus)]
-|  mem-estimate=10.19MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.10MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=2 row-size=29B cardinality=4.69M
 |  in pipelines: 00(GETNEXT)
 |
-F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=44.07MB mem-reservation=34.00MB thread-reservation=1
+F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=57.60MB mem-reservation=34.00MB thread-reservation=1
 03:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: l_orderkey, o_orderstatus
-|  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=47.56MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=2 row-size=29B cardinality=4.69M
 |  in pipelines: 00(GETNEXT)
 |
@@ -868,24 +861,24 @@ Per-Instance Resources: mem-estimate=44.07MB mem-reservation=34.00MB thread-rese
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
-|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
 |  tuple-ids=0,1 row-size=29B cardinality=5.76M
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F05:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4
-|  |  Per-Instance Resources: mem-estimate=19.60MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|--F05:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=28.05MB mem-reservation=18.00MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: o_orderkey
 |  |  runtime filters: RF000[bloom] <- o_orderkey
-|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |
 |  05:EXCHANGE [HASH(o_orderkey)]
-|  |  mem-estimate=10.10MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=10.05MB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=1 row-size=21B cardinality=1.50M
 |  |  in pipelines: 01(GETNEXT)
 |  |
-|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Instance Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=1
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     HDFS partitions=1/1 files=2 size=54.21MB
@@ -898,18 +891,18 @@ Per-Instance Resources: mem-estimate=44.07MB mem-reservation=34.00MB thread-rese
 |     in pipelines: 01(GETNEXT)
 |
 04:EXCHANGE [HASH(l_orderkey)]
-|  mem-estimate=10.07MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.04MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0 row-size=8B cardinality=6.00M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 Per-Instance Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=0
@@ -957,55 +950,55 @@ Per-Host Resources: mem-estimate=806.43MB mem-reservation=74.00MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=216.00MB Threads=5
-Per-Host Resource Estimates: Memory=1.61GB
+Max Per-Host Resource Reservation: Memory=108.00MB Threads=3
+Per-Host Resource Estimates: Memory=1.52GB
 Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem
 
 F02: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=10.69MB mem-reservation=0B 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
 |
 04:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=11.38MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=231B cardinality=6.00M
 |  in pipelines: 03(GETNEXT)
 |
-F01:PLAN FRAGMENT [HASH(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_shipin [...]
-Per-Instance Resources: mem-estimate=374.59MB mem-reservation=34.00MB thread-reservation=1
+F01:PLAN FRAGMENT [HASH(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_shipin [...]
+Per-Instance Resources: mem-estimate=737.12MB mem-reservation=34.00MB thread-reservation=1
 03: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 [...]
-|  mem-estimate=363.22MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=726.43MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=231B cardinality=6.00M
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
 02:EXCHANGE [HASH(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_shipinstruct [...]
-|  mem-estimate=11.38MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=10.69MB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=231B cardinality=6.00M
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=443.22MB mem-reservation=74.00MB thread-reservation=1
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=806.43MB mem-reservation=74.00MB thread-reservation=1
 01:AGGREGATE [STREAMING]
 |  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 [...]
-|  mem-estimate=363.22MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=726.43MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=1 row-size=231B cardinality=6.00M
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.99MB
+   HDFS partitions=1/1 files=3 size=194.00MB
    stored statistics:
-     table: rows=6.00M size=193.99MB
+     table: rows=6.00M size=194.00MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=0
@@ -1059,7 +1052,7 @@ Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.01MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
-   HDFS partitions=4/4 files=4 size=11.67KB
+   HDFS partitions=4/4 files=4 size=11.92KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/4 rows=unavailable
@@ -1077,18 +1070,18 @@ 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=143.99KB mem-reservation=0B thread-reservation=1
+|  Per-Instance Resources: mem-estimate=95.99KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: string_col, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=143.99KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=95.99KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=20B cardinality=unavailable
 |  in pipelines: 03(GETNEXT)
 |
-F01:PLAN FRAGMENT [HASH(string_col)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=128.14MB mem-reservation=34.00MB thread-reservation=1
+F01:PLAN FRAGMENT [HASH(string_col)] hosts=3 instances=4
+Per-Instance Resources: mem-estimate=128.09MB mem-reservation=34.00MB thread-reservation=1
 03:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |  group by: string_col
@@ -1097,11 +1090,11 @@ Per-Instance Resources: mem-estimate=128.14MB mem-reservation=34.00MB thread-res
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
 02:EXCHANGE [HASH(string_col)]
-|  mem-estimate=143.99KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=95.99KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=20B cardinality=unavailable
 |  in pipelines: 00(GETNEXT)
 |
-F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=4
 Per-Instance Resources: mem-estimate=144.00MB mem-reservation=34.01MB thread-reservation=1
 01:AGGREGATE [STREAMING]
 |  output: count(*)
@@ -1111,7 +1104,7 @@ Per-Instance Resources: mem-estimate=144.00MB mem-reservation=34.01MB thread-res
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
-   HDFS partitions=4/4 files=4 size=11.67KB
+   HDFS partitions=4/4 files=4 size=11.92KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/4 rows=unavailable
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
index 773fd07..99761af 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
@@ -116,8 +116,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> store_sales.ss_sold_date_sk, RF002 -> store_sales.ss_item_sk
    row-size=16B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=69.62MB Threads=15
-Per-Host Resource Estimates: Memory=168MB
+Max Per-Host Resource Reservation: Memory=52.62MB Threads=12
+Per-Host Resource Estimates: Memory=127MB
 PLAN-ROOT SINK
 |
 12:MERGING-EXCHANGE [UNPARTITIONED]
@@ -342,11 +342,11 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_item_sk, RF002 -> ss_promo_sk, RF004 -> ss_cdemo_sk, RF006 -> ss_sold_date_sk
    row-size=36B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=99.50MB Threads=23
-Per-Host Resource Estimates: Memory=417MB
+Max Per-Host Resource Reservation: Memory=84.56MB Threads=13
+Per-Host Resource Estimates: Memory=259MB
 PLAN-ROOT SINK
 |
-18:MERGING-EXCHANGE [UNPARTITIONED]
+17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
 |  limit: 100
 |
@@ -354,12 +354,12 @@ PLAN-ROOT SINK
 |  order by: i_item_id ASC
 |  row-size=60B cardinality=100
 |
-17:AGGREGATE [FINALIZE]
+16:AGGREGATE [FINALIZE]
 |  output: avg:merge(ss_quantity), avg:merge(ss_list_price), avg:merge(ss_coupon_amt), avg:merge(ss_sales_price)
 |  group by: i_item_id
 |  row-size=60B cardinality=8.85K
 |
-16:EXCHANGE [HASH(i_item_id)]
+15:EXCHANGE [HASH(i_item_id)]
 |
 09:AGGREGATE [STREAMING]
 |  output: avg(ss_quantity), avg(ss_list_price), avg(ss_coupon_amt), avg(ss_sales_price)
@@ -375,7 +375,7 @@ PLAN-ROOT SINK
 |  |  build expressions: i_item_sk
 |  |  runtime filters: RF000 <- i_item_sk
 |  |
-|  15:EXCHANGE [BROADCAST]
+|  14:EXCHANGE [BROADCAST]
 |  |
 |  03:SCAN HDFS [tpcds.item]
 |     HDFS partitions=1/1 files=1 size=4.82MB
@@ -390,14 +390,14 @@ PLAN-ROOT SINK
 |  |  build expressions: p_promo_sk
 |  |  runtime filters: RF002 <- p_promo_sk
 |  |
-|  14:EXCHANGE [BROADCAST]
+|  13:EXCHANGE [BROADCAST]
 |  |
 |  04:SCAN HDFS [tpcds.promotion]
 |     HDFS partitions=1/1 files=1 size=36.36KB
 |     predicates: (p_channel_email = 'N' OR p_channel_event = 'N')
 |     row-size=30B cardinality=300
 |
-06:HASH JOIN [INNER JOIN, PARTITIONED]
+06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_cdemo_sk = cd_demo_sk
 |  row-size=96B cardinality=263.34K
 |
@@ -406,15 +406,13 @@ PLAN-ROOT SINK
 |  |  build expressions: cd_demo_sk
 |  |  runtime filters: RF004 <- cd_demo_sk
 |  |
-|  13:EXCHANGE [HASH(cd_demo_sk)]
+|  12:EXCHANGE [BROADCAST]
 |  |
 |  01:SCAN HDFS [tpcds.customer_demographics]
 |     HDFS partitions=1/1 files=1 size=76.92MB
 |     predicates: cd_marital_status = 'W', cd_gender = 'F', cd_education_status = 'Primary'
 |     row-size=52B cardinality=97.40K
 |
-12:EXCHANGE [HASH(ss_cdemo_sk)]
-|
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
 |  row-size=44B cardinality=589.03K
@@ -664,11 +662,11 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> store_sales.ss_store_sk
    row-size=8B cardinality=84.40K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=92.27MB Threads=25
-Per-Host Resource Estimates: Memory=288MB
+Max Per-Host Resource Reservation: Memory=58.38MB Threads=14
+Per-Host Resource Estimates: Memory=187MB
 PLAN-ROOT SINK
 |
-21:MERGING-EXCHANGE [UNPARTITIONED]
+20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC
 |  limit: 100
 |
@@ -676,12 +674,12 @@ PLAN-ROOT SINK
 |  order by: s_store_name ASC
 |  row-size=32B cardinality=8
 |
-20:AGGREGATE [FINALIZE]
+19:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_net_profit)
 |  group by: s_store_name
 |  row-size=32B cardinality=8
 |
-19:EXCHANGE [HASH(s_store_name)]
+18:EXCHANGE [HASH(s_store_name)]
 |
 10:AGGREGATE [STREAMING]
 |  output: sum(ss_net_profit)
@@ -697,7 +695,7 @@ PLAN-ROOT SINK
 |  |  build expressions: substr(substr(ca_zip, 1, 5), 1, 2)
 |  |  runtime filters: RF000 <- substr(substr(ca_zip, 1, 5), 1, 2)
 |  |
-|  18:EXCHANGE [BROADCAST]
+|  17:EXCHANGE [BROADCAST]
 |  |
 |  07:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  |  hash predicates: substr(ca_zip, 1, 5) = substr(ca_zip, 1, 5)
@@ -708,20 +706,20 @@ PLAN-ROOT SINK
 |  |  |  build expressions: substr(ca_zip, 1, 5)
 |  |  |  runtime filters: RF004 <- substr(ca_zip, 1, 5)
 |  |  |
-|  |  16:AGGREGATE [FINALIZE]
+|  |  15:AGGREGATE [FINALIZE]
 |  |  |  output: count:merge(*)
 |  |  |  group by: substr(ca_zip, 1, 5)
 |  |  |  having: count(*) > 10
 |  |  |  row-size=20B cardinality=396
 |  |  |
-|  |  15:EXCHANGE [HASH(substr(ca_zip, 1, 5))]
+|  |  14:EXCHANGE [HASH(substr(ca_zip, 1, 5))]
 |  |  |
 |  |  05:AGGREGATE [STREAMING]
 |  |  |  output: count(*)
 |  |  |  group by: substr(ca_zip, 1, 5)
 |  |  |  row-size=20B cardinality=3.96K
 |  |  |
-|  |  04:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  04:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  |  hash predicates: customer_address.ca_address_sk = customer.c_current_addr_sk
 |  |  |  row-size=38B cardinality=51.30K
 |  |  |
@@ -730,21 +728,19 @@ PLAN-ROOT SINK
 |  |  |  |  build expressions: customer.c_current_addr_sk
 |  |  |  |  runtime filters: RF006 <- customer.c_current_addr_sk
 |  |  |  |
-|  |  |  14:EXCHANGE [HASH(customer.c_current_addr_sk)]
+|  |  |  13:EXCHANGE [BROADCAST]
 |  |  |  |
 |  |  |  03:SCAN HDFS [tpcds.customer]
 |  |  |     HDFS partitions=1/1 files=1 size=12.60MB
 |  |  |     predicates: c_preferred_cust_flag = 'Y'
 |  |  |     row-size=17B cardinality=50.00K
 |  |  |
-|  |  13:EXCHANGE [HASH(customer_address.ca_address_sk)]
-|  |  |
 |  |  02:SCAN HDFS [tpcds.customer_address]
 |  |     HDFS partitions=1/1 files=1 size=5.25MB
 |  |     runtime filters: RF006 -> customer_address.ca_address_sk
 |  |     row-size=21B cardinality=50.00K
 |  |
-|  17:EXCHANGE [HASH(substr(ca_zip, 1, 5))]
+|  16:EXCHANGE [HASH(substr(ca_zip, 1, 5))]
 |  |
 |  06:SCAN HDFS [tpcds.customer_address]
 |     HDFS partitions=1/1 files=1 size=5.25MB
@@ -974,8 +970,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_store_sk, RF004 -> ss_customer_sk, RF006 -> ss_sold_date_sk, RF008 -> ss_item_sk
    row-size=24B cardinality=86.28K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=107.27MB Threads=29
-Per-Host Resource Estimates: Memory=302MB
+Max Per-Host Resource Reservation: Memory=72.26MB Threads=21
+Per-Host Resource Estimates: Memory=203MB
 PLAN-ROOT SINK
 |
 22:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1252,11 +1248,11 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_item_sk, RF002 -> ss_store_sk, RF004 -> ss_cdemo_sk, RF006 -> ss_sold_date_sk
    row-size=36B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=99.39MB Threads=23
-Per-Host Resource Estimates: Memory=417MB
+Max Per-Host Resource Reservation: Memory=84.51MB Threads=13
+Per-Host Resource Estimates: Memory=260MB
 PLAN-ROOT SINK
 |
-18:MERGING-EXCHANGE [UNPARTITIONED]
+17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, s_state ASC
 |  limit: 100
 |
@@ -1264,12 +1260,12 @@ PLAN-ROOT SINK
 |  order by: i_item_id ASC, s_state ASC
 |  row-size=74B cardinality=100
 |
-17:AGGREGATE [FINALIZE]
+16:AGGREGATE [FINALIZE]
 |  output: avg:merge(ss_quantity), avg:merge(ss_list_price), avg:merge(ss_coupon_amt), avg:merge(ss_sales_price)
 |  group by: i_item_id, s_state
 |  row-size=74B cardinality=8.85K
 |
-16:EXCHANGE [HASH(i_item_id,s_state)]
+15:EXCHANGE [HASH(i_item_id,s_state)]
 |
 09:AGGREGATE [STREAMING]
 |  output: avg(ss_quantity), avg(ss_list_price), avg(ss_coupon_amt), avg(ss_sales_price)
@@ -1285,7 +1281,7 @@ PLAN-ROOT SINK
 |  |  build expressions: i_item_sk
 |  |  runtime filters: RF000 <- i_item_sk
 |  |
-|  15:EXCHANGE [BROADCAST]
+|  14:EXCHANGE [BROADCAST]
 |  |
 |  04:SCAN HDFS [tpcds.item]
 |     HDFS partitions=1/1 files=1 size=4.82MB
@@ -1300,14 +1296,14 @@ PLAN-ROOT SINK
 |  |  build expressions: s_store_sk
 |  |  runtime filters: RF002 <- s_store_sk
 |  |
-|  14:EXCHANGE [BROADCAST]
+|  13:EXCHANGE [BROADCAST]
 |  |
 |  03:SCAN HDFS [tpcds.store]
 |     HDFS partitions=1/1 files=1 size=3.08KB
 |     predicates: s_state IN ('WI', 'CA', 'TX', 'FL', 'WA', 'TN')
 |     row-size=18B cardinality=12
 |
-06:HASH JOIN [INNER JOIN, PARTITIONED]
+06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_cdemo_sk = cd_demo_sk
 |  row-size=96B cardinality=263.34K
 |
@@ -1316,15 +1312,13 @@ PLAN-ROOT SINK
 |  |  build expressions: cd_demo_sk
 |  |  runtime filters: RF004 <- cd_demo_sk
 |  |
-|  13:EXCHANGE [HASH(cd_demo_sk)]
+|  12:EXCHANGE [BROADCAST]
 |  |
 |  01:SCAN HDFS [tpcds.customer_demographics]
 |     HDFS partitions=1/1 files=1 size=76.92MB
 |     predicates: cd_marital_status = 'W', cd_gender = 'F', cd_education_status = 'Primary'
 |     row-size=52B cardinality=97.40K
 |
-12:EXCHANGE [HASH(ss_cdemo_sk)]
-|
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
 |  row-size=44B cardinality=589.03K
@@ -1529,8 +1523,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpcds.store_sales.ss_customer_sk, RF002 -> store_sales.ss_store_sk, RF004 -> store_sales.ss_sold_date_sk, RF006 -> store_sales.ss_hdemo_sk
    row-size=24B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=89.52MB Threads=23
-Per-Host Resource Estimates: Memory=280MB
+Max Per-Host Resource Reservation: Memory=68.26MB Threads=14
+Per-Host Resource Estimates: Memory=188MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1742,8 +1736,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> store_sales.ss_sold_date_sk, RF002 -> store_sales.ss_item_sk
    row-size=16B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=69.62MB Threads=13
-Per-Host Resource Estimates: Memory=166MB
+Max Per-Host Resource Reservation: Memory=51.62MB Threads=9
+Per-Host Resource Estimates: Memory=124MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1929,8 +1923,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_sold_date_sk
    row-size=12B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=53.64MB Threads=13
-Per-Host Resource Estimates: Memory=166MB
+Max Per-Host Resource Reservation: Memory=43.63MB Threads=9
+Per-Host Resource Estimates: Memory=124MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2221,8 +2215,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> tpcds.store_sales.ss_customer_sk, RF004 -> store_sales.ss_addr_sk, RF006 -> store_sales.ss_store_sk, RF008 -> store_sales.ss_sold_date_sk, RF010 -> store_sales.ss_hdemo_sk
    row-size=36B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=145.02MB Threads=31
-Per-Host Resource Estimates: Memory=377MB
+Max Per-Host Resource Reservation: Memory=107.76MB Threads=20
+Per-Host Resource Estimates: Memory=257MB
 PLAN-ROOT SINK
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2464,8 +2458,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> store_sales.ss_sold_date_sk, RF002 -> store_sales.ss_item_sk
    row-size=16B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=69.62MB Threads=13
-Per-Host Resource Estimates: Memory=167MB
+Max Per-Host Resource Reservation: Memory=51.62MB Threads=9
+Per-Host Resource Estimates: Memory=125MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2668,11 +2662,11 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_sold_date_sk, RF004 -> ss_item_sk
    row-size=20B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=76.52MB Threads=19
-Per-Host Resource Estimates: Memory=205MB
+Max Per-Host Resource Reservation: Memory=57.51MB Threads=11
+Per-Host Resource Estimates: Memory=146MB
 PLAN-ROOT SINK
 |
-15:MERGING-EXCHANGE [UNPARTITIONED]
+14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales ASC, i_manufact_id ASC
 |  limit: 100
 |
@@ -2680,12 +2674,12 @@ PLAN-ROOT SINK
 |  order by: sum_sales ASC, i_manufact_id ASC
 |  row-size=20B cardinality=96
 |
-14:AGGREGATE [FINALIZE]
+13:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_sales_price)
 |  group by: i_manufact_id, d_qoy
 |  row-size=24B cardinality=96
 |
-13:EXCHANGE [HASH(i_manufact_id,d_qoy)]
+12:EXCHANGE [HASH(i_manufact_id,d_qoy)]
 |
 07:AGGREGATE [STREAMING]
 |  output: sum(ss_sales_price)
@@ -2701,13 +2695,13 @@ PLAN-ROOT SINK
 |  |  build expressions: s_store_sk
 |  |  runtime filters: RF000 <- s_store_sk
 |  |
-|  12:EXCHANGE [BROADCAST]
+|  11:EXCHANGE [BROADCAST]
 |  |
 |  03:SCAN HDFS [tpcds.store]
 |     HDFS partitions=1/1 files=1 size=3.08KB
 |     row-size=4B cardinality=12
 |
-05:HASH JOIN [INNER JOIN, PARTITIONED]
+05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
 |  row-size=110B cardinality=96
 |
@@ -2716,15 +2710,13 @@ PLAN-ROOT SINK
 |  |  build expressions: d_date_sk
 |  |  runtime filters: RF002 <- d_date_sk
 |  |
-|  11:EXCHANGE [HASH(d_date_sk)]
+|  10:EXCHANGE [BROADCAST]
 |  |
 |  02:SCAN HDFS [tpcds.date_dim]
 |     HDFS partitions=1/1 files=1 size=9.84MB
 |     predicates: d_month_seq IN (1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222, 1223)
 |     row-size=12B cardinality=362
 |
-10:EXCHANGE [HASH(ss_sold_date_sk)]
-|
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
 |  row-size=98B cardinality=481
@@ -2860,8 +2852,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_sold_date_sk, RF002 -> ss_item_sk
    row-size=16B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=69.62MB Threads=13
-Per-Host Resource Estimates: Memory=167MB
+Max Per-Host Resource Reservation: Memory=51.62MB Threads=9
+Per-Host Resource Estimates: Memory=124MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3224,8 +3216,8 @@ PLAN-ROOT SINK
    runtime filters: RF004 -> tpcds.store_sales.ss_store_sk, RF008 -> ss_sold_date_sk
    row-size=12B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=197.53MB Threads=37
-Per-Host Resource Estimates: Memory=494MB
+Max Per-Host Resource Reservation: Memory=159.52MB Threads=25
+Per-Host Resource Estimates: Memory=356MB
 PLAN-ROOT SINK
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3555,11 +3547,11 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_sold_date_sk, RF004 -> ss_item_sk
    row-size=20B cardinality=546.31K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=81.52MB Threads=21
-Per-Host Resource Estimates: Memory=225MB
+Max Per-Host Resource Reservation: Memory=62.51MB Threads=11
+Per-Host Resource Estimates: Memory=166MB
 PLAN-ROOT SINK
 |
-19:MERGING-EXCHANGE [UNPARTITIONED]
+17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_manager_id ASC, avg_monthly_sales ASC, sum_sales ASC
 |  limit: 100
 |
@@ -3580,19 +3572,19 @@ PLAN-ROOT SINK
 |  order by: i_manager_id ASC NULLS FIRST
 |  row-size=24B cardinality=6
 |
-18:AGGREGATE [FINALIZE]
+16:AGGREGATE [FINALIZE]
 |  output: sum:merge(ss_sales_price)
 |  group by: i_manager_id, d_moy
 |  row-size=24B cardinality=6
 |
-17:EXCHANGE [HASH(i_manager_id)]
+15:EXCHANGE [HASH(i_manager_id)]
 |
 07:AGGREGATE [STREAMING]
 |  output: sum(ss_sales_price)
 |  group by: i_manager_id, d_moy
 |  row-size=24B cardinality=6
 |
-06:HASH JOIN [INNER JOIN, PARTITIONED]
+06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
 |  row-size=114B cardinality=6
 |
@@ -3601,15 +3593,13 @@ PLAN-ROOT SINK
 |  |  build expressions: s_store_sk
 |  |  runtime filters: RF000 <- s_store_sk
 |  |
-|  16:EXCHANGE [HASH(s_store_sk)]
+|  14:EXCHANGE [BROADCAST]
 |  |
 |  03:SCAN HDFS [tpcds.store]
 |     HDFS partitions=1/1 files=1 size=3.08KB
 |     row-size=4B cardinality=12
 |
-15:EXCHANGE [HASH(ss_store_sk)]
-|
-05:HASH JOIN [INNER JOIN, PARTITIONED]
+05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
 |  row-size=110B cardinality=6
 |
@@ -3618,15 +3608,13 @@ PLAN-ROOT SINK
 |  |  build expressions: d_date_sk
 |  |  runtime filters: RF002 <- d_date_sk
 |  |
-|  14:EXCHANGE [HASH(d_date_sk)]
+|  13:EXCHANGE [BROADCAST]
 |  |
 |  02:SCAN HDFS [tpcds.date_dim]
 |     HDFS partitions=1/1 files=1 size=9.84MB
 |     predicates: tpcds.date_dim.d_date_sk <= 2452275, tpcds.date_dim.d_date_sk >= 2451911, d_month_seq IN (1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1220, 1221, 1222, 1223)
 |     row-size=12B cardinality=114
 |
-13:EXCHANGE [HASH(ss_sold_date_sk)]
-|
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
 |  row-size=98B cardinality=92
@@ -3898,8 +3886,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpcds.store_sales.ss_store_sk, RF002 -> tpcds.store_sales.ss_item_sk, RF004 -> tpcds.store_sales.ss_store_sk, RF006 -> ss_sold_date_sk
    row-size=20B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=151.02MB Threads=31
-Per-Host Resource Estimates: Memory=394MB
+Max Per-Host Resource Reservation: Memory=123.01MB Threads=23
+Per-Host Resource Estimates: Memory=310MB
 PLAN-ROOT SINK
 |
 27:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4260,8 +4248,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> tpcds.store_sales.ss_customer_sk, RF004 -> store_sales.ss_addr_sk, RF006 -> store_sales.ss_store_sk, RF008 -> store_sales.ss_sold_date_sk, RF010 -> store_sales.ss_hdemo_sk
    row-size=40B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=206.64MB Threads=29
-Per-Host Resource Estimates: Memory=417MB
+Max Per-Host Resource Reservation: Memory=168.38MB Threads=17
+Per-Host Resource Estimates: Memory=292MB
 PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4559,8 +4547,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpcds.store_sales.ss_customer_sk, RF002 -> store_sales.ss_store_sk, RF004 -> store_sales.ss_sold_date_sk, RF006 -> store_sales.ss_hdemo_sk
    row-size=24B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=89.52MB Threads=23
-Per-Host Resource Estimates: Memory=282MB
+Max Per-Host Resource Reservation: Memory=68.26MB Threads=14
+Per-Host Resource Estimates: Memory=190MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4833,8 +4821,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpcds.store_sales.ss_customer_sk, RF002 -> store_sales.ss_sold_date_sk, RF004 -> store_sales.ss_store_sk, RF006 -> store_sales.ss_hdemo_sk
    row-size=36B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=85.77MB Threads=23
-Per-Host Resource Estimates: Memory=275MB
+Max Per-Host Resource Reservation: Memory=64.51MB Threads=14
+Per-Host Resource Estimates: Memory=185MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -5091,8 +5079,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_sold_date_sk, RF004 -> ss_item_sk
    row-size=20B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=108.52MB Threads=17
-Per-Host Resource Estimates: Memory=237MB
+Max Per-Host Resource Reservation: Memory=89.51MB Threads=11
+Per-Host Resource Estimates: Memory=178MB
 PLAN-ROOT SINK
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -5294,8 +5282,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss.ss_store_sk, RF002 -> ss.ss_hdemo_sk, RF004 -> ss.ss_sold_time_sk
    row-size=12B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=53.14MB Threads=15
-Per-Host Resource Estimates: Memory=179MB
+Max Per-Host Resource Reservation: Memory=41.88MB Threads=9
+Per-Host Resource Estimates: Memory=128MB
 PLAN-ROOT SINK
 |
 12:AGGREGATE [FINALIZE]
@@ -5507,8 +5495,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_sold_date_sk, RF002 -> ss_item_sk
    row-size=16B cardinality=29.76K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=113.75MB Threads=15
-Per-Host Resource Estimates: Memory=202MB
+Max Per-Host Resource Reservation: Memory=60.31MB Threads=9
+Per-Host Resource Estimates: Memory=120MB
 PLAN-ROOT SINK
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
@@ -5822,8 +5810,8 @@ PLAN-ROOT SINK
    runtime filters: RF004 -> s.ss_item_sk, RF006 -> s.ss_sold_date_sk, RF010 -> s.ss_customer_sk
    row-size=16B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=176.88MB Threads=34
-Per-Host Resource Estimates: Memory=439MB
+Max Per-Host Resource Reservation: Memory=115.00MB Threads=20
+Per-Host Resource Estimates: Memory=270MB
 PLAN-ROOT SINK
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
@@ -6425,8 +6413,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_item_sk, RF004 -> ss_sold_date_sk
    row-size=20B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=755.80MB Threads=55
-Per-Host Resource Estimates: Memory=1.07GB
+Max Per-Host Resource Reservation: Memory=698.77MB Threads=36
+Per-Host Resource Estimates: Memory=915MB
 PLAN-ROOT SINK
 |
 54:MERGING-EXCHANGE [UNPARTITIONED]
@@ -7052,8 +7040,8 @@ PLAN-ROOT SINK
    runtime filters: RF010 -> ss_sold_date_sk, RF008 -> ss_item_sk, RF006 -> ss_promo_sk, RF002 -> ss_customer_sk, RF004 -> ss_store_sk
    row-size=28B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=238.78MB Threads=55
-Per-Host Resource Estimates: Memory=627MB
+Max Per-Host Resource Reservation: Memory=166.70MB Threads=35
+Per-Host Resource Estimates: Memory=411MB
 PLAN-ROOT SINK
 |
 27:SORT
@@ -8126,8 +8114,8 @@ PLAN-ROOT SINK
    runtime filters: RF004 -> ss_sold_time_sk, RF002 -> ss_hdemo_sk
    row-size=12B cardinality=2.88M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=383.12MB Threads=127
-Per-Host Resource Estimates: Memory=1.36GB
+Max Per-Host Resource Reservation: Memory=307.06MB Threads=79
+Per-Host Resource Estimates: Memory=994MB
 PLAN-ROOT SINK
 |
 70:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
index f4313b0..c72982b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
@@ -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=122.81MB Threads=39
-Per-Host Resource Estimates: Memory=717MB
+Max Per-Host Resource Reservation: Memory=82.84MB Threads=20
+Per-Host Resource Estimates: Memory=376MB
 PLAN-ROOT SINK
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
@@ -648,11 +648,11 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> l_orderkey
    row-size=46B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=71.50MB Threads=17
-Per-Host Resource Estimates: Memory=503MB
+Max Per-Host Resource Reservation: Memory=64.75MB Threads=9
+Per-Host Resource Estimates: Memory=369MB
 PLAN-ROOT SINK
 |
-13:MERGING-EXCHANGE [UNPARTITIONED]
+11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC, o_orderdate ASC
 |  limit: 10
 |
@@ -660,19 +660,19 @@ PLAN-ROOT SINK
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC, o_orderdate ASC
 |  row-size=50B cardinality=10
 |
-12:AGGREGATE [FINALIZE]
+10:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice * (1 - l_discount))
 |  group by: l_orderkey, o_orderdate, o_shippriority
 |  row-size=50B cardinality=17.56K
 |
-11:EXCHANGE [HASH(l_orderkey,o_orderdate,o_shippriority)]
+09:EXCHANGE [HASH(l_orderkey,o_orderdate,o_shippriority)]
 |
 05:AGGREGATE [STREAMING]
 |  output: sum(l_extendedprice * (1 - l_discount))
 |  group by: l_orderkey, o_orderdate, o_shippriority
 |  row-size=50B cardinality=17.56K
 |
-04:HASH JOIN [INNER JOIN, PARTITIONED]
+04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: o_custkey = c_custkey
 |  row-size=117B cardinality=17.56K
 |
@@ -681,16 +681,14 @@ PLAN-ROOT SINK
 |  |  build expressions: c_custkey
 |  |  runtime filters: RF000 <- c_custkey
 |  |
-|  10:EXCHANGE [HASH(c_custkey)]
+|  08:EXCHANGE [BROADCAST]
 |  |
 |  00:SCAN HDFS [tpch.customer]
 |     HDFS partitions=1/1 files=1 size=23.08MB
 |     predicates: c_mktsegment = 'BUILDING'
 |     row-size=29B cardinality=30.00K
 |
-09:EXCHANGE [HASH(o_custkey)]
-|
-03:HASH JOIN [INNER JOIN, PARTITIONED]
+03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_orderkey = o_orderkey
 |  row-size=88B cardinality=57.58K
 |
@@ -699,7 +697,7 @@ PLAN-ROOT SINK
 |  |  build expressions: o_orderkey
 |  |  runtime filters: RF002 <- o_orderkey
 |  |
-|  08:EXCHANGE [HASH(o_orderkey)]
+|  07:EXCHANGE [BROADCAST]
 |  |
 |  01:SCAN HDFS [tpch.orders]
 |     HDFS partitions=1/1 files=1 size=162.56MB
@@ -707,8 +705,6 @@ PLAN-ROOT SINK
 |     runtime filters: RF000 -> o_custkey
 |     row-size=42B cardinality=150.00K
 |
-07:EXCHANGE [HASH(l_orderkey)]
-|
 02:SCAN HDFS [tpch.lineitem]
    HDFS partitions=1/1 files=1 size=718.94MB
    predicates: l_shipdate > '1995-03-15'
@@ -811,8 +807,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> l_orderkey
    row-size=52B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=72.62MB Threads=11
-Per-Host Resource Estimates: Memory=453MB
+Max Per-Host Resource Reservation: Memory=46.69MB Threads=7
+Per-Host Resource Estimates: Memory=317MB
 PLAN-ROOT SINK
 |
 09:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1043,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=137.28MB Threads=25
-Per-Host Resource Estimates: Memory=644MB
+Max Per-Host Resource Reservation: Memory=113.27MB Threads=15
+Per-Host Resource Estimates: Memory=462MB
 PLAN-ROOT SINK
 |
 20:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1414,8 +1410,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=139.78MB Threads=27
-Per-Host Resource Estimates: Memory=673MB
+Max Per-Host Resource Reservation: Memory=117.77MB Threads=18
+Per-Host Resource Estimates: Memory=499MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1770,8 +1766,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=136.05MB Threads=39
-Per-Host Resource Estimates: Memory=724MB
+Max Per-Host Resource Reservation: Memory=106.02MB Threads=28
+Per-Host Resource Estimates: Memory=504MB
 PLAN-ROOT SINK
 |
 29:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2104,23 +2100,23 @@ 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=165.89MB Threads=29
-Per-Host Resource Estimates: Memory=867MB
+Max Per-Host Resource Reservation: Memory=193.38MB Threads=18
+Per-Host Resource Estimates: Memory=647MB
 PLAN-ROOT SINK
 |
-22:MERGING-EXCHANGE [UNPARTITIONED]
+21:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: nation ASC, o_year DESC
 |
 12:SORT
 |  order by: nation ASC, o_year DESC
 |  row-size=39B cardinality=61.70K
 |
-21:AGGREGATE [FINALIZE]
+20:AGGREGATE [FINALIZE]
 |  output: sum:merge(amount)
 |  group by: nation, o_year
 |  row-size=39B cardinality=61.70K
 |
-20:EXCHANGE [HASH(nation,o_year)]
+19:EXCHANGE [HASH(nation,o_year)]
 |
 11:AGGREGATE [STREAMING]
 |  output: sum(l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity)
@@ -2136,13 +2132,13 @@ PLAN-ROOT SINK
 |  |  build expressions: n_nationkey
 |  |  runtime filters: RF000 <- n_nationkey
 |  |
-|  19:EXCHANGE [BROADCAST]
+|  18:EXCHANGE [BROADCAST]
 |  |
 |  05:SCAN HDFS [tpch.nation]
 |     HDFS partitions=1/1 files=1 size=2.15KB
 |     row-size=21B cardinality=25
 |
-09:HASH JOIN [INNER JOIN, PARTITIONED]
+09:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_partkey = ps_partkey, l_suppkey = ps_suppkey
 |  row-size=165B cardinality=574.29K
 |
@@ -2151,14 +2147,12 @@ PLAN-ROOT SINK
 |  |  build expressions: ps_partkey, ps_suppkey
 |  |  runtime filters: RF002 <- ps_partkey, RF003 <- ps_suppkey
 |  |
-|  18:EXCHANGE [HASH(ps_partkey,ps_suppkey)]
+|  17:EXCHANGE [BROADCAST]
 |  |
 |  03:SCAN HDFS [tpch.partsupp]
 |     HDFS partitions=1/1 files=1 size=112.71MB
 |     row-size=24B cardinality=800.00K
 |
-17:EXCHANGE [HASH(l_partkey,l_suppkey)]
-|
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_suppkey = s_suppkey
 |  row-size=141B cardinality=574.29K
@@ -2366,8 +2360,8 @@ PLAN-ROOT SINK
    runtime filters: RF004 -> l_orderkey
    row-size=37B cardinality=2.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=162.89MB Threads=19
-Per-Host Resource Estimates: Memory=628MB
+Max Per-Host Resource Reservation: Memory=144.88MB Threads=14
+Per-Host Resource Estimates: Memory=484MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2650,8 +2644,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> ps_suppkey
    row-size=28B cardinality=800.00K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=87.41MB Threads=26
-Per-Host Resource Estimates: Memory=591MB
+Max Per-Host Resource Reservation: Memory=53.45MB Threads=14
+Per-Host Resource Estimates: Memory=311MB
 PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2868,8 +2862,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> l_orderkey
    row-size=90B cardinality=320.78K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=100.88MB Threads=11
-Per-Host Resource Estimates: Memory=493MB
+Max Per-Host Resource Reservation: Memory=75.94MB Threads=7
+Per-Host Resource Estimates: Memory=352MB
 PLAN-ROOT SINK
 |
 09:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3017,8 +3011,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> o_custkey
    row-size=77B cardinality=150.00K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=68.38MB Threads=11
-Per-Host Resource Estimates: Memory=335MB
+Max Per-Host Resource Reservation: Memory=41.38MB Threads=6
+Per-Host Resource Estimates: Memory=170MB
 PLAN-ROOT SINK
 |
 10:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3140,8 +3134,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> l_partkey
    row-size=46B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=40.75MB Threads=9
-Per-Host Resource Estimates: Memory=313MB
+Max Per-Host Resource Reservation: Memory=32.75MB Threads=8
+Per-Host Resource Estimates: Memory=280MB
 PLAN-ROOT SINK
 |
 07:AGGREGATE [FINALIZE]
@@ -3329,8 +3323,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpch.lineitem.l_suppkey
    row-size=46B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=86.50MB Threads=16
-Per-Host Resource Estimates: Memory=531MB
+Max Per-Host Resource Reservation: Memory=67.62MB Threads=12
+Per-Host Resource Estimates: Memory=489MB
 PLAN-ROOT SINK
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3545,8 +3539,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ps_partkey
    row-size=16B cardinality=800.00K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=95.50MB Threads=15
-Per-Host Resource Estimates: Memory=396MB
+Max Per-Host Resource Reservation: Memory=50.19MB Threads=8
+Per-Host Resource Estimates: Memory=201MB
 PLAN-ROOT SINK
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3729,8 +3723,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpch.lineitem.l_partkey
    row-size=16B cardinality=6.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=75.62MB Threads=13
-Per-Host Resource Estimates: Memory=502MB
+Max Per-Host Resource Reservation: Memory=66.62MB Threads=11
+Per-Host Resource Estimates: Memory=469MB
 PLAN-ROOT SINK
 |
 12:AGGREGATE [FINALIZE]
@@ -3958,8 +3952,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF004 -> l_orderkey
    row-size=16B cardinality=6.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=298.88MB Threads=19
-Per-Host Resource Estimates: Memory=924MB
+Max Per-Host Resource Reservation: Memory=281.88MB Threads=16
+Per-Host Resource Estimates: Memory=797MB
 PLAN-ROOT SINK
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4142,8 +4136,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> l_partkey
    row-size=72B cardinality=801.95K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=38.88MB Threads=7
-Per-Host Resource Estimates: Memory=279MB
+Max Per-Host Resource Reservation: Memory=29.88MB Threads=5
+Per-Host Resource Estimates: Memory=245MB
 PLAN-ROOT SINK
 |
 06:AGGREGATE [FINALIZE]
@@ -4359,8 +4353,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpch.lineitem.l_suppkey, RF002 -> tpch.lineitem.l_partkey, RF003 -> tpch.lineitem.l_suppkey
    row-size=46B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=161.39MB Threads=23
-Per-Host Resource Estimates: Memory=613MB
+Max Per-Host Resource Reservation: Memory=123.45MB Threads=13
+Per-Host Resource Estimates: Memory=430MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4661,8 +4655,8 @@ PLAN-ROOT SINK
    predicates: l3.l_receiptdate > l3.l_commitdate
    row-size=60B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=109.14MB Threads=27
-Per-Host Resource Estimates: Memory=915MB
+Max Per-Host Resource Reservation: Memory=97.13MB Threads=22
+Per-Host Resource Estimates: Memory=792MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4911,8 +4905,8 @@ PLAN-ROOT SINK
    HDFS partitions=1/1 files=1 size=162.56MB
    row-size=8B cardinality=1.50M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=83.75MB Threads=16
-Per-Host Resource Estimates: Memory=416MB
+Max Per-Host Resource Reservation: Memory=41.88MB Threads=9
+Per-Host Resource Estimates: Memory=213MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-broadcast-joins.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-broadcast-joins.test
new file mode 100644
index 0000000..905c1fd
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-broadcast-joins.test
@@ -0,0 +1,127 @@
+====
+---- QUERY
+# Spilling broadcast join between lineitem and orders.
+# Simplified from TPC-Q21.
+set buffer_pool_limit=50m;
+select straight_join s_name, count(*)
+from lineitem join /*+broadcast*/ orders on o_orderkey = l_orderkey
+    join supplier on s_suppkey = l_suppkey
+where o_orderstatus = 'F'
+group by s_name
+order by count(*) desc, s_name
+limit 20
+---- RESULTS
+'Supplier#000002293',350
+'Supplier#000006337',349
+'Supplier#000008446',349
+'Supplier#000009636',349
+'Supplier#000006566',347
+'Supplier#000000219',346
+'Supplier#000000870',346
+'Supplier#000001689',346
+'Supplier#000002493',345
+'Supplier#000000208',344
+'Supplier#000008565',343
+'Supplier#000001941',342
+'Supplier#000006467',341
+'Supplier#000007489',341
+'Supplier#000007079',340
+'Supplier#000000326',339
+'Supplier#000003041',339
+'Supplier#000002196',338
+'Supplier#000005229',338
+'Supplier#000006159',338
+---- TYPES
+STRING,BIGINT
+---- RUNTIME_PROFILE
+# Verify that at least one of the joins was spilled.
+row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
+====
+---- QUERY
+# Spilling broadcast join that will repartition.
+# Force a bushy plan where join of customer and orders feeds into the broadcast join.
+set buffer_pool_limit=58m;
+select straight_join o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate,
+    o_orderpriority, o_clerk, o_shippriority, c_name, c_nationkey, c_address, c_comment
+from lineitem join /*+broadcast*/
+  (select *
+   from orders
+      join customer on o_custkey = c_custkey) v on o_orderkey = l_orderkey
+group by 1,2,3,4,5,6,7,8,9,10,11,12
+order by count(*) desc, o_orderkey
+limit 10
+---- RESULTS
+7,39136,'O',252004.18,'1996-01-10','2-HIGH','Clerk#000000470',0,'Customer#000039136',5,'afZJC1mWpwvsfKT0211ZD6NQXVGETfl','y? express theodolites haggle against the bold instructions. slyly regular accoun'
+68,28547,'O',330793.52,'1998-04-18','3-MEDIUM','Clerk#000000440',0,'Customer#000028547',1,'AeWmD3BLrsSkmRY7O,wbB75i6Ll','y regular foxes nag quickly after the express, regular deposits. regular, bold requests nod furi'
+129,71134,'F',261013.14,'1992-11-19','5-LOW','Clerk#000000859',0,'Customer#000071134',8,'QrIUhKsAaaU8pLxHNHpG mN1F n0eWQxE2',' ideas. furiously pending packages dazzle blithely. special platelets nod f'
+164,779,'F',301925.76,'1992-10-21','5-LOW','Clerk#000000209',0,'Customer#000000779',5,'2cTZiS4ulZ74edT,RmDnh4ZaCrphMMh Ff2','old dependencies. pains haggle fluffily carefull'
+194,61724,'F',176707.84,'1992-04-05','3-MEDIUM','Clerk#000000352',0,'Customer#000061724',8,'konflahJ4RK4riiSPR3 duNNLr5n Es7gF','regular accounts. carefully ironic requests '
+225,33031,'P',220441.09,'1995-05-25','1-URGENT','Clerk#000000177',0,'Customer#000033031',8,'YFlDbKd1jeGbXz WgshqgAslqv,T4Rfe','e quickly. slyly special pinto beans nag quickly? carefully ironic de'
+226,127466,'F',313134.62,'1993-03-10','2-HIGH','Clerk#000000756',0,'Customer#000127466',17,'0,cSa,GNNBHU','tes haggle furiously during the unusual, even'
+322,133546,'F',205623.50,'1992-03-19','1-URGENT','Clerk#000000158',0,'Customer#000133546',5,'J0vOJt9OkiZ0R3kE,RtYvv','. final gifts are slyly pending packages. even instructio'
+326,75986,'O',327413.14,'1995-06-04','2-HIGH','Clerk#000000466',0,'Customer#000075986',21,'h3vC,A ibqYy2VagkCNfRiQ','y after the carefully bold instructions. special, regular instructions cajo'
+354,138268,'O',217160.72,'1996-03-14','2-HIGH','Clerk#000000511',0,'Customer#000138268',9,'HCMHttdHqpeDYf','inal, express deposits against the slowly'
+---- TYPES
+BIGINT, BIGINT, STRING, DECIMAL, STRING, STRING, STRING, INT, STRING, SMALLINT, STRING, STRING
+---- RUNTIME_PROFILE
+# Verify that at least one of the joins was spilled and repartitioned.
+row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
+row_regex: .*NumRepartitions: .* \([1-9][0-9]*\)
+====
+---- QUERY
+# Spilling broadcast join with empty probe-side partitions.
+set buffer_pool_limit=50m;
+select straight_join count(*)
+from
+lineitem a join /*+broadcast*/ lineitem b on a.l_orderkey = b.l_orderkey
+where
+a.l_partkey = 1 and b.l_orderkey < 1000000;
+---- RESULTS
+19
+---- TYPES
+BIGINT
+---- RUNTIME_PROFILE
+# Verify that at least one of the joins was spilled and that same hash table builds
+# were *not* skipped - that optimization is disabled for shared broadcast joins.
+row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
+row_regex: .*NumHashTableBuildsSkipped: 0 
+====
+---- QUERY
+# Spilling broadcast join that has some empty build partitions.
+# This needs to be calibrated so that the join has enough memory to
+# fit all of the duplicate values for a given key in memory.
+set buffer_pool_limit=60m;
+select straight_join s_name, o_clerk, count(*)
+from lineitem
+  join /*+broadcast*/ orders on (o_orderkey % 8) = l_orderkey
+  join supplier on s_suppkey = l_suppkey
+where o_orderstatus = 'F'
+group by s_name, o_clerk
+order by count(*) desc, s_name, o_clerk limit 20;
+---- RESULTS
+'Supplier#000000035','Clerk#000000337',126
+'Supplier#000001759','Clerk#000000225',126
+'Supplier#000002269','Clerk#000000225',126
+'Supplier#000003074','Clerk#000000225',126
+'Supplier#000003928','Clerk#000000337',126
+'Supplier#000007758','Clerk#000000225',126
+'Supplier#000008571','Clerk#000000337',126
+'Supplier#000009440','Clerk#000000225',126
+'Supplier#000009607','Clerk#000000225',126
+'Supplier#000009799','Clerk#000000225',126
+'Supplier#000000638','Clerk#000000461',125
+'Supplier#000000650','Clerk#000000718',125
+'Supplier#000001534','Clerk#000000461',125
+'Supplier#000001798','Clerk#000000718',125
+'Supplier#000001883','Clerk#000000718',125
+'Supplier#000003474','Clerk#000000718',125
+'Supplier#000003701','Clerk#000000461',125
+'Supplier#000004633','Clerk#000000461',125
+'Supplier#000006540','Clerk#000000718',125
+'Supplier#000007311','Clerk#000000461',125
+---- TYPES
+STRING, STRING, BIGINT
+---- RUNTIME_PROFILE
+# Verify that at least one of the joins was spilled.
+row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
+====
diff --git a/tests/common/test_dimensions.py b/tests/common/test_dimensions.py
index c4e31e9..c352c83 100644
--- a/tests/common/test_dimensions.py
+++ b/tests/common/test_dimensions.py
@@ -115,6 +115,12 @@ def create_avro_snappy_dimension(workload):
       TableFormatInfo.create_from_string(dataset, 'avro/snap/block'))
 
 
+def create_kudu_dimension(workload):
+  dataset = get_dataset_from_workload(workload)
+  return ImpalaTestDimension('table_format',
+      TableFormatInfo.create_from_string(dataset, 'kudu/none'))
+
+
 def create_client_protocol_dimension():
   # IMPALA-8864: Older python versions do not support SSLContext object that the thrift
   # http client implementation depends on. Falls back to a dimension without http
diff --git a/tests/failure/test_failpoints.py b/tests/failure/test_failpoints.py
index 7d6978c..c3711aa 100644
--- a/tests/failure/test_failpoints.py
+++ b/tests/failure/test_failpoints.py
@@ -52,7 +52,11 @@ QUERIES = [
   "select 1 from alltypessmall order by id limit 100",
   "select * from alltypessmall union all select * from alltypessmall",
   "select row_number() over (partition by int_col order by id) from alltypessmall",
-  "select c from (select id c from alltypessmall order by id limit 10) v where c = 1"
+  "select c from (select id c from alltypessmall order by id limit 10) v where c = 1",
+  """SELECT STRAIGHT_JOIN *
+           FROM alltypes t1
+                  JOIN /*+broadcast*/ alltypesagg t2 ON t1.id = t2.id
+           WHERE t2.int_col < 1000"""
 ]
 
 @SkipIf.skip_hbase # -skip_hbase argument specified
diff --git a/tests/query_test/test_cancellation.py b/tests/query_test/test_cancellation.py
index 8770733..b12725c 100644
--- a/tests/query_test/test_cancellation.py
+++ b/tests/query_test/test_cancellation.py
@@ -34,11 +34,19 @@ LINEITEM_PK = 'l_orderkey, l_partkey, l_suppkey, l_linenumber'
 # Queries to execute, mapped to a unique PRIMARY KEY for use in CTAS with Kudu. If None
 # is specified for the PRIMARY KEY, it will not be used in a CTAS statement on Kudu.
 # Use the TPC-H dataset because tables are large so queries take some time to execute.
-QUERIES = {'select l_returnflag from lineitem' : None,
-           'select count(l_returnflag) pk from lineitem' : 'pk',
-           'select * from lineitem limit 50' : LINEITEM_PK,
-           'compute stats lineitem' : None,
-           'select * from lineitem order by l_orderkey' : LINEITEM_PK}
+QUERIES = {'select l_returnflag from lineitem': None,
+           'select count(l_returnflag) pk from lineitem': 'pk',
+           'select * from lineitem limit 50': LINEITEM_PK,
+           'compute stats lineitem': None,
+           'select * from lineitem order by l_orderkey': LINEITEM_PK,
+           '''SELECT STRAIGHT_JOIN *
+           FROM lineitem
+                  JOIN /*+broadcast*/ orders ON o_orderkey = l_orderkey
+                  JOIN supplier ON s_suppkey = l_suppkey
+           WHERE o_orderstatus = 'F'
+           ORDER BY l_orderkey
+           LIMIT 10000''': LINEITEM_PK
+           }
 
 QUERY_TYPE = ["SELECT", "CTAS"]
 
diff --git a/tests/query_test/test_spilling.py b/tests/query_test/test_spilling.py
index 48daf49..035d79d 100644
--- a/tests/query_test/test_spilling.py
+++ b/tests/query_test/test_spilling.py
@@ -22,7 +22,7 @@ from tests.common.environ import ImpalaTestClusterProperties
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfNotHdfsMinicluster
 from tests.common.test_dimensions import (create_exec_option_dimension_from_dict,
-    create_parquet_dimension)
+    create_kudu_dimension, create_parquet_dimension)
 
 IMPALA_TEST_CLUSTER_PROPERTIES = ImpalaTestClusterProperties.get_instance()
 
@@ -133,3 +133,29 @@ class TestSpillingNoDebugActionDimensions(ImpalaTestSuite):
        These tests either run with no debug action set or set their own debug action."""
     self.run_test_case('QueryTest/spilling-no-debug-action', vector)
 
+
+@pytest.mark.xfail(IMPALA_TEST_CLUSTER_PROPERTIES.is_remote_cluster(),
+                   reason='Queries may not spill on larger clusters')
+class TestSpillingBroadcastJoins(ImpalaTestSuite):
+  """Tests specifically targeted at shared broadcast joins for mt_dop."""
+  @classmethod
+  def get_workload(self):
+    return 'functional-query'
+
+  @classmethod
+  def add_test_dimensions(cls):
+    super(TestSpillingBroadcastJoins, cls).add_test_dimensions()
+    cls.ImpalaTestMatrix.clear_constraints()
+    # Use parquet because it has 9 input splits for lineitem, hence can have a
+    # higher effective dop than parquet, which only has 3 splits.
+    cls.ImpalaTestMatrix.add_dimension(create_kudu_dimension('tpch'))
+    debug_action_dims = CORE_DEBUG_ACTION_DIMS
+    if cls.exploration_strategy() == 'exhaustive':
+      debug_action_dims = CORE_DEBUG_ACTION_DIMS + EXHAUSTIVE_DEBUG_ACTION_DIMS
+    # Tests are calibrated so that they can execute and spill with this page size.
+    cls.ImpalaTestMatrix.add_dimension(
+        create_exec_option_dimension_from_dict({'default_spillable_buffer_size': ['256k'],
+          'debug_action': debug_action_dims, 'mt_dop': [3]}))
+
+  def test_spilling_broadcast_joins(self, vector):
+    self.run_test_case('QueryTest/spilling-broadcast-joins', vector)