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/02/20 02:02:58 UTC

[impala] 03/03: IMPALA-4224: execute separate join builds fragments

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

commit 0bb056e525794fca41cd333bc2896098566945bb
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Mon Nov 25 18:28:07 2019 -0800

    IMPALA-4224: execute separate join builds fragments
    
    This enables parallel plans with the join build in a
    separate fragment and fixes all of the ensuing fallout.
    After this change, mt_dop plans with joins have separate
    build fragments. There is still a 1:1 relationship between
    join nodes and builders, so the builders are only accessed
    by the join node's thread after it is handed off. This lets
    us defer the work required to make PhjBuilder and NljBuilder
    safe to be shared between nodes.
    
    Planner changes:
    * Combined the parallel and distributed planning code paths.
    * Misc fixes to generate reasonable thrift structures in the
      query exec requests, i.e. containing the right nodes.
    * Fixes to resource calculations for the separate build plans.
    ** Calculate separate join/build resource consumption.
    ** Simplified the resource estimation by calculating resource
       consumption for each fragment separately, and assuming that
       all fragments hit their peak resource consumption at the
       same time. IMPALA-9255 is the follow-on to make the resource
       estimation more accurate.
    
    Scheduler changes:
    * Various fixes to handle multiple TPlanExecInfos correctly,
      which are generated by the planner for the different cohorts.
    * Add logic to colocate build fragments with parent fragments.
    
    Runtime filter changes:
    * Build sinks now produce runtime filters, which required
      planner and coordinator fixes to handle.
    
    DataSink changes:
    * Close the input plan tree before calling FlushFinal() to release
      resources. This depends on Send() not holding onto references
      to input batches, which was true except for NljBuilder. This
      invariant is documented.
    
    Join builder changes:
    * Add a common base class for PhjBuilder and NljBuilder with
      functions to handle synchronisation with the join node.
    * Close plan tree earlier in FragmentInstanceState::Exec()
      so that peak resource requirements are lower.
    * The NLJ always copies input batches, so that it can close
      its input tree.
    
    JoinNode changes:
    * Join node blocks waiting for build-side to be ready,
      then eventually signals that it's done, allowing the builder
      to be cleaned up.
    * NLJ and PHJ nodes handle both the integrated builder and
      the external builder. There is a 1:1 relationship between
      the node and the builder, so we don't deal with thread safety
      yet.
    * Buffer reservations are transferred between the builder and join
      node when running with the separate builder. This is not really
      necessary right now, since it is all single-threaded, but will
      be important for the shared broadcast.
      - The builder transfers memory for probe buffers to the join node
        at the end of each build phase.
      - At end of each probe phase, reservation needs to be handed back
        to builder (or released).
    
    ExecSummary changes:
    * The summary logic was modified to handle connecting fragments
      via join builds. The logic is an extension of what was used
      for exchanges.
    
    Testing:
    * Enable --unlock_mt_dop for end-to-end tests
    * Migrate some tests to run as part of end-to-end tests instead of
      custom cluster.
    * Add mt_dop dimension to various end-to-end tests to provide
      coverage of join queries, spill-to-disk and cancellation.
    * Ran a single node TPC-H and TPC-DS stress test with mt_dop=0
      and mt_dop=4.
    
    Perf:
    * Ran TPC-H scale factor 30 locally with mt_dop=0. No significant
      change.
    
    Change-Id: I4403c8e62d9c13854e7830602ee613f8efc80c58
    Reviewed-on: http://gerrit.cloudera.org:8080/14859
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exec/CMakeLists.txt                         |   1 +
 be/src/exec/blocking-join-node.cc                  | 128 +++++---
 be/src/exec/blocking-join-node.h                   |  87 +++++-
 be/src/exec/data-sink.cc                           |  12 +-
 be/src/exec/data-sink.h                            |  14 +-
 be/src/exec/exec-node.h                            |   8 +-
 be/src/exec/join-builder.cc                        | 112 +++++++
 be/src/exec/join-builder.h                         | 201 +++++++++++++
 be/src/exec/nested-loop-join-builder.cc            |  46 ++-
 be/src/exec/nested-loop-join-builder.h             |  46 ++-
 be/src/exec/nested-loop-join-node.cc               |  52 ++--
 be/src/exec/nested-loop-join-node.h                |   6 +-
 be/src/exec/partitioned-hash-join-builder.cc       | 289 +++++++++++++-----
 be/src/exec/partitioned-hash-join-builder.h        | 226 ++++++++------
 be/src/exec/partitioned-hash-join-node.cc          | 151 ++++++----
 be/src/exec/partitioned-hash-join-node.h           |  31 +-
 be/src/runtime/bufferpool/buffer-pool-internal.h   |   9 +-
 be/src/runtime/bufferpool/buffer-pool-test.cc      |   6 +
 be/src/runtime/bufferpool/buffer-pool.cc           |  43 ++-
 be/src/runtime/bufferpool/buffer-pool.h            |   9 +-
 be/src/runtime/coordinator-backend-state.cc        |   1 +
 be/src/runtime/coordinator.cc                      |  39 ++-
 be/src/runtime/fragment-instance-state.cc          |  27 +-
 be/src/runtime/fragment-instance-state.h           |   9 +-
 be/src/runtime/initial-reservations.cc             |   7 +-
 be/src/runtime/row-batch.cc                        |   3 +-
 be/src/runtime/runtime-state.cc                    |  13 +
 be/src/runtime/runtime-state.h                     |  19 +-
 be/src/runtime/spillable-row-batch-queue.h         |   3 +-
 be/src/util/summary-util.cc                        |   6 +-
 bin/run-all-tests.sh                               |   2 +-
 common/thrift/DataSinks.thrift                     |  21 +-
 common/thrift/ExecStats.thrift                     |   5 +-
 common/thrift/PlanNodes.thrift                     |  35 ++-
 common/thrift/Types.thrift                         |   1 -
 .../java/org/apache/impala/planner/DataSink.java   |  13 +
 .../org/apache/impala/planner/HashJoinNode.java    |  61 +++-
 .../org/apache/impala/planner/JoinBuildSink.java   |  31 +-
 .../java/org/apache/impala/planner/JoinNode.java   |  95 ++++--
 .../apache/impala/planner/NestedLoopJoinNode.java  |  17 +-
 .../org/apache/impala/planner/ParallelPlanner.java |  10 +-
 .../org/apache/impala/planner/PlanFragment.java    | 166 +++++++----
 .../java/org/apache/impala/planner/PlanNode.java   |  31 +-
 .../java/org/apache/impala/planner/Planner.java    |  41 +--
 .../org/apache/impala/planner/ResourceProfile.java |  30 +-
 .../java/org/apache/impala/service/Frontend.java   |  26 +-
 shell/impala_client.py                             |   9 +-
 .../PlannerTest/mem-limit-broadcast-join.test      |   2 +-
 ...t-dop-validation-hdfs-num-rows-est-enabled.test |  23 +-
 .../queries/PlannerTest/mt-dop-validation.test     |  23 +-
 .../queries/PlannerTest/resource-requirements.test | 172 +++++------
 .../PlannerTest/spillable-buffer-sizing.test       |  86 +++---
 .../queries/PlannerTest/tpcds-all.test             | 324 ++++++++++-----------
 .../queries/PlannerTest/tpch-all.test              | 198 ++++++-------
 .../queries/QueryTest/spilling-large-rows.test     |   2 +-
 .../queries/QueryTest/spilling.test                |   6 +-
 tests/common/impala_test_suite.py                  |   1 +
 tests/custom_cluster/test_mt_dop.py                |  24 --
 tests/failure/test_failpoints.py                   |   9 +-
 tests/query_test/test_cancellation.py              |   6 +
 tests/query_test/test_join_queries.py              |  14 +
 tests/query_test/test_mt_dop.py                    |  27 +-
 tests/query_test/test_nested_types.py              |  34 +++
 tests/query_test/test_runtime_filters.py           |  23 ++
 tests/query_test/test_spilling.py                  |  18 +-
 65 files changed, 2159 insertions(+), 1031 deletions(-)

diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index e7d2caa..aef40d4 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -68,6 +68,7 @@ add_library(Exec
   hbase-scan-node.cc
   hbase-table-scanner.cc
   incr-stats-util.cc
+  join-builder.cc
   nested-loop-join-builder.cc
   nested-loop-join-node.cc
   non-grouping-aggregator.cc
diff --git a/be/src/exec/blocking-join-node.cc b/be/src/exec/blocking-join-node.cc
index 87cd7a0..1ed8e8f 100644
--- a/be/src/exec/blocking-join-node.cc
+++ b/be/src/exec/blocking-join-node.cc
@@ -17,16 +17,19 @@
 
 #include "exec/blocking-join-node.h"
 
+#include <algorithm>
 #include <sstream>
 
-#include "exec/data-sink.h"
+#include "exec/join-builder.h"
 #include "exprs/scalar-expr.h"
+#include "runtime/exec-env.h"
 #include "runtime/fragment-instance-state.h"
 #include "runtime/mem-tracker.h"
+#include "runtime/query-state.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
-#include "runtime/tuple-row.h"
 #include "runtime/thread-resource-mgr.h"
+#include "runtime/tuple-row.h"
 #include "util/debug-util.h"
 #include "util/runtime-profile-counters.h"
 #include "util/thread.h"
@@ -41,15 +44,12 @@ using namespace impala;
 const char* BlockingJoinNode::LLVM_CLASS_NAME = "class.impala::BlockingJoinNode";
 
 Status BlockingJoinPlanNode::Init(const TPlanNode& tnode, RuntimeState* state) {
+  DCHECK(tnode.__isset.join_node);
   RETURN_IF_ERROR(PlanNode::Init(tnode, state));
-  TJoinOp::type join_op;
-  if (tnode_->node_type == TPlanNodeType::HASH_JOIN_NODE) {
-    join_op = tnode.hash_join_node.join_op;
-  } else {
-    DCHECK(tnode_->node_type == TPlanNodeType::NESTED_LOOP_JOIN_NODE);
-    join_op = tnode.nested_loop_join_node.join_op;
-  }
-  DCHECK(!IsSemiJoin(join_op) || conjuncts_.size() == 0);
+  build_row_desc_ = state->obj_pool()->Add(
+      new RowDescriptor(state->desc_tbl(), tnode.join_node.build_tuples,
+        tnode.join_node.nullable_build_tuples));
+  DCHECK(!IsSemiJoin(tnode.join_node.join_op) || conjuncts_.size() == 0);
   return Status::OK();
 }
 
@@ -71,23 +71,22 @@ BlockingJoinNode::~BlockingJoinNode() {
 }
 
 Status BlockingJoinNode::Prepare(RuntimeState* state) {
+  DCHECK_EQ(UseSeparateBuild(state->query_options()) ? 1 : 2, children_.size());
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(ExecNode::Prepare(state));
 
-  runtime_profile_->AddLocalTimeCounter(bind<int64_t>(
-      &BlockingJoinNode::LocalTimeCounterFn, runtime_profile_->total_time_counter(),
-      child(0)->runtime_profile()->total_time_counter(),
-      child(1)->runtime_profile()->total_time_counter(),
-      &built_probe_overlap_stop_watch_));
   build_timer_ = ADD_TIMER(runtime_profile(), "BuildTime");
   probe_timer_ = ADD_TIMER(runtime_profile(), "ProbeTime");
   build_row_counter_ = ADD_COUNTER(runtime_profile(), "BuildRows", TUnit::UNIT);
   probe_row_counter_ = ADD_COUNTER(runtime_profile(), "ProbeRows", TUnit::UNIT);
 
+  // The right child (if present) must match the build row layout.
+  DCHECK(children_.size() == 1 || build_row_desc().Equals(*children_[1]->row_desc()))
+      << build_row_desc().DebugString() << " " << children_[1]->row_desc()->DebugString();
   // Validate the row desc layout is what we expect because the current join
   // implementation relies on it to enable some optimizations.
-  int num_left_tuples = child(0)->row_desc()->tuple_descriptors().size();
-  int num_build_tuples = child(1)->row_desc()->tuple_descriptors().size();
+  int num_probe_tuples = probe_row_desc().tuple_descriptors().size();
+  int num_build_tuples = build_row_desc().tuple_descriptors().size();
 
 #ifndef NDEBUG
   switch (join_op_) {
@@ -95,13 +94,13 @@ Status BlockingJoinNode::Prepare(RuntimeState* state) {
     case TJoinOp::LEFT_SEMI_JOIN:
     case TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN: {
       // Only return the surviving probe-side tuples.
-      DCHECK(row_desc()->Equals(*child(0)->row_desc()));
+      DCHECK(row_desc()->Equals(probe_row_desc()));
       break;
     }
     case TJoinOp::RIGHT_ANTI_JOIN:
     case TJoinOp::RIGHT_SEMI_JOIN: {
       // Only return the surviving build-side tuples.
-      DCHECK(row_desc()->Equals(*child(1)->row_desc()));
+      DCHECK(row_desc()->Equals(build_row_desc()));
       break;
     }
     default: {
@@ -111,20 +110,22 @@ Status BlockingJoinNode::Prepare(RuntimeState* state) {
       //   result[0] = left[0]
       //   result[1] = build[0]
       //   result[2] = build[1]
-      for (int i = 0; i < num_left_tuples; ++i) {
-        TupleDescriptor* desc = child(0)->row_desc()->tuple_descriptors()[i];
+      for (int i = 0; i < num_probe_tuples; ++i) {
+        TupleDescriptor* desc = probe_row_desc().tuple_descriptors()[i];
         DCHECK_EQ(i, row_desc()->GetTupleIdx(desc->id()));
       }
       for (int i = 0; i < num_build_tuples; ++i) {
-        TupleDescriptor* desc = child(1)->row_desc()->tuple_descriptors()[i];
-        DCHECK_EQ(num_left_tuples + i, row_desc()->GetTupleIdx(desc->id()));
+        TupleDescriptor* desc = build_row_desc().tuple_descriptors()[i];
+        DCHECK_EQ(num_probe_tuples + i, row_desc()->GetTupleIdx(desc->id()))
+            << row_desc()->DebugString() << "\n" << probe_row_desc().DebugString() << "\n"
+            << build_row_desc().DebugString();
       }
       break;
     }
   }
 #endif
 
-  probe_tuple_row_size_ = num_left_tuples * sizeof(Tuple*);
+  probe_tuple_row_size_ = num_probe_tuples * sizeof(Tuple*);
   build_tuple_row_size_ = num_build_tuples * sizeof(Tuple*);
 
   if (IsSemiJoin(join_op_)) {
@@ -132,10 +133,8 @@ Status BlockingJoinNode::Prepare(RuntimeState* state) {
         new char[probe_tuple_row_size_ + build_tuple_row_size_]);
   }
 
-  build_batch_.reset(
-      new RowBatch(child(1)->row_desc(), state->batch_size(), mem_tracker()));
-  probe_batch_.reset(
-      new RowBatch(child(0)->row_desc(), state->batch_size(), mem_tracker()));
+  build_batch_.reset(new RowBatch(&build_row_desc(), state->batch_size(), mem_tracker()));
+  probe_batch_.reset(new RowBatch(&probe_row_desc(), state->batch_size(), mem_tracker()));
   return Status::OK();
 }
 
@@ -153,7 +152,8 @@ void BlockingJoinNode::Close(RuntimeState* state) {
 }
 
 void BlockingJoinNode::ProcessBuildInputAsync(
-    RuntimeState* state, DataSink* build_sink, Status* status) {
+    RuntimeState* state, JoinBuilder* build_sink, Status* status) {
+  DCHECK(!UseSeparateBuild(state->query_options()));
   DCHECK(status != nullptr);
   SCOPED_THREAD_COUNTER_MEASUREMENT(state->total_thread_statistics());
   {
@@ -181,31 +181,72 @@ void BlockingJoinNode::ProcessBuildInputAsync(
   state->resource_pool()->ReleaseThreadToken(false);
 }
 
-Status BlockingJoinNode::Open(RuntimeState* state) {
+Status BlockingJoinNode::OpenImpl(RuntimeState* state, JoinBuilder** separate_builder) {
   RETURN_IF_ERROR(ExecNode::Open(state));
   eos_ = false;
   probe_side_eos_ = false;
+
+  if (open_called_) return Status::OK();
+  // The below code should only run once.
+  if (UseSeparateBuild(state->query_options())) {
+    // Find the input fragment's build sink. We do this in the Open() phase so we don't
+    // block this finstance's Prepare() phase on the build finstance's Prepare() phase.
+    const vector<TJoinBuildInput>& build_inputs =
+        state->instance_ctx().join_build_inputs;
+    auto it = std::find_if(build_inputs.begin(), build_inputs.end(),
+        [this](const TJoinBuildInput& bi) { return bi.join_node_id == id_; });
+    DCHECK(it != build_inputs.end());
+    FragmentInstanceState* build_finstance;
+    RETURN_IF_ERROR(state->query_state()->GetFInstanceState(
+        it->input_finstance_id, &build_finstance));
+    TDataSinkType::type build_sink_type =
+        build_finstance->fragment_ctx().fragment.output_sink.type;
+    DCHECK(IsJoinBuildSink(build_sink_type));
+    *separate_builder = build_finstance->GetJoinBuildSink();
+    DCHECK(*separate_builder != nullptr);
+  } 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
+    // 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(),
+        child(0)->runtime_profile()->total_time_counter(),
+        child(1)->runtime_profile()->total_time_counter(),
+        &built_probe_overlap_stop_watch_));
+  }
+  open_called_ = true;
   return Status::OK();
 }
 
 Status BlockingJoinNode::ProcessBuildInputAndOpenProbe(
-    RuntimeState* state, DataSink* build_sink) {
-  // If this node is not inside a subplan, we are running with mt_dop=0 (i.e. no
-  // fragment-level multithreading) and can get a thread token, initiate the
-  // construction of the build-side table in a separate thread, so that the left child
-  // can do any initialisation in parallel. Otherwise, do this in the main thread.
-  // Inside a subplan we expect Open() to be called a number of times proportional to the
-  // input data of the SubplanNode, so we prefer doing processing the build input in the
-  // main thread, assuming that thread creation is expensive relative to a single subplan
-  // iteration.
+    RuntimeState* state, JoinBuilder* build_sink) {
+  // This function implements various strategies for executing Open() on the left child
+  // and doing or waiting for the join build. Some allow the two to proceed in parallel,
+  // while others do them serially. Generally parallelism yields better performance
+  // except inside a subplan. There we expect Open() to be called a number of times
+  // proportional to the input data of the SubplanNode, so processing the build input
+  // in the main thread can be more efficient, assuming that thread creation is expensive
+  // relative to a single subplan iteration.
   //
   // In this block, we also compute the 'overlap' time for the left and right child. This
   // is the time (i.e. clock reads) when the right child stops overlapping with the left
   // child. For the single threaded case, the left and right child never overlap. For the
   // build side in a different thread, the overlap stops when the left child Open()
   // returns.
-  if (!IsInSubplan() && state->query_options().mt_dop == 0
-      && state->resource_pool()->TryAcquireThreadToken()) {
+  if (UseSeparateBuild(state->query_options())) {
+    // Open the left child in parallel before waiting for the build fragment to maximise
+    // parallelism. The build execution is done concurrently by the build finstance's
+    // thread.
+    RETURN_IF_ERROR(child(0)->Open(state));
+    // 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));
+    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
+    // table build in a separate thread.
     Status build_side_status;
     runtime_profile()->AppendExecOption("Join Build-Side Prepared Asynchronously");
     string thread_name = Substitute("join-build-thread (finst:$0, plan-node-id:$1)",
@@ -283,8 +324,9 @@ Status BlockingJoinNode::GetFirstProbeRow(RuntimeState* state) {
 }
 
 template <bool ASYNC_BUILD>
-Status BlockingJoinNode::SendBuildInputToSink(RuntimeState* state,
-    DataSink* build_sink) {
+Status BlockingJoinNode::SendBuildInputToSink(
+    RuntimeState* state, JoinBuilder* build_sink) {
+  DCHECK(!UseSeparateBuild(state->query_options()));
   {
     SCOPED_TIMER(build_timer_);
     RETURN_IF_ERROR(build_sink->Open(state));
diff --git a/be/src/exec/blocking-join-node.h b/be/src/exec/blocking-join-node.h
index e433833..f801ad1 100644
--- a/be/src/exec/blocking-join-node.h
+++ b/be/src/exec/blocking-join-node.h
@@ -30,6 +30,7 @@
 
 namespace impala {
 
+class JoinBuilder;
 class RowBatch;
 class TupleRow;
 
@@ -39,11 +40,38 @@ class BlockingJoinPlanNode : public PlanNode {
   /// work, e.g. creating expr trees.
   virtual Status Init(const TPlanNode& tnode, RuntimeState* state) override;
   virtual Status CreateExecNode(RuntimeState* state, ExecNode** node) const override = 0;
+
+  /// Returns true if this join node will use a separate builder that is the root sink
+  /// of a different fragment. Otherwise the builder is owned by this node and consumes
+  /// input from the second child node. This depends on the containing subplan being
+  /// initialized, and isn't accurate until the whole PlanNode tree has been initialized.
+  bool UseSeparateBuild(const TQueryOptions& query_options) const {
+    return !IsInSubplan() && query_options.mt_dop > 0 && query_options.num_nodes != 1;
+  }
+
+  const RowDescriptor& probe_row_desc() const { return *children_[0]->row_descriptor_; }
+  const RowDescriptor& build_row_desc() const {
+    DCHECK(build_row_desc_ != nullptr);
+    return *build_row_desc_;
+  }
+
+ protected:
+  /// This is the same as the RowDescriptor of the build sink, if the join build is
+  /// separate, or the right child, if the join build is integrated into the node.
+  /// Owned by RuntimeState's object pool.
+  RowDescriptor* build_row_desc_ = nullptr;
 };
 
-/// Abstract base class for join nodes that block while consuming all rows from their
-/// right child in Open(). There is no implementation of Reset() because the Open()
-/// sufficiently covers setting members into a 'reset' state.
+/// Abstract base class for join nodes that block in Open() until all rows from the
+/// right input plan tree have been processed.
+///
+/// BlockingJoinNode and JoinBuilder subclasses interact together to implement a blocking
+/// join: the builder. Two modes are supported: an integrated join build, where the
+/// JoinBuilder is owned by the BlockingJoinNode, and a separate join build, where the
+/// JoinBuilder is owned by a separate build fragment co-located in the same Impala
+/// daemon, and the join node synchronizes with the builder to access build-side data
+/// structures.
+///
 /// TODO: Remove the restriction that the tuples in the join's output row have to
 /// correspond to the order of its child exec nodes. See the DCHECKs in Init().
 
@@ -58,8 +86,10 @@ class BlockingJoinNode : public ExecNode {
   /// Prepare() work, e.g. codegen.
   virtual Status Prepare(RuntimeState* state);
 
+  /// Helper called by subclass's Open() implementation.
   /// Calls ExecNode::Open() and initializes 'eos_' and 'probe_side_eos_'.
-  virtual Status Open(RuntimeState* state);
+  /// If the join build is separate, the join builder is returned in *separate_builder.
+  Status OpenImpl(RuntimeState* state, JoinBuilder** separate_builder);
 
   /// Transfers resources from 'probe_batch_' to 'row_batch'.
   virtual Status Reset(RuntimeState* state, RowBatch* row_batch);
@@ -74,6 +104,14 @@ class BlockingJoinNode : public ExecNode {
   const std::string node_name_;
   TJoinOp::type join_op_;
 
+  /// True if OpenImpl() was called.
+  bool open_called_ = false;
+
+  /// True if this join node has called WaitForInitialBuild() on the corresponding
+  /// separate join builder. This means that CloseFromProbe() needs to be called
+  /// on the builder.
+  bool waited_for_build_ = false;
+
   /// Store in node to avoid reallocating. Cleared after build completes.
   boost::scoped_ptr<RowBatch> build_batch_;
 
@@ -106,7 +144,7 @@ class BlockingJoinNode : public ExecNode {
   RuntimeProfile::Counter* probe_row_counter_;   // num probe (left child) rows
 
   /// Stopwatch that measures the build child's Open/GetNext time that overlaps
-  /// with the probe child Open().
+  /// with the probe child Open(). Not used for separate join builds.
   MonotonicStopWatch built_probe_overlap_stop_watch_;
 
   // True for a join node subclass if the build side can be closed before the probe
@@ -118,14 +156,15 @@ class BlockingJoinNode : public ExecNode {
   // TODO: IMPALA-4179: this should always be true once resource transfer has been fixed.
   virtual bool CanCloseBuildEarly() const { return false; }
 
+  /// Acquire resources for this ExecNode required for the build phase.
   /// Called by BlockingJoinNode after opening child(1) succeeds and before
-  /// SendBuildInputToSink is called to allocate resources for this ExecNode.
+  /// this node either waits for the separate build or calls SendBuildInputToSink().
   virtual Status AcquireResourcesForBuild(RuntimeState* state) { return Status::OK(); }
 
   /// Processes the build-side input, which should be already open, by sending it to
   /// 'build_sink', wand opens the probe side. Will do both concurrently if not in a
   /// subplan and an extra thread token is available.
-  Status ProcessBuildInputAndOpenProbe(RuntimeState* state, DataSink* build_sink);
+  Status ProcessBuildInputAndOpenProbe(RuntimeState* state, JoinBuilder* build_sink);
 
   /// Set up 'current_probe_row_' to point to the first input row from the left child
   /// (probe side). Fills 'probe_batch_' with rows from the left child and updates
@@ -195,20 +234,40 @@ class BlockingJoinNode : public ExecNode {
       const RuntimeProfile::Counter* right_child_time,
       const MonotonicStopWatch* child_overlap_timer);
 
+  const BlockingJoinPlanNode& plan_node() const {
+    return static_cast<const BlockingJoinPlanNode&>(plan_node_);
+  }
+
+  /// Returns true if this join node is using a separate builder that is the root sink
+  /// of a different fragment. Otherwise the builder is owned by this node and consumes
+  /// input from the second child node.
+  bool UseSeparateBuild(const TQueryOptions& query_options) const {
+    return plan_node().UseSeparateBuild(query_options);
+  }
+
+  const RowDescriptor& probe_row_desc() const {
+    return plan_node().probe_row_desc();
+  }
+
+  const RowDescriptor& build_row_desc() const {
+    return plan_node().build_row_desc();
+  }
+
  private:
-  /// Helper function to process the build input by sending it to a DataSink. The build
-  /// input must already be open before calling this. ASYNC_BUILD enables timers that
-  /// impose some overhead but are required if the build is processed concurrently with
-  /// the Open() of the left child.
+  /// Helper function to process the build input by sending it to the integrated
+  /// JoinBuilder. The build input must already be open before calling this. ASYNC_BUILD
+  /// enables timers that impose some overhead but are required if the build is processed
+  /// concurrently with the Open() of the left child.
   template <bool ASYNC_BUILD>
-  Status SendBuildInputToSink(RuntimeState* state, DataSink* build_sink);
+  Status SendBuildInputToSink(RuntimeState* state, JoinBuilder* build_sink);
 
   /// The main function for the thread that opens the build side and processes the build
   /// input asynchronously.  Its status is returned in the 'status' promise. If
   /// 'build_sink' is non-NULL, it is used for the build. Otherwise, ProcessBuildInput()
   /// is called on the subclass.
-  void ProcessBuildInputAsync(RuntimeState* state, DataSink* build_sink, Status* status);
+  void ProcessBuildInputAsync(
+      RuntimeState* state, JoinBuilder* build_sink, Status* status);
 };
-}
+} // namespace impala
 
 #endif
diff --git a/be/src/exec/data-sink.cc b/be/src/exec/data-sink.cc
index b50d33e..a4fb3ac 100644
--- a/be/src/exec/data-sink.cc
+++ b/be/src/exec/data-sink.cc
@@ -28,6 +28,8 @@
 #include "exec/kudu-util.h"
 #include "exec/blocking-plan-root-sink.h"
 #include "exec/buffered-plan-root-sink.h"
+#include "exec/nested-loop-join-builder.h"
+#include "exec/partitioned-hash-join-builder.h"
 #include "exec/plan-root-sink.h"
 #include "exprs/scalar-expr.h"
 #include "gen-cpp/ImpalaInternalService_constants.h"
@@ -87,8 +89,14 @@ Status DataSinkConfig::CreateConfig(const TDataSink& thrift_sink,
     case TDataSinkType::PLAN_ROOT_SINK:
       data_sink = pool->Add(new PlanRootSinkConfig());
       break;
-    case TDataSinkType::JOIN_BUILD_SINK:
-    // IMPALA-4224 - join build sink not supported in backend execution.
+    case TDataSinkType::HASH_JOIN_BUILDER: {
+      data_sink = pool->Add(new PhjBuilderConfig());
+      break;
+    }
+    case TDataSinkType::NESTED_LOOP_JOIN_BUILDER: {
+      data_sink = pool->Add(new NljBuilderConfig());
+      break;
+    }
     default:
       stringstream error_msg;
       map<int, const char*>::const_iterator i =
diff --git a/be/src/exec/data-sink.h b/be/src/exec/data-sink.h
index c8e296c..58f8af4 100644
--- a/be/src/exec/data-sink.h
+++ b/be/src/exec/data-sink.h
@@ -115,7 +115,13 @@ class DataSink {
   /// Call before Send() to open the sink and initialize output expression evaluators.
   virtual Status Open(RuntimeState* state);
 
-  /// Send a row batch into this sink. Send() may modify 'batch' by acquiring its state.
+  /// Send a row batch into this sink. Generally, Send() should not retain any references
+  /// to data in 'batch' after it returns, so that the caller can free 'batch' and all
+  /// associated memory. This is a hard requirement if the sink is being used as the
+  /// output sink of the fragment, but can be relaxed in certain contexts, e.g. an
+  /// embedded NljBuilder.
+  /// TODO: IMPALA-5832: we could allow sinks to acquire resources of 'batch' if we
+  /// make it possible to always acquire referenced memory.
   virtual Status Send(RuntimeState* state, RowBatch* batch) = 0;
 
   /// Flushes any remaining buffered state.
@@ -176,5 +182,11 @@ class DataSink {
   std::vector<ScalarExpr*> output_exprs_;
   std::vector<ScalarExprEvaluator*> output_expr_evals_;
 };
+
+static inline bool IsJoinBuildSink(const TDataSinkType::type& type) {
+  return type == TDataSinkType::HASH_JOIN_BUILDER
+      || type == TDataSinkType::NESTED_LOOP_JOIN_BUILDER;
+}
+
 } // namespace impala
 #endif
diff --git a/be/src/exec/exec-node.h b/be/src/exec/exec-node.h
index e57648e..fa90ffb 100644
--- a/be/src/exec/exec-node.h
+++ b/be/src/exec/exec-node.h
@@ -81,6 +81,8 @@ class PlanNode {
 
   virtual ~PlanNode(){}
 
+  bool IsInSubplan() const { return containing_subplan_ != nullptr; }
+
   /// TODO: IMPALA-9216: Add accessor methods for these members instead of making
   /// them public.
   /// Reference to the thrift node that represents this PlanNode.
@@ -99,7 +101,7 @@ class PlanNode {
   std::vector<PlanNode*> children_;
 
   /// Pointer to the containing SubplanPlanNode or NULL if not inside a subplan.
-  /// Set by the containing SubplanPlanNode::Prepare() before Prepare() is called on
+  /// Set by the containing SubplanPlanNode::Init() before Init() is called on
   /// 'this' node. Not owned.
   SubplanPlanNode* containing_subplan_ = nullptr;
 
@@ -320,7 +322,7 @@ class ExecNode {
   bool IsNodeCodegenDisabled() const;
 
   /// Returns true if this node is inside the right-hand side plan tree of a SubplanNode.
-  bool IsInSubplan() const { return plan_node_.containing_subplan_ != nullptr; }
+  bool IsInSubplan() const { return plan_node_.IsInSubplan(); }
 
   /// Names of counters shared by all exec nodes
   static const std::string ROW_THROUGHPUT_COUNTER;
@@ -388,7 +390,7 @@ class ExecNode {
   RowDescriptor& row_descriptor_;
 
   /// Resource information sent from the frontend.
-  const TBackendResourceProfile resource_profile_;
+  const TBackendResourceProfile& resource_profile_;
 
   /// debug-only: if debug_action_ is not INVALID, node will perform action in
   /// debug_phase_
diff --git a/be/src/exec/join-builder.cc b/be/src/exec/join-builder.cc
new file mode 100644
index 0000000..18a0a38
--- /dev/null
+++ b/be/src/exec/join-builder.cc
@@ -0,0 +1,112 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/join-builder.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+Status JoinBuilderConfig::Init(
+    const TDataSink& tsink, const RowDescriptor* input_row_desc, RuntimeState* state) {
+  RETURN_IF_ERROR(DataSinkConfig::Init(tsink, input_row_desc, state));
+  join_node_id_ = tsink.join_build_sink.dest_node_id;
+  join_op_ = tsink.join_build_sink.join_op;
+  return Status::OK();
+}
+
+JoinBuilder::JoinBuilder(TDataSinkId sink_id, const JoinBuilderConfig& sink_config,
+    const string& name, RuntimeState* state)
+  : 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) {}
+
+JoinBuilder::~JoinBuilder() {
+  DCHECK_EQ(0, probe_refcount_);
+}
+
+void JoinBuilder::CloseFromProbe(RuntimeState* join_node_state) {
+  if (is_separate_build_) {
+    bool last_probe;
+    {
+      unique_lock<mutex> l(separate_build_lock_);
+      --probe_refcount_;
+      last_probe = probe_refcount_ == 0;
+      VLOG(3) << "JoinBuilder (id=" << join_node_id_ << ")"
+              << "closed from finstance "
+              << PrintId(join_node_state->fragment_instance_id())
+              << "probe_refcount_=" << probe_refcount_;
+      DCHECK_GE(probe_refcount_, 0);
+    }
+    // Only need to notify when the probe count is zero.
+    if (last_probe) build_wakeup_cv_.NotifyAll();
+  } else {
+    Close(join_node_state);
+  }
+}
+
+Status JoinBuilder::WaitForInitialBuild(RuntimeState* join_node_state) {
+  DCHECK(is_separate_build_);
+  join_node_state->AddCancellationCV(&probe_wakeup_cv_);
+  VLOG(2) << "JoinBuilder (id=" << join_node_id_ << ")"
+          << " WaitForInitialBuild() called by finstance "
+          << PrintId(join_node_state->fragment_instance_id());
+  unique_lock<mutex> l(separate_build_lock_);
+  // Wait until either the build is ready to use or this finstance has been cancelled.
+  // We can't safely pick up the build side if the build side was cancelled - instead we
+  // need to wait for this finstance to be cancelled.
+  while (!ready_to_probe_ && !join_node_state->is_cancelled()) {
+    probe_wakeup_cv_.Wait(l);
+  }
+  if (join_node_state->is_cancelled()) {
+    VLOG(2) << "Finstance " << PrintId(join_node_state->fragment_instance_id())
+            << " cancelled while waiting for JoinBuilder (id=" << join_node_id_ << ")";
+    return Status::CANCELLED;
+  }
+  ++probe_refcount_;
+  --outstanding_probes_;
+  VLOG(2) << "JoinBuilder (id=" << join_node_id_ << ")"
+          << " initial build handoff to finstance "
+          << PrintId(join_node_state->fragment_instance_id())
+          << " probe_refcount_=" << probe_refcount_
+          << " outstanding_probes_=" << outstanding_probes_;
+  DCHECK_GE(outstanding_probes_, 0);
+  return Status::OK();
+}
+
+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);
+  }
+  // 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
new file mode 100644
index 0000000..3e8b148
--- /dev/null
+++ b/be/src/exec/join-builder.h
@@ -0,0 +1,201 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <boost/thread/locks.hpp>
+
+#include "exec/data-sink.h"
+#include "util/condition-variable.h"
+
+namespace impala {
+
+class NljBuilder;
+class PhjBuilder;
+
+class JoinBuilderConfig : public DataSinkConfig {
+ public:
+  ~JoinBuilderConfig() override {}
+
+ protected:
+  friend class JoinBuilder;
+  friend class NljBuilder;
+  friend class PhjBuilder;
+
+  Status Init(const TDataSink& tsink, const RowDescriptor* input_row_desc,
+      RuntimeState* state) override;
+
+  /// The ID of the join plan node this is associated with.
+  int join_node_id_;
+
+  /// The join operation this is building for.
+  TJoinOp::type join_op_;
+};
+
+/// Join builder for use with BlockingJoinNode.
+///
+/// Implements the DataSink interface but also exposes some methods for direct use by
+/// BlockingJoinNode, e.g. the implemention of the protocol to hand off completed
+/// builds to the join node when the plan has a separate join fragment.
+///
+/// Example of Synchronization
+/// --------------------------
+/// The below sequence diagram shows an example of how the probe and build finstance
+/// threads synchronize to do the nested loop join build, hand it off to the probe thread
+/// and then release resources at the end. For simplicitly, this particular sequence
+/// assumes that Prepare() was already called on the build finstance thread so
+/// GetFInstanceState() returns immediately (otherwise Open() would block waiting for
+/// the build finstance to be prepared).
+///
+///
+/// +------------------+ +--------------------+       +-------------+  +-----------------+
+/// | probe finstance  | | NestedLoopJoinNode |       | NljBuilder  |  | build finstance |
+/// | thread           | |                    |       |             |  | thread          |
+/// +------------------+ +--------------------+       +-------------+  +-----------------+
+///         |             |                              |                             |
+///         | Open()      |                              |                             |
+///         |------------>|                              |                             |
+///         |             |                              |                      Open() |
+///         |             |                              |<----------------------------|
+///         |             |                              |       Send() multiple times |
+///         |             |                              |<----------------------------|
+///         |             | WaitForInitialBuild()        |                             |
+///         |             | blocks on probe_wakeup_cv_   |                             |
+///         |             |----------------------------->|                             |
+///         |             |                              |       Send() multiple times |
+///         |             |                              |<----------------------------|
+///         |             |                              | FlushFinal()                |
+///         |             |                              | calls HandoffToProbesAndWait()
+///         |             |                              | signals probe_wakeup_cv_    |
+///         |             |                              | blocks on build_wakeup_cv_  |
+///         |             |                              |<----------------------------|
+///         |             | WaitForInitialBuild() returns|                             |
+///         |             |<-----------------------------|                             |
+///         |             |                              |                             |
+///         |             | accesses data structures in  |                             |
+///         |             | read-only manner             |                             |
+///         |             |----------------------------->|                             |
+///         |             |                              |                             |
+///         |             | CloseFromProbe()             |                             |
+///         |             | signals build_wakeup_cv_     |                             |
+///         |             |----------------------------->|                             |
+///         |             |                              | FlushFinal() returns        |
+///         |             |                              |---------------------------->|
+///         |             |                              |                             |
+///         |             |                              | Close()                     |
+///         |             |                              |<----------------------------|
+///
+/// Various alternative flows are possible:
+/// * WaitForInitialBuild() may be called after FlushFinal(), in which case it can return
+///   immediately.
+/// * The query may be cancelled while the threads are blocked in WaitForInitialBuild()
+///   or HandoffToProbesAndWait(). In this case the threads need to get unblocked in a
+///   safe manner. Details are documented in the respective functions.
+///
+/// Other implementations, such as the partitioned hash join, may do additional
+/// synchronization in between WaitForInitialBuild() returning and CloseFromProbe() being
+/// called, instead of the simple read-only access of the NLJ. The details are are left
+/// to the specific implementations, but must be thread-safe if the builder is shared
+/// between multiple threads.
+
+class JoinBuilder : public DataSink {
+ public:
+  /// Construct the join builder. This is a separate build if 'sink_id' is valid, or
+  /// an integrated build if 'sink_id' is -1.
+  JoinBuilder(TDataSinkId sink_id, const JoinBuilderConfig& sink_config,
+      const string& name, RuntimeState* state);
+  virtual ~JoinBuilder();
+
+  /// Waits for a separate join build to complete and be ready for probing. Always called
+  /// from a BlockingJoinNode subclass in a different fragment instance.
+  ///
+  /// Returns OK if the initial build succeeded. Returns CANCELLED if the join node's
+  /// fragment is cancelled. If an error occurs executing the build side of the join,
+  /// we rely on query-wide cancellation to cancel the probe side of the join, instead
+  /// of propagating the error/cancellation via the JoinBuilder. In case of cancellation
+  /// or error, this function blocks until the join node's finstance is cancelled so
+  /// that the CANCELLED status does not race with the root cause of the query failure.
+  Status WaitForInitialBuild(RuntimeState* join_node_state);
+
+  /// This is called from BlockingJoinNode to signal that the node is done with the
+  /// builder. If this is an embedded join build, this simply closes the builder. If it is
+  /// 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);
+
+ protected:
+  /// ID of the join node that this builder is associated with.
+  const int join_node_id_;
+
+  /// The join operation this is building for.
+  const TJoinOp::type join_op_;
+
+  /// True if this is a separate DataSink at the root of its own fragment. Otherwise this
+  /// is embedded in a PartitionedHashJoinNode.
+  const bool is_separate_build_;
+
+  /////////////////////////////////////////////////////////////////////
+  /// BEGIN: Members that are used only when is_separate_build_ is true
+
+  // Lock used for synchronization between threads from the build and probe side (i.e.
+  // the build fragment thread and the probe-side thread executing the join node).
+  boost::mutex separate_build_lock_;
+
+  // Probe-side threads block on this while waiting for initial_build_complete_ = true
+  // (or for probe finstance cancellation).
+  // Protected by 'separate_build_lock_'.
+  ConditionVariable probe_wakeup_cv_;
+
+  // The build-side thread blocks on this while waiting for 'outstanding_probes_' and
+  // 'probe_refcount_' to go to zero (or for build finstance cancellation).
+  // Protected by 'separate_build_lock_'.
+  ConditionVariable build_wakeup_cv_;
+
+  // Set to true when the builder is ready for the probe side to use. Set to true in
+  // FlushFinal().
+  // Protected by 'separate_build_lock_'.
+  bool ready_to_probe_ = false;
+
+  // Number of probe-side threads that are expected to call WaitForInitialBuild()
+  // but have not yet called it.
+  // Protected by 'separate_build_lock_'.
+  int outstanding_probes_ = 0;
+
+  // Number of probe-side threads using this builder - i.e. WaitForInitialBuild() calls
+  // without a corresponding CloseBuilder() call.
+  // Protected by 'separate_build_lock_'.
+  int probe_refcount_ = 0;
+
+  /// END: Members that are used only when is_separate_build_ is true
+  /////////////////////////////////////////////////////////////////////
+
+  /// Called by the build-side thread of a separate join build once the initial build has
+  /// been completed and it is ready to hand off to probe-side threads. Blocks until the
+  /// probe side is finished using the build, or the build finstance is cancelled. When
+  /// this returns, no probe-side threads will be using the build and
+  /// WaitForInitialBuild() will return CANCELLED.
+  ///
+  /// This should be called by the FlushFinal() method of the subclass, after all other
+  /// resources from the build fragment have been released.
+  /// TODO: IMPALA-9255: reconsider this so that the build-side thread can exit instead
+  /// of being blocked indefinitely.
+  void HandoffToProbesAndWait(RuntimeState* build_side_state);
+};
+}
diff --git a/be/src/exec/nested-loop-join-builder.cc b/be/src/exec/nested-loop-join-builder.cc
index 4932a67..efdaa0d 100644
--- a/be/src/exec/nested-loop-join-builder.cc
+++ b/be/src/exec/nested-loop-join-builder.cc
@@ -22,23 +22,53 @@
 #include "runtime/mem-tracker.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
+#include "util/runtime-profile-counters.h"
 
 #include "common/names.h"
 
 using namespace impala;
 
-NljBuilder* NljBuilder::CreateSink(const RowDescriptor* row_desc, RuntimeState* state) {
+DataSink* NljBuilderConfig::CreateSink(const TPlanFragmentCtx& fragment_ctx,
+    const TPlanFragmentInstanceCtx& fragment_instance_ctx,
+    RuntimeState* state) const {
+  // We have one fragment per sink, so we can use the fragment index as the sink ID.
+  TDataSinkId sink_id = fragment_ctx.fragment.idx;
+  return NljBuilder::CreateSeparateBuilder(sink_id, *this, state);
+}
+
+Status NljBuilderConfig::Init(
+    const TDataSink& tsink, const RowDescriptor* input_row_desc, RuntimeState* state) {
+  RETURN_IF_ERROR(JoinBuilderConfig::Init(tsink, input_row_desc, state));
+  return Status::OK();
+}
+
+NljBuilder* NljBuilder::CreateEmbeddedBuilder(
+    const RowDescriptor* row_desc, RuntimeState* state, int join_node_id) {
   ObjectPool* pool = state->obj_pool();
-  DataSinkConfig* sink_config = pool->Add(new NljBuilderConfig());
+  NljBuilderConfig* sink_config = pool->Add(new NljBuilderConfig());
+  sink_config->join_node_id_ = join_node_id;
   sink_config->tsink_ = pool->Add(new TDataSink());
   sink_config->input_row_desc_ = row_desc;
   return pool->Add(new NljBuilder(*sink_config, state));
 }
 
-NljBuilder::NljBuilder(const DataSinkConfig& sink_config, RuntimeState* state)
-  : DataSink(-1, sink_config, "Nested Loop Join Builder", state),
+NljBuilder::NljBuilder(
+    TDataSinkId sink_id, const NljBuilderConfig& sink_config, RuntimeState* state)
+  : JoinBuilder(sink_id, sink_config, "Nested Loop Join Builder", state),
     build_batch_cache_(row_desc_, state->batch_size()) {}
 
+
+NljBuilder::NljBuilder(const NljBuilderConfig& sink_config, RuntimeState* state)
+  : JoinBuilder(-1, sink_config, "Nested Loop Join Builder", state),
+    build_batch_cache_(row_desc_, state->batch_size()) {}
+
+NljBuilder* NljBuilder::CreateSeparateBuilder(
+    TDataSinkId sink_id, const NljBuilderConfig& sink_config, RuntimeState* state) {
+  return state->obj_pool()->Add(new NljBuilder(sink_id, sink_config, state));
+}
+
+NljBuilder::~NljBuilder() {}
+
 Status NljBuilder::Prepare(RuntimeState* state, MemTracker* parent_mem_tracker) {
   RETURN_IF_ERROR(DataSink::Prepare(state, parent_mem_tracker));
   return Status::OK();
@@ -49,12 +79,15 @@ Status NljBuilder::Open(RuntimeState* state) {
 }
 
 Status NljBuilder::Send(RuntimeState* state, RowBatch* batch) {
+  SCOPED_TIMER(profile()->total_time_counter());
   // Swap the contents of the batch into a batch owned by the builder.
   RowBatch* build_batch = GetNextEmptyBatch();
   build_batch->AcquireState(batch);
 
   AddBuildBatch(build_batch);
-  if (build_batch->needs_deep_copy() || build_batch->num_buffers() > 0) {
+  if (is_separate_build_
+      || build_batch->flush_mode() == RowBatch::FlushMode::FLUSH_RESOURCES
+      || build_batch->num_buffers() > 0) {
     // This batch and earlier batches may refer to resources passed from the child
     // that aren't owned by the row batch itself. Deep copying ensures that the row
     // batches are backed by memory owned by this node that is safe to hold on to.
@@ -69,6 +102,7 @@ Status NljBuilder::Send(RuntimeState* state, RowBatch* batch) {
 }
 
 Status NljBuilder::FlushFinal(RuntimeState* state) {
+  SCOPED_TIMER(profile()->total_time_counter());
   if (copied_build_batches_.total_num_rows() > 0) {
     // To simplify things, we only want to process one list, so we need to copy
     // the remaining input batches.
@@ -77,10 +111,12 @@ Status NljBuilder::FlushFinal(RuntimeState* state) {
 
   DCHECK(copied_build_batches_.total_num_rows() == 0 ||
       input_build_batches_.total_num_rows() == 0);
+  if (is_separate_build_) HandoffToProbesAndWait(state);
   return Status::OK();
 }
 
 void NljBuilder::Reset() {
+  DCHECK(!is_separate_build_);
   build_batch_cache_.Reset();
   input_build_batches_.Reset();
   copied_build_batches_.Reset();
diff --git a/be/src/exec/nested-loop-join-builder.h b/be/src/exec/nested-loop-join-builder.h
index bdc3eb0..b66fd23 100644
--- a/be/src/exec/nested-loop-join-builder.h
+++ b/be/src/exec/nested-loop-join-builder.h
@@ -18,50 +18,59 @@
 #ifndef IMPALA_EXEC_NESTED_LOOP_JOIN_BUILDER_H
 #define IMPALA_EXEC_NESTED_LOOP_JOIN_BUILDER_H
 
+#include "common/atomic.h"
 #include "exec/blocking-join-node.h"
-#include "exec/data-sink.h"
+#include "exec/join-builder.h"
 #include "exec/row-batch-cache.h"
 #include "exec/row-batch-list.h"
 #include "runtime/descriptors.h"
 
 namespace impala {
 
-/// Dummy class needed to create an instance of the sink.
-class NljBuilderConfig : public DataSinkConfig {
+class NljBuilderConfig : public JoinBuilderConfig {
  public:
   DataSink* CreateSink(const TPlanFragmentCtx& fragment_ctx,
       const TPlanFragmentInstanceCtx& fragment_instance_ctx,
-      RuntimeState* state) const override {
-    DCHECK(false) << "Not Implemented";
-    return nullptr;
-  }
+      RuntimeState* state) const override;
 
   ~NljBuilderConfig() override {}
+
+ protected:
+  Status Init(const TDataSink& tsink, const RowDescriptor* input_row_desc,
+      RuntimeState* state) override;
 };
 
 /// Builder for the NestedLoopJoinNode that accumulates the build-side rows for the join.
-/// Implements the DataSink interface but also exposes some methods for direct use by
-/// NestedLoopJoinNode.
+/// Implements the JoinBuilder and DataSink interfaces but also exposes some methods for
+/// direct use by NestedLoopJoinNode.
 ///
 /// The builder will operate in one of two modes depending on the memory ownership of
 /// row batches pulled from the child node on the build side. If the row batches own all
 /// tuple memory, the non-copying mode is used and row batches are simply accumulated in
 /// the builder. If the batches reference tuple data they do not own, the copying mode
-/// is used and all data is deep copied into memory owned by the builder.
-class NljBuilder : public DataSink {
+/// is used and all data is deep copied into memory owned by the builder. We always
+/// use the copying mode for separate build sinks so that the source fragment plan tree
+/// can be safely torn down.
+class NljBuilder : public JoinBuilder {
  public:
-
   /// To be used by the NestedLoopJoinNode to create an instance of this sink.
-  static NljBuilder* CreateSink(const RowDescriptor* row_desc, RuntimeState* state);
+  static NljBuilder* CreateEmbeddedBuilder(
+      const RowDescriptor* row_desc, RuntimeState* state, int join_node_id);
+  // Factory method for separate builder.
+  static NljBuilder* CreateSeparateBuilder(
+      TDataSinkId sink_id, const NljBuilderConfig& sink_config, RuntimeState* state);
+
+  ~NljBuilder();
 
   /// Implementations of DataSink interface methods.
   virtual Status Prepare(RuntimeState* state, MemTracker* parent_mem_tracker) override;
   virtual Status Open(RuntimeState* state) override;
   virtual Status Send(RuntimeState* state, RowBatch* batch) override;
   virtual Status FlushFinal(RuntimeState* state) override;
-  virtual void Close(RuntimeState* state) override;
+  void Close(RuntimeState* state) override;
 
   /// Reset the builder to the same state as it was in after calling Open().
+  /// Not valid to call on a separate join build.
   void Reset();
 
   /// Returns the next build batch that should be filled and passed to AddBuildBatch().
@@ -77,7 +86,8 @@ class NljBuilder : public DataSink {
   inline void AddBuildBatch(RowBatch* batch) { input_build_batches_.AddRowBatch(batch); }
 
   /// Return a pointer to the final list of build batches.
-  /// Only valid to call after FlushFinal() has been called.
+  /// Only valid to call after FlushFinal() has been called. The returned build batches
+  /// are not mutated and valid to use until Close() is called on the builder.
   RowBatchList* GetFinalBuildBatches() {
     if (copied_build_batches_.total_num_rows() > 0) {
       DCHECK_EQ(input_build_batches_.total_num_rows(), 0);
@@ -91,7 +101,11 @@ class NljBuilder : public DataSink {
   inline RowBatchList* copied_build_batches() { return &copied_build_batches_; }
 
  private:
-  NljBuilder(const DataSinkConfig& sink_config, RuntimeState* state);
+  // Constructor for builder embedded in NestedLoopJoinNode.
+  NljBuilder(const NljBuilderConfig& sink_config, RuntimeState* state);
+  // Constructor for separate builder.
+  NljBuilder(
+      TDataSinkId sink_id, const NljBuilderConfig& sink_config, RuntimeState* state);
 
   /// Deep copy all build batches in 'input_build_batches_' to 'copied_build_batches_'.
   /// Resets all the source batches and clears 'input_build_batches_'.
diff --git a/be/src/exec/nested-loop-join-node.cc b/be/src/exec/nested-loop-join-node.cc
index c4d8379..bd76ddc 100644
--- a/be/src/exec/nested-loop-join-node.cc
+++ b/be/src/exec/nested-loop-join-node.cc
@@ -38,14 +38,13 @@ using namespace strings;
 
 Status NestedLoopJoinPlanNode::Init(const TPlanNode& tnode, RuntimeState* state) {
   RETURN_IF_ERROR(BlockingJoinPlanNode::Init(tnode, state));
-  DCHECK(tnode.__isset.nested_loop_join_node);
+  DCHECK(tnode.join_node.__isset.nested_loop_join_node);
   // join_conjunct_evals_ are evaluated in the context of rows assembled from
   // all inner and outer tuples.
-  RowDescriptor full_row_desc(
-      *children_[0]->row_descriptor_, *children_[1]->row_descriptor_);
-  RETURN_IF_ERROR(ScalarExpr::Create(tnode.nested_loop_join_node.join_conjuncts,
+  RowDescriptor full_row_desc(probe_row_desc(), build_row_desc());
+  RETURN_IF_ERROR(ScalarExpr::Create(tnode.join_node.nested_loop_join_node.join_conjuncts,
       full_row_desc, state, &join_conjuncts_));
-  DCHECK(tnode.nested_loop_join_node.join_op != TJoinOp::CROSS_JOIN
+  DCHECK(tnode.join_node.join_op != TJoinOp::CROSS_JOIN
       || join_conjuncts_.size() == 0)
       << "Join conjuncts in a cross join";
   return Status::OK();
@@ -60,7 +59,7 @@ Status NestedLoopJoinPlanNode::CreateExecNode(
 
 NestedLoopJoinNode::NestedLoopJoinNode(
     ObjectPool* pool, const NestedLoopJoinPlanNode& pnode, const DescriptorTbl& descs)
-  : BlockingJoinNode("NestedLoopJoinNode", pnode.tnode_->nested_loop_join_node.join_op,
+  : BlockingJoinNode("NestedLoopJoinNode", pnode.tnode_->join_node.join_op,
         pool, pnode, descs),
     build_batches_(NULL),
     current_build_row_idx_(0),
@@ -75,14 +74,21 @@ NestedLoopJoinNode::~NestedLoopJoinNode() {
 Status NestedLoopJoinNode::Open(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   ScopedOpenEventAdder ea(this);
-  RETURN_IF_ERROR(BlockingJoinNode::Open(state));
+  JoinBuilder* tmp_builder = nullptr;
+  RETURN_IF_ERROR(BlockingJoinNode::OpenImpl(state, &tmp_builder));
+  if (builder_ == nullptr) {
+    DCHECK(UseSeparateBuild(state->query_options()));
+    builder_ = dynamic_cast<NljBuilder*>(tmp_builder);
+    DCHECK(builder_ != nullptr);
+  }
   RETURN_IF_ERROR(ScalarExprEvaluator::Open(join_conjunct_evals_, state));
 
   // Check for errors and free expr result allocations before opening children.
   RETURN_IF_CANCELLED(state);
   RETURN_IF_ERROR(QueryMaintenance(state));
 
-  if (child(1)->type() == TPlanNodeType::type::SINGULAR_ROW_SRC_NODE) {
+  if (num_children() == 2 &&
+      child(1)->type() == TPlanNodeType::type::SINGULAR_ROW_SRC_NODE) {
     DCHECK(IsInSubplan());
     // When inside a subplan, open the first child before doing the build such that
     // UnnestNodes on the probe side are opened and project their unnested collection
@@ -111,14 +117,18 @@ Status NestedLoopJoinNode::Prepare(RuntimeState* state) {
   RETURN_IF_ERROR(BlockingJoinNode::Prepare(state));
   RETURN_IF_ERROR(ScalarExprEvaluator::Create(join_conjuncts_, state,
       pool_, expr_perm_pool(), expr_results_pool(), &join_conjunct_evals_));
-  builder_ = NljBuilder::CreateSink(child(1)->row_desc(), state);
-  RETURN_IF_ERROR(builder_->Prepare(state, mem_tracker()));
-  runtime_profile()->PrependChild(builder_->profile());
+
+  if (!UseSeparateBuild(state->query_options())) {
+    builder_ = NljBuilder::CreateEmbeddedBuilder(&build_row_desc(), state, id_);
+    RETURN_IF_ERROR(builder_->Prepare(state, mem_tracker()));
+    runtime_profile()->PrependChild(builder_->profile());
+  }
 
   // For some join modes we need to record the build rows with matches in a bitmap.
-  if (join_op_ == TJoinOp::RIGHT_ANTI_JOIN || join_op_ == TJoinOp::RIGHT_SEMI_JOIN ||
-      join_op_ == TJoinOp::RIGHT_OUTER_JOIN || join_op_ == TJoinOp::FULL_OUTER_JOIN) {
-    if (child(1)->type() == TPlanNodeType::type::SINGULAR_ROW_SRC_NODE) {
+  if (join_op_ == TJoinOp::RIGHT_ANTI_JOIN || join_op_ == TJoinOp::RIGHT_SEMI_JOIN
+      || join_op_ == TJoinOp::RIGHT_OUTER_JOIN || join_op_ == TJoinOp::FULL_OUTER_JOIN) {
+    if (num_children() == 2
+        && child(1)->type() == TPlanNodeType::type::SINGULAR_ROW_SRC_NODE) {
       // Allocate a fixed-size bitmap with a single element if we have a singular
       // row source node as our build child.
       int64_t bitmap_mem_usage = Bitmap::MemUsage(1);
@@ -150,9 +160,14 @@ void NestedLoopJoinNode::Close(RuntimeState* state) {
   ScalarExprEvaluator::Close(join_conjunct_evals_, state);
   ScalarExpr::Close(join_conjuncts_);
   if (builder_ != NULL) {
-    // IMPALA-6595: builder must be closed before child.
-    DCHECK(builder_->is_closed() || !children_[1]->is_closed());
-    builder_->Close(state);
+    // IMPALA-6595: builder must be closed before child. The separate build case is
+    // handled in FragmentInstanceState.
+    DCHECK(UseSeparateBuild(state->query_options()) || builder_->is_closed()
+        || !children_[1]->is_closed());
+    if (!UseSeparateBuild(state->query_options()) || waited_for_build_) {
+      builder_->CloseFromProbe(state);
+      waited_for_build_ = false;
+    }
   }
   build_batches_ = NULL;
   if (matching_build_rows_ != NULL) {
@@ -165,8 +180,9 @@ void NestedLoopJoinNode::Close(RuntimeState* state) {
 Status NestedLoopJoinNode::ConstructSingularBuildSide(RuntimeState* state) {
   // Optimized path for a common subplan shape with a singular row src node on the build
   // side that avoids expensive timers, virtual function calls, and other overhead.
-  DCHECK_EQ(child(1)->type(), TPlanNodeType::type::SINGULAR_ROW_SRC_NODE);
   DCHECK(IsInSubplan());
+  DCHECK_EQ(2, num_children());
+  DCHECK_EQ(child(1)->type(), TPlanNodeType::type::SINGULAR_ROW_SRC_NODE);
   RowBatch* batch = builder_->GetNextEmptyBatch();
   bool eos;
   RETURN_IF_ERROR(child(1)->GetNext(state, batch, &eos));
diff --git a/be/src/exec/nested-loop-join-node.h b/be/src/exec/nested-loop-join-node.h
index ae784ca..0af115f 100644
--- a/be/src/exec/nested-loop-join-node.h
+++ b/be/src/exec/nested-loop-join-node.h
@@ -66,12 +66,14 @@ class NestedLoopJoinNode : public BlockingJoinNode {
   /////////////////////////////////////////
   /// BEGIN: Members that must be Reset()
 
-  /// The build side rows of the join. Created in Prepare() and owned by runtime state.
+  /// 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.
   NljBuilder* builder_ = nullptr;
 
   /// Pointer to the RowBatchList (owned by 'builder_') that contains the batches to
   /// use during the probe phase.
-  RowBatchList* build_batches_;
+  RowBatchList* build_batches_ = nullptr;
 
   RowBatchList::TupleRowIterator build_row_iterator_;
 
diff --git a/be/src/exec/partitioned-hash-join-builder.cc b/be/src/exec/partitioned-hash-join-builder.cc
index 1779dba..c0e85e0 100644
--- a/be/src/exec/partitioned-hash-join-builder.cc
+++ b/be/src/exec/partitioned-hash-join-builder.cc
@@ -49,34 +49,36 @@ static const string PREPARE_FOR_READ_FAILED_ERROR_MSG =
 using namespace impala;
 using strings::Substitute;
 
-Status PhjBuilderConfig::Init(
-    const TDataSink& tsink, const RowDescriptor* input_row_desc, RuntimeState* state) {
-  return Status("Not Implemented.");
+static string ConstructBuilderName(int join_node_id) {
+  return Substitute("Hash Join Builder (join_node_id=$0)", join_node_id);
 }
 
 DataSink* PhjBuilderConfig::CreateSink(const TPlanFragmentCtx& fragment_ctx,
     const TPlanFragmentInstanceCtx& fragment_instance_ctx, RuntimeState* state) const {
-  DCHECK(false) << "Not Implemented";
-  return nullptr;
+  // We have one fragment per sink, so we can use the fragment index as the sink ID.
+  TDataSinkId sink_id = fragment_ctx.fragment.idx;
+  ObjectPool* pool = state->obj_pool();
+  return pool->Add(new PhjBuilder(sink_id, *this, state));
 }
 
 PhjBuilder* PhjBuilderConfig::CreateSink(BufferPool::ClientHandle* buffer_pool_client,
-    const std::string& join_node_label, int64_t spillable_buffer_size,
-    int64_t max_row_buffer_size, RuntimeState* state) const {
+    int64_t spillable_buffer_size, int64_t max_row_buffer_size,
+    RuntimeState* state) const {
   ObjectPool* pool = state->obj_pool();
-  return pool->Add(new PhjBuilder(*this, buffer_pool_client, join_node_label,
-      spillable_buffer_size, max_row_buffer_size, state));
+  return pool->Add(new PhjBuilder(*this, buffer_pool_client, spillable_buffer_size,
+        max_row_buffer_size, state));
 }
 
 Status PhjBuilderConfig::CreateConfig(RuntimeState* state, int join_node_id,
     TJoinOp::type join_op, const RowDescriptor* build_row_desc,
     const std::vector<TEqJoinCondition>& eq_join_conjuncts,
-    const std::vector<TRuntimeFilterDesc>& filters, const PhjBuilderConfig** sink) {
+    const std::vector<TRuntimeFilterDesc>& filters, uint32_t hash_seed,
+    const PhjBuilderConfig** sink) {
   ObjectPool* pool = state->obj_pool();
-  TDataSink* tsink = pool->Add(new TDataSink()); // just a dummy object.
+  TDataSink* tsink = pool->Add(new TDataSink());
   PhjBuilderConfig* data_sink = pool->Add(new PhjBuilderConfig());
-  RETURN_IF_ERROR(data_sink->Init(
-      state, *tsink, join_node_id, join_op, build_row_desc, eq_join_conjuncts, filters));
+  RETURN_IF_ERROR(data_sink->Init(state, join_node_id, join_op, build_row_desc,
+      eq_join_conjuncts, filters, hash_seed, tsink));
   *sink = data_sink;
   return Status::OK();
 }
@@ -115,46 +117,94 @@ Status PhjBuilderConfig::InitExprsAndFilters(RuntimeState* state,
   return Status::OK();
 }
 
-Status PhjBuilderConfig::Init(RuntimeState* state, const TDataSink& tsink,
-    int join_node_id, TJoinOp::type join_op, const RowDescriptor* build_row_desc,
+Status PhjBuilderConfig::Init(RuntimeState* state, int join_node_id,
+    TJoinOp::type join_op, const RowDescriptor* build_row_desc,
     const std::vector<TEqJoinCondition>& eq_join_conjuncts,
-    const std::vector<TRuntimeFilterDesc>& filters) {
-  RETURN_IF_ERROR(DataSinkConfig::Init(tsink, build_row_desc, state));
-  join_node_id_ = join_node_id;
-  join_op_ = join_op;
+    const std::vector<TRuntimeFilterDesc>& filters, uint32_t hash_seed,
+    TDataSink* tsink) {
+  tsink->__isset.join_build_sink = true;
+  tsink->join_build_sink.__set_dest_node_id(join_node_id);
+  tsink->join_build_sink.__set_join_op(join_op);
+  RETURN_IF_ERROR(JoinBuilderConfig::Init(*tsink, build_row_desc, state));
+  hash_seed_ = hash_seed;
   return InitExprsAndFilters(state, eq_join_conjuncts, filters);
 }
 
+Status PhjBuilderConfig::Init(
+    const TDataSink& tsink, const RowDescriptor* input_row_desc, RuntimeState* state) {
+  RETURN_IF_ERROR(JoinBuilderConfig::Init(tsink, input_row_desc, state));
+  const TJoinBuildSink& build_sink = tsink.join_build_sink;
+  hash_seed_ = build_sink.hash_seed;
+  resource_profile_ = &tsink.resource_profile;
+  return InitExprsAndFilters(
+      state, tsink.join_build_sink.eq_join_conjuncts, build_sink.runtime_filters);
+}
+
 const char* PhjBuilder::LLVM_CLASS_NAME = "class.impala::PhjBuilder";
 
+PhjBuilder::PhjBuilder(
+    TDataSinkId sink_id, const PhjBuilderConfig& sink_config, RuntimeState* state)
+  : JoinBuilder(
+        sink_id, sink_config, ConstructBuilderName(sink_config.join_node_id_), state),
+    runtime_state_(state),
+    hash_seed_(sink_config.hash_seed_),
+    resource_profile_(sink_config.resource_profile_),
+    reservation_manager_(),
+    buffer_pool_client_(reservation_manager_.buffer_pool_client()),
+    spillable_buffer_size_(resource_profile_->spillable_buffer_size),
+    max_row_buffer_size_(resource_profile_->max_row_buffer_size),
+    build_exprs_(sink_config.build_exprs_),
+    is_not_distinct_from_(sink_config.is_not_distinct_from_),
+    filter_exprs_(sink_config.filter_exprs_) {
+  DCHECK_GT(sink_config.hash_seed_, 0);
+  for (const TRuntimeFilterDesc& filter_desc : sink_config.filter_descs_) {
+    filter_ctxs_.emplace_back();
+    filter_ctxs_.back().filter = state->filter_bank()->RegisterFilter(filter_desc, true);
+  }
+}
+
 PhjBuilder::PhjBuilder(const PhjBuilderConfig& sink_config,
-    BufferPool::ClientHandle* buffer_pool_client, const std::string& join_node_label,
-    int64_t spillable_buffer_size, int64_t max_row_buffer_size, RuntimeState* state)
-  : DataSink(-1, sink_config,
-        Substitute("Hash Join Builder (join_node_id=$0)", sink_config.join_node_id_),
-        state),
+    BufferPool::ClientHandle* buffer_pool_client, int64_t spillable_buffer_size,
+    int64_t max_row_buffer_size, RuntimeState* state)
+  : JoinBuilder(
+        -1, sink_config, ConstructBuilderName(sink_config.join_node_id_), state),
     runtime_state_(state),
-    join_node_id_(sink_config.join_node_id_),
-    join_node_label_(join_node_label),
-    join_op_(sink_config.join_op_),
+    hash_seed_(sink_config.hash_seed_),
+    resource_profile_(nullptr),
+    reservation_manager_(),
     buffer_pool_client_(buffer_pool_client),
     spillable_buffer_size_(spillable_buffer_size),
     max_row_buffer_size_(max_row_buffer_size),
     build_exprs_(sink_config.build_exprs_),
     is_not_distinct_from_(sink_config.is_not_distinct_from_),
     filter_exprs_(sink_config.filter_exprs_) {
+  DCHECK_GT(sink_config.hash_seed_, 0);
   for (const TRuntimeFilterDesc& filter_desc : sink_config.filter_descs_) {
     filter_ctxs_.emplace_back();
     filter_ctxs_.back().filter = state->filter_bank()->RegisterFilter(filter_desc, true);
   }
 }
 
+PhjBuilder::~PhjBuilder() {}
+
 Status PhjBuilder::Prepare(RuntimeState* state, MemTracker* parent_mem_tracker) {
   RETURN_IF_ERROR(DataSink::Prepare(state, parent_mem_tracker));
+  if (is_separate_build_) {
+    const TDebugOptions& instance_debug_options = state->instance_ctx().debug_options;
+    bool debug_option_enabled = instance_debug_options.node_id == -1
+        || instance_debug_options.node_id == join_node_id_;
+    // SET_DENY_RESERVATION_PROBABILITY should behave the same as if it were applied to
+    // the join node.
+    reservation_manager_.Init(Substitute("$0 ptr=$1", name_, this), profile(),
+        state->instance_buffer_reservation(), mem_tracker_.get(), *resource_profile_,
+        debug_option_enabled ? instance_debug_options : TDebugOptions());
+  }
+
   RETURN_IF_ERROR(HashTableCtx::Create(&obj_pool_, state, build_exprs_, build_exprs_,
-      HashTableStoresNulls(), is_not_distinct_from_, state->fragment_hash_seed(),
-      MAX_PARTITION_DEPTH, row_desc_->tuple_descriptors().size(), expr_perm_pool_.get(),
-      expr_results_pool_.get(), expr_results_pool_.get(), &ht_ctx_));
+      HashTableStoresNulls(join_op_, is_not_distinct_from_), is_not_distinct_from_,
+      hash_seed_, MAX_PARTITION_DEPTH, row_desc_->tuple_descriptors().size(),
+      expr_perm_pool_.get(), expr_results_pool_.get(), expr_results_pool_.get(),
+      &ht_ctx_));
 
   DCHECK_EQ(filter_exprs_.size(), filter_ctxs_.size());
   for (int i = 0; i < filter_exprs_.size(); ++i) {
@@ -182,6 +232,12 @@ Status PhjBuilder::Prepare(RuntimeState* state, MemTracker* parent_mem_tracker)
 }
 
 Status PhjBuilder::Open(RuntimeState* state) {
+  SCOPED_TIMER(profile()->total_time_counter());
+  if (!buffer_pool_client_->is_registered()) {
+    DCHECK(is_separate_build_) << "Client is registered by PhjNode if not separate";
+    DCHECK_GE(resource_profile_->min_reservation, MinReservation().second);
+    RETURN_IF_ERROR(reservation_manager_.ClaimBufferReservation(state));
+  }
   // Need to init here instead of constructor so that buffer_pool_client_ is registered.
   if (probe_stream_reservation_.is_closed()) {
     probe_stream_reservation_.Init(buffer_pool_client_);
@@ -207,6 +263,7 @@ Status PhjBuilder::Open(RuntimeState* state) {
 }
 
 Status PhjBuilder::Send(RuntimeState* state, RowBatch* batch) {
+  SCOPED_TIMER(profile()->total_time_counter());
   SCOPED_TIMER(partition_build_rows_timer_);
   bool build_filters = ht_ctx_->level() == 0 && filter_ctxs_.size() > 0;
   if (process_build_batch_fn_ == nullptr) {
@@ -232,6 +289,7 @@ Status PhjBuilder::Send(RuntimeState* state, RowBatch* batch) {
 }
 
 Status PhjBuilder::FlushFinal(RuntimeState* state) {
+  SCOPED_TIMER(profile()->total_time_counter());
   int64_t num_build_rows = 0;
   for (const unique_ptr<Partition>& partition : hash_partitions_) {
     num_build_rows += partition->build_rows()->num_rows();
@@ -275,12 +333,17 @@ Status PhjBuilder::FlushFinal(RuntimeState* state) {
     RETURN_IF_ERROR(null_aware_partition_->Spill(BufferedTupleStream::UNPIN_ALL));
   }
 
-  RETURN_IF_ERROR(BuildHashTablesAndReserveProbeBuffers());
+  HashJoinState next_state;
   if (state_ == HashJoinState::PARTITIONING_BUILD) {
-    UpdateState(HashJoinState::PARTITIONING_PROBE);
+    next_state = HashJoinState::PARTITIONING_PROBE;
   } else {
     DCHECK_ENUM_EQ(state_, HashJoinState::REPARTITIONING_BUILD);
-    UpdateState(HashJoinState::REPARTITIONING_PROBE);
+    next_state = HashJoinState::REPARTITIONING_PROBE;
+  }
+  RETURN_IF_ERROR(BuildHashTablesAndReserveProbeBuffers(next_state));
+  UpdateState(next_state);
+  if (state_ == HashJoinState::PARTITIONING_PROBE && is_separate_build_) {
+    HandoffToProbesAndWait(state);
   }
   return Status::OK();
 }
@@ -300,11 +363,13 @@ void PhjBuilder::Close(RuntimeState* state) {
   ScalarExpr::Close(build_exprs_);
   obj_pool_.Clear();
   probe_stream_reservation_.Close();
+  if (is_separate_build_) reservation_manager_.Close(state);
   DataSink::Close(state);
   closed_ = true;
 }
 
 void PhjBuilder::Reset(RowBatch* row_batch) {
+  DCHECK(!is_separate_build_);
   DCHECK_EQ(0, probe_stream_reservation_.GetReservation());
   state_ = HashJoinState::PARTITIONING_BUILD;
   expr_results_pool_->Clear();
@@ -356,7 +421,7 @@ string PhjBuilder::PrintState() const {
 Status PhjBuilder::CreateAndPreparePartition(
     int level, unique_ptr<Partition>* partition) {
   *partition = make_unique<Partition>(runtime_state_, this, level);
-  Status status = (*partition)->build_rows()->Init(join_node_label_, true);
+  Status status = (*partition)->build_rows()->Init(name_, true);
   if (!status.ok()) goto error;
   bool got_buffer;
   status = (*partition)->build_rows()->PrepareForWrite(&got_buffer);
@@ -452,7 +517,7 @@ Status PhjBuilder::SpillPartition(BufferedTupleStream::UnpinMode mode,
 // For now, we go with a greedy solution.
 //
 // TODO: implement the knapsack solution.
-Status PhjBuilder::BuildHashTablesAndReserveProbeBuffers() {
+Status PhjBuilder::BuildHashTablesAndReserveProbeBuffers(HashJoinState next_state) {
   DCHECK_EQ(PARTITION_FANOUT, hash_partitions_.size());
 
   for (int i = 0; i < PARTITION_FANOUT; ++i) {
@@ -476,8 +541,7 @@ Status PhjBuilder::BuildHashTablesAndReserveProbeBuffers() {
   // building hash tables because allocating probe buffers may cause some more partitions
   // to be spilled. This avoids wasted work on building hash tables for partitions that
   // won't fit in memory alongside the required probe buffers.
-  bool input_is_spilled = ht_ctx_->level() > 0;
-  RETURN_IF_ERROR(ReserveProbeBuffers(input_is_spilled));
+  RETURN_IF_ERROR(ReserveProbeBuffers(next_state));
 
   for (int i = 0; i < PARTITION_FANOUT; ++i) {
     Partition* partition = hash_partitions_[i].get();
@@ -492,19 +556,18 @@ Status PhjBuilder::BuildHashTablesAndReserveProbeBuffers() {
   }
   // We may have spilled additional partitions while building hash tables, we need to
   // reserve memory for the probe buffers for those additional spilled partitions.
-  RETURN_IF_ERROR(ReserveProbeBuffers(input_is_spilled));
+  RETURN_IF_ERROR(ReserveProbeBuffers(next_state));
+  if (is_separate_build_) {
+    // The builder may have some surplus reservation. Release it so that it can be
+    // used by the probe side or by other operators.
+    RETURN_IF_ERROR(reservation_manager_.ReleaseUnusedReservation());
+  }
   return Status::OK();
 }
 
-Status PhjBuilder::ReserveProbeBuffers(bool input_is_spilled) {
+Status PhjBuilder::ReserveProbeBuffers(HashJoinState next_state) {
   DCHECK_EQ(PARTITION_FANOUT, hash_partitions_.size());
-
-  // We need a write buffer for probe rows for each spilled partition, and a read buffer
-  // if the input is a spilled partition (i.e. that we are repartitioning the input).
-  int num_probe_streams =
-      GetNumSpilledPartitions(hash_partitions_) + (input_is_spilled ? 1 : 0);
-  int64_t per_stream_reservation = spillable_buffer_size_;
-  int64_t addtl_reservation = num_probe_streams * per_stream_reservation
+  int64_t addtl_reservation = CalcProbeStreamReservation(next_state)
       - probe_stream_reservation_.GetReservation();
 
   // Loop until either we get enough reservation or all partitions are spilled (in which
@@ -515,32 +578,67 @@ Status PhjBuilder::ReserveProbeBuffers(bool input_is_spilled) {
           BufferedTupleStream::UNPIN_ALL, &spilled_partition));
     // Don't need to create a probe stream for the null-aware partition.
     if (spilled_partition != null_aware_partition_.get()) {
-      addtl_reservation += per_stream_reservation;
+      addtl_reservation += spillable_buffer_size_;
     }
   }
+  VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") saved reservation "
+          << addtl_reservation;
   buffer_pool_client_->SaveReservation(&probe_stream_reservation_, addtl_reservation);
   return Status::OK();
 }
 
-PhjBuilder::HashPartitions PhjBuilder::BeginInitialProbe(
-    BufferPool::ClientHandle* probe_client) {
+Status PhjBuilder::BeginInitialProbe(
+    BufferPool::ClientHandle* probe_client, HashPartitions* partitions) {
+  DCHECK_EQ(is_separate_build_, probe_client != buffer_pool_client_);
   DCHECK_ENUM_EQ(state_, HashJoinState::PARTITIONING_PROBE);
   DCHECK_EQ(PARTITION_FANOUT, hash_partitions_.size());
-  TransferProbeStreamReservation(probe_client);
-  return HashPartitions(ht_ctx_->level(), &hash_partitions_, non_empty_build_);
+  RETURN_IF_ERROR(TransferProbeStreamReservation(probe_client));
+  *partitions = HashPartitions(ht_ctx_->level(), &hash_partitions_, non_empty_build_);
+  return Status::OK();
 }
 
-void PhjBuilder::TransferProbeStreamReservation(BufferPool::ClientHandle* probe_client) {
-  // An extra buffer is needed for reading spilled input stream, unless we're doing the
-  // initial partitioning of rows from the left child.
-  int num_buffers = GetNumSpilledPartitions(hash_partitions_)
-      + (state_ == HashJoinState::PARTITIONING_PROBE ? 0 : 1);
+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
+          << " to probe client.";
+  if (probe_reservation == 0) return Status::OK();
   int64_t saved_reservation = probe_stream_reservation_.GetReservation();
-  DCHECK_GE(saved_reservation, spillable_buffer_size_ * num_buffers);
+  DCHECK_GE(saved_reservation, probe_reservation);
+
+  buffer_pool_client_->RestoreReservation(&probe_stream_reservation_, probe_reservation);
+  if (is_separate_build_) {
+    bool success;
+    RETURN_IF_ERROR(buffer_pool_client_->TransferReservationTo(
+          probe_client, probe_reservation, &success));
+    DCHECK(success) << "Transferring within query shouldn't violate reservation limits.";
+  }
+  return Status::OK();
+}
 
-  // TODO: in future we may need to support different clients for the probe.
-  DCHECK_EQ(probe_client, buffer_pool_client_);
-  probe_client->RestoreReservation(&probe_stream_reservation_, saved_reservation);
+int64_t PhjBuilder::CalcProbeStreamReservation(HashJoinState next_state) const {
+  // We need a read buffer if the input is a spilled partition (i.e. we are repartitioning
+  // the input).
+  bool need_probe_buffer;
+  if (next_state == HashJoinState::PARTITIONING_PROBE) {
+    need_probe_buffer = false;
+  } else {
+    DCHECK(next_state == HashJoinState::PROBING_SPILLED_PARTITION
+        || next_state == HashJoinState::REPARTITIONING_PROBE)
+      << static_cast<int>(next_state);
+    DCHECK_GT(spilled_partitions_.size(), 0);
+    need_probe_buffer = spilled_partitions_.back()->num_spilled_probe_rows() > 0;
+  }
+  DCHECK(next_state == HashJoinState::PROBING_SPILLED_PARTITION
+      || hash_partitions_.size() > 0);
+  int num_spilled_partitions = GetNumSpilledPartitions(hash_partitions_);
+  int num_buffers = num_spilled_partitions + (need_probe_buffer ? 1 : 0);
+  int num_max_sized_buffers =
+      (num_spilled_partitions > 0 ? 1 : 0) + (need_probe_buffer ? 1 : 0);
+  DCHECK_LE(num_max_sized_buffers, num_buffers);
+  return num_max_sized_buffers * max_row_buffer_size_ +
+         (num_buffers - num_max_sized_buffers) * spillable_buffer_size_;
 }
 
 int PhjBuilder::GetNumSpilledPartitions(const vector<unique_ptr<Partition>>& partitions) {
@@ -553,10 +651,17 @@ int PhjBuilder::GetNumSpilledPartitions(const vector<unique_ptr<Partition>>& par
   return num_spilled;
 }
 
-void PhjBuilder::DoneProbingHashPartitions(
+Status PhjBuilder::DoneProbingHashPartitions(
     const int64_t num_spilled_probe_rows[PARTITION_FANOUT],
+    BufferPool::ClientHandle* probe_client,
     deque<unique_ptr<Partition>>* output_partitions, RowBatch* batch) {
+  DCHECK_EQ(is_separate_build_, probe_client != buffer_pool_client_);
   DCHECK(output_partitions->empty());
+  VLOG(3) << "PHJ(node_id=" << join_node_id_ << ") done probing hash partitions.";
+  // Calculate the reservation before cleaning up 'hash_partitions_' and
+  // 'spilled_partitions_', which the calculation depends on.
+  int64_t probe_reservation = CalcProbeStreamReservation(state_);
+  DCHECK_GE(probe_client->GetUnusedReservation(), probe_reservation);
   if (state_ == HashJoinState::REPARTITIONING_PROBE) {
     // Finished repartitioning this partition. Discard before pushing more spilled
     // partitions onto 'spilled_partitions_'.
@@ -586,11 +691,22 @@ void PhjBuilder::DoneProbingHashPartitions(
       partition->Close(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();
 }
 
-void PhjBuilder::DoneProbingSinglePartition(
-      deque<unique_ptr<Partition>>* output_partitions, RowBatch* batch) {
+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_);
   if (NeedToProcessUnmatchedBuildRows(join_op_)) {
     // 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
@@ -601,6 +717,14 @@ void PhjBuilder::DoneProbingSinglePartition(
     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) {
@@ -639,6 +763,8 @@ void PhjBuilder::InsertRuntimeFilters(TupleRow* build_row) noexcept {
 }
 
 void PhjBuilder::PublishRuntimeFilters(int64_t num_build_rows) {
+  VLOG(3) << "Join builder (join_node_id_=" << join_node_id_ << ") publishing "
+          << filter_ctxs_.size() << " filters.";
   int32_t num_enabled_filters = 0;
   // Use 'num_build_rows' to estimate FP-rate of each Bloom filter, and publish
   // 'always-true' filters if it's too high. Doing so saves CPU at the coordinator,
@@ -685,6 +811,7 @@ void PhjBuilder::PublishRuntimeFilters(int64_t num_build_rows) {
 Status PhjBuilder::BeginSpilledProbe(
     BufferPool::ClientHandle* probe_client, bool* repartitioned,
      Partition** input_partition, HashPartitions* new_partitions) {
+  DCHECK_EQ(is_separate_build_, probe_client != buffer_pool_client_);
   DCHECK(!spilled_partitions_.empty());
   DCHECK_EQ(0, hash_partitions_.size());
   // Pick the next spilled partition to process. The partition will stay in
@@ -697,7 +824,7 @@ Status PhjBuilder::BeginSpilledProbe(
 
   if (partition->num_spilled_probe_rows() == 0) {
     // If there are no probe rows, there's no need to build the hash table, and
-    // only partitions with NeedToProcessUnmatcheBuildRows() will have been added
+    // only partitions with NeedToProcessUnmatchedBuildRows() will have been added
     // to 'spilled_partitions_' in DoneProbingHashPartitions().
     DCHECK(NeedToProcessUnmatchedBuildRows(join_op_));
     bool got_read_buffer = false;
@@ -715,15 +842,15 @@ Status PhjBuilder::BeginSpilledProbe(
   // Set aside memory required for reading the probe stream, so that we don't use
   // it for the hash table.
   buffer_pool_client_->SaveReservation(
-      &probe_stream_reservation_, spillable_buffer_size_);
+      &probe_stream_reservation_, max_row_buffer_size_);
 
   // Try to build a hash table for the spilled build partition.
   bool built;
   RETURN_IF_ERROR(partition->BuildHashTable(&built));
   if (built) {
-    TransferProbeStreamReservation(probe_client);
     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.
@@ -740,7 +867,7 @@ 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_, spillable_buffer_size_);
+      &probe_stream_reservation_, max_row_buffer_size_);
   // All reservation should be available for repartitioning.
   DCHECK_EQ(0, probe_stream_reservation_.GetReservation());
   DCHECK_EQ(0, buffer_pool_client_->GetUsedReservation());
@@ -758,7 +885,7 @@ Status PhjBuilder::BeginSpilledProbe(
         next_partition_level, num_input_rows, DebugString(),
         buffer_pool_client_->DebugString());
   }
-  TransferProbeStreamReservation(probe_client);
+  RETURN_IF_ERROR(TransferProbeStreamReservation(probe_client));
   *repartitioned = true;
   *new_partitions = HashPartitions(ht_ctx_->level(), &hash_partitions_, non_empty_build_);
   return Status::OK();
@@ -812,11 +939,23 @@ int64_t PhjBuilder::LargestPartitionRows() const {
   return max_rows;
 }
 
-bool PhjBuilder::HashTableStoresNulls() const {
-  return join_op_ == TJoinOp::RIGHT_OUTER_JOIN || join_op_ == TJoinOp::RIGHT_ANTI_JOIN
-      || join_op_ == TJoinOp::FULL_OUTER_JOIN
-      || std::accumulate(is_not_distinct_from_.begin(), is_not_distinct_from_.end(),
-             false, std::logical_or<bool>());
+bool PhjBuilder::HashTableStoresNulls(
+    TJoinOp::type join_op, const vector<bool>& is_not_distinct_from) {
+  return join_op == TJoinOp::RIGHT_OUTER_JOIN || join_op == TJoinOp::RIGHT_ANTI_JOIN
+      || join_op == TJoinOp::FULL_OUTER_JOIN
+      || std::accumulate(is_not_distinct_from.begin(), is_not_distinct_from.end(), false,
+             std::logical_or<bool>());
+}
+
+void PhjBuilder::ReturnReservation(
+    BufferPool::ClientHandle* probe_client, int64_t bytes) {
+  DCHECK(is_separate_build_);
+  DCHECK(buffer_pool_client_ != probe_client);
+  bool success;
+  Status status =
+      probe_client->TransferReservationTo(buffer_pool_client_, bytes, &success);
+  DCHECK(status.ok()) << status.GetDetail() << " shouldn't have any dirty pages to flush";
+  DCHECK(success) << "Transferring within query shouldn't violate reservation limits.";
 }
 
 PhjBuilder::Partition::Partition(RuntimeState* state, PhjBuilder* parent, int level)
@@ -984,8 +1123,8 @@ void PhjBuilder::Codegen(LlvmCodeGen* codegen) {
   Status codegen_status;
 
   // Context required to generate hash table codegened methods.
-  HashTableConfig hash_table_config(
-      build_exprs_, build_exprs_, HashTableStoresNulls(), is_not_distinct_from_);
+  HashTableConfig hash_table_config(build_exprs_, build_exprs_,
+      HashTableStoresNulls(join_op_, is_not_distinct_from_), is_not_distinct_from_);
   // Codegen for hashing rows with the builder's hash table context.
   llvm::Function* hash_fn;
   codegen_status =
diff --git a/be/src/exec/partitioned-hash-join-builder.h b/be/src/exec/partitioned-hash-join-builder.h
index 9e3f27b..204d0b1 100644
--- a/be/src/exec/partitioned-hash-join-builder.h
+++ b/be/src/exec/partitioned-hash-join-builder.h
@@ -20,18 +20,20 @@
 
 #include <deque>
 #include <memory>
+#include <utility>
 #include <vector>
 #include <boost/scoped_ptr.hpp>
 
 #include "common/object-pool.h"
 #include "common/status.h"
-#include "exec/data-sink.h"
 #include "exec/filter-context.h"
 #include "exec/hash-table.h"
+#include "exec/join-builder.h"
 #include "exec/join-op.h"
 #include "runtime/buffered-tuple-stream.h"
 #include "runtime/bufferpool/buffer-pool.h"
 #include "runtime/bufferpool/suballocator.h"
+#include "runtime/reservation-manager.h"
 
 namespace impala {
 
@@ -45,33 +47,28 @@ class ScalarExprEvaluator;
 /// directly by the PartitionedHashJoinPlanNode. Since it is expected to only be created
 /// and used by PartitionedHashJoinPlanNode only, the DataSinkConfig::Init() and
 /// DataSinkConfig::CreateSink() are not implemented for it.
-class PhjBuilderConfig : public DataSinkConfig {
+class PhjBuilderConfig : public JoinBuilderConfig {
  public:
-    DataSink* CreateSink(const TPlanFragmentCtx& fragment_ctx,
-      const TPlanFragmentInstanceCtx& fragment_instance_ctx,
-      RuntimeState* state) const override;
+  DataSink* CreateSink(const TPlanFragmentCtx& fragment_ctx,
+    const TPlanFragmentInstanceCtx& fragment_instance_ctx,
+    RuntimeState* state) const override;
 
-  /// Creates an instance of PhjBuilder data sink in the state's object pool. To be used
-  /// only by PartitionedHashJoinPlanNode.
+  /// Creates a PhjBuilder for embedded use within a PartitionedHashJoinNode.
   PhjBuilder* CreateSink(BufferPool::ClientHandle* buffer_pool_client,
-      const std::string& join_node_label, int64_t spillable_buffer_size,
-      int64_t max_row_buffer_size, RuntimeState* state) const;
+      int64_t spillable_buffer_size, int64_t max_row_buffer_size,
+      RuntimeState* state) const;
 
-  /// Creates an instance of this class in the state's object pool. To be used only by
+  /// Creates a PhjBuilderConfig for embedded use within a PartitionedHashJoinNode.
+  /// Creates the object in the state's object pool. To be used only by
   /// PartitionedHashJoinPlanNode.
   static Status CreateConfig(RuntimeState* state, int join_node_id, TJoinOp::type join_op,
       const RowDescriptor* build_row_desc,
       const std::vector<TEqJoinCondition>& eq_join_conjuncts,
-      const std::vector<TRuntimeFilterDesc>& filters, const PhjBuilderConfig** sink);
+      const std::vector<TRuntimeFilterDesc>& filters, uint32_t hash_seed,
+      const PhjBuilderConfig** sink);
 
   ~PhjBuilderConfig() override {}
 
-  /// The ID of the plan join node this is associated with.
-  int join_node_id_;
-
-  /// The join operation this is building for.
-  TJoinOp::type join_op_;
-
   /// Expressions over input rows for hash table build.
   std::vector<ScalarExpr*> build_exprs_;
 
@@ -87,16 +84,27 @@ class PhjBuilderConfig : public DataSinkConfig {
   /// The runtime filter descriptors of filters produced by this builder.
   vector<TRuntimeFilterDesc> filter_descs_;
 
+  /// Seed used for hashing rows. Must match seed used in the PartitionedHashJoinNode.
+  uint32_t hash_seed_;
+
+  /// Resource information sent from the frontend. Non-null if this is a separate join
+  /// build.
+  const TBackendResourceProfile* resource_profile_ = nullptr;
+
  protected:
+  /// Initialization for separate sink.
   Status Init(const TDataSink& tsink, const RowDescriptor* input_row_desc,
       RuntimeState* state) override;
 
  private:
-  /// Helper method used by CreateConfig()
-  Status Init(RuntimeState* state, const TDataSink& tsink, int join_node_id,
-      TJoinOp::type join_op, const RowDescriptor* build_row_desc,
+  /// Helper method used by CreateConfig() to initialize embedded builder.
+  /// 'tsink' does not need to be initialized by the caller - all values to be used are
+  /// passed in as arguments and this function fills in required fields in 'tsink'.
+  Status Init(RuntimeState* state, int join_node_id, TJoinOp::type join_op,
+      const RowDescriptor* build_row_desc,
       const std::vector<TEqJoinCondition>& eq_join_conjuncts,
-      const std::vector<TRuntimeFilterDesc>& filters);
+      const std::vector<TRuntimeFilterDesc>& filters, uint32_t hash_seed,
+      TDataSink* tsink);
 
   /// Initializes the build and filter expressions and creates a copy of the filter
   /// descriptors that will be generated by this sink.
@@ -142,21 +150,25 @@ enum class HashJoinState {
 /// one write buffer, which is sufficient to spill the partition's probe rows to disk
 /// without allocating additional buffers.
 ///
-/// After this initial partitioning, the join node probes the in-memory hash partitions.
-/// The join node then drives processing of any spilled partitions, calling
-/// Partition::BuildHashTable() to build hash tables for a spilled partition or calling
-/// RepartitionBuildInput() to repartition a level n partition into multiple level n + 1
-/// partitions.
+/// After this initial partitioning, the join node probes the in-memory hash partitions,
+/// after which it calls DoneProbingHashPartitions(). Then the spilling algorithm can
+/// commence, with the join node calling BeginSpilledProbe() and DoneProbing*() methods
+/// until all spilled partitions are processed.
 ///
-/// Both the PartitionedHashJoinNode and the builder share a BufferPool client
-/// and the corresponding reservations. Different stages of the spilling algorithm
-/// require different mixes of build and probe buffers and hash tables, so we can
-/// share the reservation to minimize the combined memory requirement. Memory for
-/// probe-side buffers is reserved in the builder then handed off to the probe side
-/// to implement this reservation sharing.
+/// Both the PartitionedHashJoinNode and the builder share memory reservation. Different
+/// stages of the spilling algorithm require different mixes of build and probe buffers
+/// and hash tables, so we can share the reservation to minimize the combined memory
+/// requirement. Memory for probe-side buffers is reserved in the builder then handed
+/// off to the probe side to implement this reservation sharing. When the builder is
+/// integrated into the join node, this is implemented with a shared BufferPool client.
+/// When the build is separate, reservation is transferred between the builder's and the
+/// join node's clients as needed. The probe client is passed into various methods as
+/// 'probe_client'. If the join is integrated, 'probe_client' must be the same client
+/// as was passed into the constructor. If the join is separate, 'probe_client' must
+/// be a different client.
 ///
 /// The full hash join algorithm is documented in PartitionedHashJoinNode.
-class PhjBuilder : public DataSink {
+class PhjBuilder : public JoinBuilder {
  public:
   /// Number of initial partitions to create. Must be a power of two.
   static const int PARTITION_FANOUT = 16;
@@ -179,9 +191,16 @@ class PhjBuilder : public DataSink {
 
   using PartitionId = int;
 
+  // Constructor for separate join build.
+  PhjBuilder(
+      TDataSinkId sink_id, const PhjBuilderConfig& sink_config, RuntimeState* state);
+  // Constructor for join builder embedded in a PartitionedHashJoinNode. Shares
+  // 'buffer_pool_client' with the parent node and inherits buffer sizes from
+  // the parent node.
   PhjBuilder(const PhjBuilderConfig& sink_config,
-      BufferPool::ClientHandle* buffer_pool_client, const std::string& join_node_label,
-      int64_t spillable_buffer_size, int64_t max_row_buffer_size, RuntimeState* state);
+      BufferPool::ClientHandle* buffer_pool_client, int64_t spillable_buffer_size,
+      int64_t max_row_buffer_size, RuntimeState* state);
+  ~PhjBuilder();
 
   /// Implementations of DataSink interface methods.
   virtual Status Prepare(RuntimeState* state, MemTracker* parent_mem_tracker) override;
@@ -200,6 +219,7 @@ class PhjBuilder : public DataSink {
   /////////////////////////////////////////
 
   /// Reset the builder to the same state as it was in after calling Open().
+  /// Not valid to call on a separate join build.
   void Reset(RowBatch* row_batch);
 
   /// Represents a set of hash partitions to be handed off to the probe side.
@@ -231,13 +251,14 @@ class PhjBuilder : public DataSink {
     bool non_empty_build;
   };
 
-  /// Get hash partitions and reservation for the initial partitionining of the probe
-  /// side. Only valid to call once when in state PARTITIONING_PROBE.
+  /// Get hash partitions and reservation for the initial partitioning of the probe
+  /// side. Only valid to call once per PartitionedHashJoinNode when in state
+  /// PARTITIONING_PROBE (i.e. once if the build is not shared).
   /// When this function returns successfully, 'probe_client' will have enough
   /// reservation for a write buffer for each spilled partition.
-  /// Return the current set of hash partitions.
-  /// TODO: IMPALA-9156: this will be a synchronization point for shared join build.
-  HashPartitions BeginInitialProbe(BufferPool::ClientHandle* probe_client);
+  /// Return the current set of hash partitions in 'partitions'.
+  Status BeginInitialProbe(
+      BufferPool::ClientHandle* probe_client, HashPartitions* partitions);
 
   /// Pick a spilled partition to process (returned in *input_partition) and
   /// prepare to probe it. Builds a hash table over *input_partition
@@ -257,7 +278,7 @@ class PhjBuilder : public DataSink {
       Partition** input_partition, HashPartitions* new_partitions);
 
   /// Called after probing of the hash partitions returned by BeginInitialProbe() or
-  /// BeginSpilledProbe() (when *repartitioned as true) is complete, i.e. all of the
+  /// BeginSpilledProbe() (when *repartitioned is true) is complete, i.e. all of the
   /// corresponding probe rows have been processed by PartitionedHashJoinNode. The number
   /// of spilled probe rows per partition must be passed in via 'num_spilled_probe_rows'
   /// so that the builder can determine whether a spilled partition needs to be retained.
@@ -265,8 +286,14 @@ class PhjBuilder : public DataSink {
   /// 'output_partitions' for build modes like right outer join that output unmatched
   /// rows. Close other in-memory partitions, attaching any tuple data to 'batch' if
   /// 'batch' is non-NULL. Closes spilled partitions if no more processing is needed.
+  ///
+  /// The reservation that was transferred to 'probe_client' in Begin*Probe() is
+  /// 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.
-  void DoneProbingHashPartitions(const int64_t num_spilled_probe_rows[PARTITION_FANOUT],
+  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);
 
   /// Called after probing of a single spilled partition returned by
@@ -280,8 +307,13 @@ class PhjBuilder : public DataSink {
   ///
   /// If no build rows need to be returned, closes the build partition and attaches any
   /// tuple data to 'batch' if 'batch' is non-NULL.
+  ///
+  /// The reservation that was transferred to 'probe_client' in Begin*Probe() is
+  /// 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.
-  void DoneProbingSinglePartition(
+  Status DoneProbingSinglePartition(BufferPool::ClientHandle* probe_client,
       std::deque<std::unique_ptr<Partition>>* output_partitions, RowBatch* batch);
 
   /// Close the null aware partition (if there is one) and set it to NULL.
@@ -297,21 +329,26 @@ class PhjBuilder : public DataSink {
   }
 
   /// True if the hash table may contain rows with one or more NULL join keys. This
-  /// depends on the join type and the equijoin conjuncts.
-  /// Valid to call after InitExprsAndFilters(). Thread-safe.
-  bool HashTableStoresNulls() const;
+  /// depends on the join type, passed in via 'join_op' and the 'is_not_distinct_from'
+  /// flags of the equijoin conjuncts, which are passed in via 'is_not_distinct_from'.
+  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
+  /// have some surplus reservation.
+  /// 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.
   HashJoinState state() const { return state_; }
 
-  /// Valid to call after InitExprsAndFilters(). Thread-safe.
-  inline const std::vector<bool>& is_not_distinct_from() const {
-    return is_not_distinct_from_;
-  }
-
   /// 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(); }
@@ -422,19 +459,31 @@ class PhjBuilder : public DataSink {
 
   /// Computes the minimum reservation required to execute the spilling partitioned
   /// hash algorithm successfully for any input size (assuming enough disk space is
-  /// available for spilled rows). The buffers are used for buffering both build and
-  /// probe rows at different times, so the total requirement is the peak sum of build
-  /// and probe buffers required.
+  /// available for spilled rows). This includes buffers used by the build side,
+  /// the probe side, and buffers that are shared between build and probe.
   /// We need one output buffer per partition to partition the build or probe side. We
   /// need one additional buffer for the input while repartitioning the build or probe.
-  /// For NAAJ, we need 3 additional buffers for 'null_aware_partition_',
-  /// 'null_aware_probe_partition_' and 'null_probe_rows_'.
-  int64_t MinReservation() const {
+  /// For NAAJ, we need an additional buffer for 'null_aware_partition_' on the build
+  /// side and two additional buffers for 'null_aware_probe_partition_' and
+  /// 'null_probe_rows_' on the probe side.
+  /// Returns a pair with the probe and build reservation requirements.
+  std::pair<int64_t, int64_t> MinReservation() const {
     // Must be kept in sync with HashJoinNode.computeNodeResourceProfile() in fe.
-    int num_reserved_buffers = PARTITION_FANOUT + 1;
-    if (join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) num_reserved_buffers += 3;
-    // Two of the buffers must fit the maximum row.
-    return spillable_buffer_size_ * (num_reserved_buffers - 2) + max_row_buffer_size_ * 2;
+    int num_reserved_build_buffers = PARTITION_FANOUT + 1;
+    int64_t probe_reservation = 0;
+    if (join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) {
+      num_reserved_build_buffers += 1;
+      // One of the NAAJ buffers needs to fit the max row, since we write/read
+      // one stream at a time. If the build is integrated, we already have a max-sized
+      // buffer accounted for in the build reservation.
+      probe_reservation = is_separate_build_ ?
+        max_row_buffer_size_ + spillable_buffer_size_ : spillable_buffer_size_ * 2;
+    }
+    // Two of the build buffers must fit the maximum row for use as read and write
+    // buffers while repartitioning a stream.
+    return {probe_reservation,
+        spillable_buffer_size_ * (num_reserved_build_buffers - 2)
+            + max_row_buffer_size_ * 2};
   }
 
   /// Class name in LLVM IR.
@@ -501,14 +550,18 @@ class PhjBuilder : public DataSink {
   ///       probe stream memory is reserved for this partition.
   /// 3. spilled. The build rows are fully unpinned and the probe stream is prepared.
   ///       Memory for a probe stream write buffer is reserved for this partition.
-  Status BuildHashTablesAndReserveProbeBuffers() WARN_UNUSED_RESULT;
+  ///
+  /// 'next_state' is the state that this will transition into after building the hash
+  /// tables, either PARTITIONING_PROBE or REPARTITIONING_PROBE.
+  Status BuildHashTablesAndReserveProbeBuffers(HashJoinState next_state);
 
   /// 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 (indicated by input_is_spilled). 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(bool input_is_spilled) WARN_UNUSED_RESULT;
+  /// is a spilled partition (determined by 'next_state' - either PARTITIONING_PROBE or
+  /// REPARTITIONING_PROBE). 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);
 
   /// Returns the number of partitions in 'partitions' that are spilled.
   static int GetNumSpilledPartitions(
@@ -517,7 +570,11 @@ class PhjBuilder : public DataSink {
   /// 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.
-  void TransferProbeStreamReservation(BufferPool::ClientHandle* probe_client);
+  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_'.
+  int64_t CalcProbeStreamReservation(HashJoinState state) const;
 
   /// Creates new hash partitions and repartitions 'input_partition' into PARTITION_FANOUT
   /// new partitions with level input_partition->level() + 1. The previous hash partitions
@@ -568,25 +625,30 @@ class PhjBuilder : public DataSink {
 
   RuntimeState* const runtime_state_;
 
-  /// The ID of the plan join node this is associated with.
-  const int join_node_id_;
-
-  /// The label of the plan join node this is associated with.
-  const std::string join_node_label_;
-
-  /// The join operation this is building for.
-  const TJoinOp::type join_op_;
+  /// Seed used for hashing rows. Must match seed used in the PartitionedHashJoinNode.
+  const uint32_t hash_seed_;
 
   /// Pool for objects with same lifetime as builder.
   ObjectPool obj_pool_;
 
+  /// Resource information sent from the frontend. Non-null if this is a separate join
+  /// build.
+  const TBackendResourceProfile* const resource_profile_;
+
+  /// Wraps the buffer pool client. Only used if this is a separate build sink. The node's
+  /// minimum reservation is claimed in Open(). After this, the client must hold onto
+  /// at least the minimum reservation so that it can be returned to the initial
+  /// reservations pool in Close().
+  ReservationManager reservation_manager_;
+
   /// Client to the buffer pool, used to allocate build partition buffers and hash tables.
   /// When probing, the spilling algorithm keeps some build partitions in memory while
-  /// using memory for probe buffers for spilled partitions. To support dynamically
-  /// dividing memory between build and probe, this client is shared between the builder
-  /// and the PartitionedHashJoinNode.
-  /// TODO: this approach to sharing will not work for spilling broadcast joins with a
-  /// 1:N relationship from builders to join nodes.
+  /// using memory for probe buffers for spilled partitions.
+  /// Memory is shared between build and probe in different ways, depending on whether
+  /// this is a separate join build (i.e. 'is_separate_build_' is true). If a separate
+  /// build, this builder has its own buffer pool client, and transfer reservation to
+  /// the probe client when needed. If the builder is embedded in the join node, this
+  /// is just a pointer to the join node's client so no transfer is required.
   BufferPool::ClientHandle* buffer_pool_client_;
 
   /// The default and max buffer sizes to use in the build streams.
@@ -601,7 +663,6 @@ class PhjBuilder : public DataSink {
 
   /// is_not_distinct_from_[i] is true if and only if the ith equi-join predicate is IS
   /// NOT DISTINCT FROM, rather than equality.
-  /// Set in InitExprsAndFilters() and constant thereafter.
   std::vector<bool> is_not_distinct_from_;
 
   /// Expressions for evaluating input rows for insertion into runtime filters.
@@ -726,6 +787,5 @@ class PhjBuilder : public DataSink {
   InsertBatchFn insert_batch_fn_ = nullptr;
   InsertBatchFn insert_batch_fn_level0_ = nullptr;
 };
-}
-
+} // namespace impala
 #endif
diff --git a/be/src/exec/partitioned-hash-join-node.cc b/be/src/exec/partitioned-hash-join-node.cc
index 9e643b0..e131d8b 100644
--- a/be/src/exec/partitioned-hash-join-node.cc
+++ b/be/src/exec/partitioned-hash-join-node.cc
@@ -50,36 +50,39 @@ using strings::Substitute;
 
 Status PartitionedHashJoinPlanNode::Init(const TPlanNode& tnode, RuntimeState* state) {
   RETURN_IF_ERROR(BlockingJoinPlanNode::Init(tnode, state));
-  DCHECK(tnode.__isset.hash_join_node);
+  DCHECK(tnode.__isset.join_node);
+  DCHECK(tnode.join_node.__isset.hash_join_node);
   const vector<TEqJoinCondition>& eq_join_conjuncts =
-      tnode.hash_join_node.eq_join_conjuncts;
-  // TODO: change PhjBuilder::InitExprsAndFilters to accept the runtime filter contexts
-  // and build_exprs_ generated here in init and not create its own. Then pass those in
-  // during Prepare phase.
-
+      tnode.join_node.hash_join_node.eq_join_conjuncts;
   for (const TEqJoinCondition& eq_join_conjunct : eq_join_conjuncts) {
     ScalarExpr* probe_expr;
     RETURN_IF_ERROR(ScalarExpr::Create(
-        eq_join_conjunct.left,*children_[0]->row_descriptor_, state, &probe_expr));
+        eq_join_conjunct.left, probe_row_desc(), state, &probe_expr));
     probe_exprs_.push_back(probe_expr);
     ScalarExpr* build_expr;
     RETURN_IF_ERROR(ScalarExpr::Create(
-        eq_join_conjunct.right, *children_[1]->row_descriptor_, state, &build_expr));
+        eq_join_conjunct.right, build_row_desc(), state, &build_expr));
     build_exprs_.push_back(build_expr);
+    is_not_distinct_from_.push_back(eq_join_conjunct.is_not_distinct_from);
   }
+
   // other_join_conjuncts_ are evaluated in the context of rows assembled from all build
   // and probe tuples; full_row_desc is not necessarily the same as the output row desc,
   // e.g., because semi joins only return the build xor probe tuples
-  RowDescriptor full_row_desc(
-      *children_[0]->row_descriptor_, *children_[1]->row_descriptor_);
-  RETURN_IF_ERROR(ScalarExpr::Create(tnode.hash_join_node.other_join_conjuncts,
+  RowDescriptor full_row_desc(probe_row_desc(), build_row_desc());
+  RETURN_IF_ERROR(ScalarExpr::Create(tnode.join_node.hash_join_node.other_join_conjuncts,
       full_row_desc, state, &other_join_conjuncts_));
-  DCHECK(tnode.hash_join_node.join_op != TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN
+  DCHECK(tnode.join_node.join_op != TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN
       || eq_join_conjuncts.size() == 1);
+  hash_seed_ = tnode.join_node.hash_join_node.hash_seed;
 
+  // Create the config always. It is only used if UseSeparateBuild() is true, but in
+  // Init(), IsInSubplan() isn't available yet.
+  // TODO: simplify this by ensuring that UseSeparateBuild() is accurate in Init().
   RETURN_IF_ERROR(PhjBuilderConfig::CreateConfig(state, tnode_->node_id,
-      tnode_->hash_join_node.join_op, children_[1]->row_descriptor_, eq_join_conjuncts,
-      tnode_->runtime_filters, &phj_builder_config));
+      tnode_->join_node.join_op, &build_row_desc(), eq_join_conjuncts,
+      tnode_->runtime_filters, tnode_->join_node.hash_join_node.hash_seed,
+      &phj_builder_config));
   return Status::OK();
 }
 
@@ -92,18 +95,12 @@ Status PartitionedHashJoinPlanNode::CreateExecNode(
 
 PartitionedHashJoinNode::PartitionedHashJoinNode(RuntimeState* state,
     const PartitionedHashJoinPlanNode& pnode, const DescriptorTbl& descs)
-  : BlockingJoinNode("PartitionedHashJoinNode", pnode.tnode_->hash_join_node.join_op,
+  : BlockingJoinNode("PartitionedHashJoinNode", pnode.tnode_->join_node.join_op,
         state->obj_pool(), pnode, descs),
     build_exprs_(pnode.build_exprs_),
     probe_exprs_(pnode.probe_exprs_),
     other_join_conjuncts_(pnode.other_join_conjuncts_) {
   memset(hash_tbls_, 0, sizeof(HashTable*) * PARTITION_FANOUT);
-  // TODO: allow PhjBuilder to be the sink of a separate fragment. For now, PhjBuilder is
-  // owned by this node, but duplicates some state (exprs, etc) in anticipation of it
-  // being separated out further.
-  builder_ = pnode.phj_builder_config->CreateSink(buffer_pool_client(), label(),
-      resource_profile_.spillable_buffer_size, resource_profile_.max_row_buffer_size,
-      state);
 }
 
 PartitionedHashJoinNode::~PartitionedHashJoinNode() {
@@ -116,8 +113,15 @@ Status PartitionedHashJoinNode::Prepare(RuntimeState* state) {
 
   RETURN_IF_ERROR(BlockingJoinNode::Prepare(state));
   runtime_state_ = state;
-  RETURN_IF_ERROR(builder_->Prepare(state, mem_tracker()));
-  runtime_profile()->PrependChild(builder_->profile());
+  if (!UseSeparateBuild(state->query_options())) {
+    const PhjBuilderConfig& builder_config =
+      *static_cast<const PartitionedHashJoinPlanNode&>(plan_node_).phj_builder_config;
+    builder_ = builder_config.CreateSink(buffer_pool_client(),
+          resource_profile_.spillable_buffer_size, resource_profile_.max_row_buffer_size,
+          state);
+    RETURN_IF_ERROR(builder_->Prepare(state, mem_tracker()));
+    runtime_profile()->PrependChild(builder_->profile());
+  }
 
   RETURN_IF_ERROR(ScalarExprEvaluator::Create(other_join_conjuncts_, state, pool_,
       expr_perm_pool(), expr_results_pool(), &other_join_conjunct_evals_));
@@ -131,11 +135,13 @@ Status PartitionedHashJoinNode::Prepare(RuntimeState* state) {
   // QueryMaintenance(). Values of probe exprs may need to live longer until the
   // cache is reset so are stored in 'probe_expr_results_pool_', which is cleared
   // manually at the appropriate time.
+  const vector<bool>& is_not_distinct_from =
+      static_cast<const PartitionedHashJoinPlanNode&>(plan_node_).is_not_distinct_from_;
   RETURN_IF_ERROR(HashTableCtx::Create(pool_, state, build_exprs_, probe_exprs_,
-      builder_->HashTableStoresNulls(), builder_->is_not_distinct_from(),
-      state->fragment_hash_seed(), MAX_PARTITION_DEPTH,
-      child(1)->row_desc()->tuple_descriptors().size(), expr_perm_pool(),
-      expr_results_pool(), probe_expr_results_pool_.get(), &ht_ctx_));
+      PhjBuilder::HashTableStoresNulls(join_op_, is_not_distinct_from),
+      is_not_distinct_from, hash_seed(), MAX_PARTITION_DEPTH,
+      build_row_desc().tuple_descriptors().size(), expr_perm_pool(), expr_results_pool(),
+      probe_expr_results_pool_.get(), &ht_ctx_));
   if (join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) {
     null_aware_eval_timer_ = ADD_TIMER(runtime_profile(), "NullAwareAntiJoinEvalTime");
   }
@@ -155,8 +161,11 @@ void PartitionedHashJoinNode::Codegen(RuntimeState* state) {
   LlvmCodeGen* codegen = state->codegen();
   DCHECK(codegen != NULL);
 
-  // Codegen the build side.
-  builder_->Codegen(codegen);
+  // Codegen the build side (if integrated into this join node).
+  if (!UseSeparateBuild(state->query_options())) {
+    DCHECK(builder_ != nullptr);
+    builder_->Codegen(codegen);
+  }
 
   // Codegen the probe side.
   TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
@@ -168,7 +177,13 @@ void PartitionedHashJoinNode::Codegen(RuntimeState* state) {
 Status PartitionedHashJoinNode::Open(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   ScopedOpenEventAdder ea(this);
-  RETURN_IF_ERROR(BlockingJoinNode::Open(state));
+  JoinBuilder* tmp_builder = nullptr;
+  RETURN_IF_ERROR(BlockingJoinNode::OpenImpl(state, &tmp_builder));
+  if (builder_ == nullptr) {
+    DCHECK(UseSeparateBuild(state->query_options()));
+    builder_ = dynamic_cast<PhjBuilder*>(tmp_builder);
+    DCHECK(builder_ != nullptr);
+  }
   RETURN_IF_ERROR(ht_ctx_->Open(state));
   RETURN_IF_ERROR(ScalarExprEvaluator::Open(other_join_conjunct_evals_, state));
 
@@ -183,7 +198,8 @@ Status PartitionedHashJoinNode::Open(RuntimeState* state) {
 
   RETURN_IF_ERROR(BlockingJoinNode::ProcessBuildInputAndOpenProbe(state, builder_));
 
-  build_hash_partitions_ = builder_->BeginInitialProbe(buffer_pool_client());
+  RETURN_IF_ERROR(
+      builder_->BeginInitialProbe(buffer_pool_client(), &build_hash_partitions_));
   RETURN_IF_ERROR(PrepareForPartitionedProbe());
 
   RETURN_IF_ERROR(BlockingJoinNode::GetFirstProbeRow(state));
@@ -195,8 +211,15 @@ Status PartitionedHashJoinNode::Open(RuntimeState* state) {
 }
 
 Status PartitionedHashJoinNode::AcquireResourcesForBuild(RuntimeState* state) {
-  DCHECK_GE(resource_profile_.min_reservation, builder_->MinReservation());
   if (!buffer_pool_client()->is_registered()) {
+    // Ensure the frontend computed enough reservation for this join to execute.
+    pair<int64_t, int64_t> min_reservation = builder_->MinReservation();
+    if (UseSeparateBuild(state->query_options())) {
+      DCHECK_GE(resource_profile_.min_reservation, min_reservation.first);
+    } else {
+      DCHECK_GE(resource_profile_.min_reservation,
+            min_reservation.first + min_reservation.second);
+    }
     RETURN_IF_ERROR(ClaimBufferReservation(state));
   }
   if (join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) {
@@ -254,14 +277,26 @@ void PartitionedHashJoinNode::CloseAndDeletePartitions(RowBatch* row_batch) {
 void PartitionedHashJoinNode::Close(RuntimeState* state) {
   if (is_closed()) return;
   if (ht_ctx_ != nullptr) {
-    ht_ctx_->StatsCountersAdd(builder_->ht_stats_profile());
+    if (builder_ != nullptr) ht_ctx_->StatsCountersAdd(builder_->ht_stats_profile());
     ht_ctx_->Close(state);
     ht_ctx_.reset();
   }
   output_unmatched_batch_.reset();
   output_unmatched_batch_iter_.reset();
   CloseAndDeletePartitions(nullptr);
-  if (builder_ != nullptr) builder_->Close(state);
+  if (builder_ != nullptr) {
+    bool separate_build = UseSeparateBuild(state->query_options());
+    if (!separate_build || waited_for_build_) {
+      if (separate_build
+          && buffer_pool_client()->GetReservation() > resource_profile_.min_reservation) {
+        // Transfer back surplus reservation, which we may have borrowed from 'builder_'.
+        builder_->ReturnReservation(buffer_pool_client(),
+            buffer_pool_client()->GetReservation() - resource_profile_.min_reservation);
+      }
+      builder_->CloseFromProbe(state);
+      waited_for_build_ = false;
+    }
+  }
   ScalarExprEvaluator::Close(other_join_conjunct_evals_, state);
   ScalarExpr::Close(build_exprs_);
   ScalarExpr::Close(probe_exprs_);
@@ -273,10 +308,9 @@ void PartitionedHashJoinNode::Close(RuntimeState* state) {
 PartitionedHashJoinNode::ProbePartition::ProbePartition(RuntimeState* state,
     PartitionedHashJoinNode* parent, PhjBuilder::Partition* build_partition)
   : build_partition_(build_partition),
-    probe_rows_(make_unique<BufferedTupleStream>(state,
-      parent->child(0)->row_desc(), parent->buffer_pool_client(),
-      parent->resource_profile_.spillable_buffer_size,
-      parent->resource_profile_.max_row_buffer_size)) {}
+    probe_rows_(make_unique<BufferedTupleStream>(state, &parent->probe_row_desc(),
+        parent->buffer_pool_client(), parent->resource_profile_.spillable_buffer_size,
+        parent->resource_profile_.max_row_buffer_size)) {}
 
 PartitionedHashJoinNode::ProbePartition::~ProbePartition() {
   DCHECK(IsClosed());
@@ -827,7 +861,7 @@ Status PartitionedHashJoinNode::InitNullAwareProbePartition() {
 Status PartitionedHashJoinNode::InitNullProbeRows() {
   RuntimeState* state = runtime_state_;
   null_probe_rows_ =
-      make_unique<BufferedTupleStream>(state, child(0)->row_desc(), buffer_pool_client(),
+      make_unique<BufferedTupleStream>(state, &probe_row_desc(), buffer_pool_client(),
           resource_profile_.spillable_buffer_size, resource_profile_.max_row_buffer_size);
   // Start with stream pinned, unpin later if needed.
   RETURN_IF_ERROR(null_probe_rows_->Init(label(), true));
@@ -913,7 +947,7 @@ Status PartitionedHashJoinNode::OutputNullAwareProbeRows(
         builder_->null_aware_partition()->build_rows();
     RETURN_IF_ERROR(null_build_stream->PrepareForRead(false, &got_reservation));
     DCHECK(got_reservation) << "Should have been pinned";
-    RowBatch null_build_batch(child(1)->row_desc(), state->batch_size(), mem_tracker());
+    RowBatch null_build_batch(&build_row_desc(), state->batch_size(), mem_tracker());
     bool eos;
     do {
       RETURN_IF_ERROR(null_build_stream->GetNext(&null_build_batch, &eos));
@@ -1054,7 +1088,7 @@ Status PartitionedHashJoinNode::EvaluateNullProbe(
 
   ScalarExprEvaluator* const* join_conjunct_evals = other_join_conjunct_evals_.data();
   int num_join_conjuncts = other_join_conjuncts_.size();
-  RowBatch probe_batch(child(0)->row_desc(), runtime_state_->batch_size(), mem_tracker());
+  RowBatch probe_batch(&probe_row_desc(), runtime_state_->batch_size(), mem_tracker());
 
   bool pinned;
   RETURN_IF_ERROR(build->PinStream(&pinned));
@@ -1073,7 +1107,7 @@ Status PartitionedHashJoinNode::EvaluateNullProbe(
       bool got_reservation;
       RETURN_IF_ERROR(build->PrepareForRead(false, &got_reservation));
       DCHECK(got_reservation) << "Should have been pinned";
-      RowBatch build_batch(child(1)->row_desc(), state->batch_size(), mem_tracker());
+      RowBatch build_batch(&build_row_desc(), state->batch_size(), mem_tracker());
       bool build_eos;
       do {
         RETURN_IF_ERROR(build->GetNext(&build_batch, &build_eos));
@@ -1102,12 +1136,18 @@ Status PartitionedHashJoinNode::DoneProbing(RuntimeState* state, RowBatch* batch
   // At this point all the rows have been read from the probe side for all partitions in
   // hash_partitions_.
   VLOG(2) << "Probe Side Consumed\n" << NodeDebugString();
+  // Clean up input partition first to free up probe reservation before calling
+  // DoneProbing*().
+  if (input_partition_ != nullptr) {
+    input_partition_->Close(batch);
+    input_partition_.reset();
+  }
   if (builder_->state() == HashJoinState::PROBING_SPILLED_PARTITION) {
     // Need to clean up single in-memory build partition instead of hash partitions.
     DCHECK(build_hash_partitions_.hash_partitions == nullptr);
-    DCHECK(input_partition_ != nullptr);
-    builder_->DoneProbingSinglePartition(
-        &output_build_partitions_, IsLeftSemiJoin(join_op_) ? nullptr : batch);
+    RETURN_IF_ERROR(builder_->DoneProbingSinglePartition(
+          buffer_pool_client(), &output_build_partitions_,
+        IsLeftSemiJoin(join_op_) ? nullptr : batch));
   } else {
     // Walk the partitions that had hash tables built for the probe phase and either
     // close them or move them to 'spilled_partitions_'.
@@ -1146,12 +1186,9 @@ Status PartitionedHashJoinNode::DoneProbing(RuntimeState* state, RowBatch* batch
     }
     probe_hash_partitions_.clear();
     build_hash_partitions_.Reset();
-    builder_->DoneProbingHashPartitions(num_spilled_probe_rows,
-        &output_build_partitions_, IsLeftSemiJoin(join_op_) ? nullptr : batch);
-  }
-  if (input_partition_ != nullptr) {
-    input_partition_->Close(batch);
-    input_partition_.reset();
+    RETURN_IF_ERROR(
+        builder_->DoneProbingHashPartitions(num_spilled_probe_rows, buffer_pool_client(),
+          &output_build_partitions_, IsLeftSemiJoin(join_op_) ? nullptr : batch));
   }
   if (!output_build_partitions_.empty()) {
     DCHECK(output_unmatched_batch_iter_.get() == nullptr);
@@ -1160,7 +1197,7 @@ Status PartitionedHashJoinNode::DoneProbing(RuntimeState* state, RowBatch* batch
       hash_tbl_iterator_ = output_partition->hash_tbl()->FirstUnmatched(ht_ctx_.get());
     } else {
       output_unmatched_batch_.reset(new RowBatch(
-          child(1)->row_desc(), runtime_state_->batch_size(), builder_->mem_tracker()));
+          &build_row_desc(), runtime_state_->batch_size(), builder_->mem_tracker()));
       output_unmatched_batch_iter_.reset(
           new RowBatch::Iterator(output_unmatched_batch_.get(), 0));
     }
@@ -1304,8 +1341,8 @@ Status PartitionedHashJoinNode::CodegenCreateOutputRow(
   llvm::Value* build_row_arg =
       builder.CreateBitCast(args[3], tuple_row_working_type, "build");
 
-  int num_probe_tuples = child(0)->row_desc()->tuple_descriptors().size();
-  int num_build_tuples = child(1)->row_desc()->tuple_descriptors().size();
+  int num_probe_tuples = probe_row_desc().tuple_descriptors().size();
+  int num_build_tuples = build_row_desc().tuple_descriptors().size();
 
   // Copy probe row
   codegen->CodegenMemcpy(&builder, out_row_arg, probe_row_arg, probe_tuple_row_size_);
@@ -1362,8 +1399,10 @@ Status PartitionedHashJoinNode::CodegenProcessProbeBatch(
   llvm::Function* hash_fn;
   llvm::Function* murmur_hash_fn;
   // Context required to generate hash table codegened methods.
+  auto& plan_node = static_cast<const PartitionedHashJoinPlanNode&>(plan_node_);
   HashTableConfig hash_table_config(build_exprs_, probe_exprs_,
-      builder_->HashTableStoresNulls(), builder_->is_not_distinct_from());
+      PhjBuilder::HashTableStoresNulls(join_op_, plan_node.is_not_distinct_from_),
+      plan_node.is_not_distinct_from_);
 
   RETURN_IF_ERROR(
       HashTableCtx::CodegenHashRow(codegen, false, hash_table_config, &hash_fn));
@@ -1487,7 +1526,7 @@ Status PartitionedHashJoinNode::CodegenProcessProbeBatch(
   // Replace hash-table parameters with constants.
   HashTableCtx::HashTableReplacedConstants replaced_constants;
   const bool stores_duplicates = true;
-  const int num_build_tuples = child(1)->row_desc()->tuple_descriptors().size();
+  const int num_build_tuples = build_row_desc().tuple_descriptors().size();
   RETURN_IF_ERROR(HashTableCtx::ReplaceHashTableConstants(codegen, hash_table_config,
       stores_duplicates, num_build_tuples, process_probe_batch_fn, &replaced_constants));
   DCHECK_GE(replaced_constants.stores_nulls, 1);
diff --git a/be/src/exec/partitioned-hash-join-node.h b/be/src/exec/partitioned-hash-join-node.h
index 9c6f16b..78cc851 100644
--- a/be/src/exec/partitioned-hash-join-node.h
+++ b/be/src/exec/partitioned-hash-join-node.h
@@ -50,12 +50,19 @@ class PartitionedHashJoinPlanNode : public BlockingJoinPlanNode {
   std::vector<ScalarExpr*> build_exprs_;
   std::vector<ScalarExpr*> probe_exprs_;
 
+  /// is_not_distinct_from_[i] is true if and only if the ith equi-join predicate is IS
+  /// NOT DISTINCT FROM, rather than equality.
+  std::vector<bool> is_not_distinct_from_;
+
   /// Non-equi-join conjuncts from the ON clause.
   std::vector<ScalarExpr*> other_join_conjuncts_;
 
   /// Data sink config object for creating a phj builder that will be eventually used by
   /// the exec node.
   const PhjBuilderConfig* phj_builder_config;
+
+  /// Seed used for hashing rows.
+  uint32_t hash_seed_;
 };
 
 /// Operator to perform partitioned hash join, spilling to disk as necessary. This
@@ -83,12 +90,12 @@ class PartitionedHashJoinPlanNode : public BlockingJoinPlanNode {
 ///
 /// The above algorithm is implemented as a state machine with the following phases:
 ///
-///   1. [PARTITIONING_BUILD or REPARTITIONING_BUILD] Read build rows from child(1) OR
-///      from the spilled build rows of a partition and partition them into the builder's
-///      hash partitions. If there is sufficient memory, all build partitions are kept
-///      in memory. Otherwise, build partitions are spilled as needed to free up memory.
-///      Finally, build a hash table for each in-memory partition and create a probe
-///      partition with a write buffer for each spilled partition.
+///   1. [PARTITIONING_BUILD or REPARTITIONING_BUILD] Read build rows from the right
+///      input plan tree OR from the spilled build rows of a partition and partition them
+///      into the builder's hash partitions. If there is sufficient memory, all build
+///      partitions are kept in memory. Otherwise, build partitions are spilled as needed
+///      to free up memory. Finally, build a hash table for each in-memory partition and
+///      create a probe partition with a write buffer for each spilled partition.
 ///
 ///      After the phase, the algorithm advances from PARTITIONING_BUILD to
 ///      PARTITIONING_PROBE or from REPARTITIONING_BUILD to REPARTITIONING_PROBE.
@@ -518,12 +525,16 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
   Status CodegenProcessProbeBatch(
       LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) WARN_UNUSED_RESULT;
 
+  uint32_t hash_seed() const {
+    return static_cast<const PartitionedHashJoinPlanNode&>(plan_node_).hash_seed_;
+  }
+
   std::string NodeDebugString() const;
 
   RuntimeState* runtime_state_;
 
   /// Our equi-join predicates "<lhs> = <rhs>" are separated into
-  /// build_exprs_ (over child(1)) and probe_exprs_ (over child(0))
+  /// build_exprs_ (over the build input row) and probe_exprs_ (over child(0))
   std::vector<ScalarExpr*> build_exprs_;
   std::vector<ScalarExpr*> probe_exprs_;
 
@@ -556,8 +567,10 @@ class PartitionedHashJoinNode : public BlockingJoinNode {
   /// State of the probing algorithm. Used to drive the state machine in GetNext().
   ProbeState probe_state_ = ProbeState::PROBE_COMPLETE;
 
-  /// The build-side of the join. Initialized in constructor and owned by runtime state.
-  PhjBuilder* builder_;
+  /// 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.
+  PhjBuilder* builder_ = nullptr;
 
   /// Last set of hash partitions obtained from builder_. Only valid when the
   /// builder's state is PARTITIONING_PROBE or REPARTITIONING_PROBE.
diff --git a/be/src/runtime/bufferpool/buffer-pool-internal.h b/be/src/runtime/bufferpool/buffer-pool-internal.h
index a0aa906..98d9264 100644
--- a/be/src/runtime/bufferpool/buffer-pool-internal.h
+++ b/be/src/runtime/bufferpool/buffer-pool-internal.h
@@ -254,6 +254,9 @@ class BufferPool::Client {
   /// Implementation of ClientHandle::DecreaseReservationTo().
   Status DecreaseReservationTo(int64_t max_decrease, int64_t target_bytes) WARN_UNUSED_RESULT;
 
+  /// Implementations of ClientHandle::TransferReservationTo().
+  Status TransferReservationTo(ReservationTracker* dst, int64_t bytes, bool* transferred);
+
   /// Called after a buffer of 'len' is freed via the FreeBuffer() API to update
   /// internal accounting and release the buffer to the client's reservation. No page or
   /// client locks should be held by the caller.
@@ -310,8 +313,10 @@ class BufferPool::Client {
   /// Must be called once before allocating or reclaiming a buffer of 'len'. Ensures that
   /// enough dirty pages are flushed to disk to satisfy the buffer pool's internal
   /// invariants after the allocation. 'lock_' should be held by the caller via
-  /// 'client_lock'
-  Status CleanPages(boost::unique_lock<boost::mutex>* client_lock, int64_t len);
+  /// 'client_lock'. If 'lazy_flush' is true, only write out pages if needed to reclaim
+  /// 'len', and do not return a write error if the error prevents flushing enough pages.
+  Status CleanPages(boost::unique_lock<boost::mutex>* client_lock, int64_t len,
+      bool lazy_flush = false);
 
   /// Initiates asynchronous writes of dirty unpinned pages to disk. Ensures that at
   /// least 'min_bytes_to_write' bytes of writes will be written asynchronously. May
diff --git a/be/src/runtime/bufferpool/buffer-pool-test.cc b/be/src/runtime/bufferpool/buffer-pool-test.cc
index 58744a6..b6d6f91 100644
--- a/be/src/runtime/bufferpool/buffer-pool-test.cc
+++ b/be/src/runtime/bufferpool/buffer-pool-test.cc
@@ -1575,6 +1575,12 @@ void BufferPoolTest::TestWriteError(int write_delay_ms) {
   EXPECT_EQ(TErrorCode::SCRATCH_ALLOCATION_FAILED, error.code());
   EXPECT_FALSE(pages[0].is_pinned());
 
+  // Transferring reservation does not result in an error.
+  bool transferred;
+  EXPECT_OK(
+      client.TransferReservationTo(&global_reservations_, TEST_BUFFER_LEN, &transferred));
+  EXPECT_TRUE(transferred);
+
   DestroyAll(&pool, &client, &pages);
   pool.DeregisterClient(&client);
   global_reservations_.Close();
diff --git a/be/src/runtime/bufferpool/buffer-pool.cc b/be/src/runtime/bufferpool/buffer-pool.cc
index e36dfb8..70f3437 100644
--- a/be/src/runtime/bufferpool/buffer-pool.cc
+++ b/be/src/runtime/bufferpool/buffer-pool.cc
@@ -26,6 +26,7 @@
 #include "runtime/bufferpool/buffer-allocator.h"
 #include "util/bit-util.h"
 #include "util/cpu-info.h"
+#include "util/debug-util.h"
 #include "util/metrics.h"
 #include "util/runtime-profile-counters.h"
 #include "util/time.h"
@@ -337,9 +338,15 @@ bool BufferPool::ClientHandle::TransferReservationFrom(
   return src->TransferReservationTo(impl_->reservation(), bytes);
 }
 
-bool BufferPool::ClientHandle::TransferReservationTo(
-    ReservationTracker* dst, int64_t bytes) {
-  return impl_->reservation()->TransferReservationTo(dst, bytes);
+Status BufferPool::ClientHandle::TransferReservationTo(
+    ReservationTracker* dst, int64_t bytes, bool* transferred) {
+  return impl_->TransferReservationTo(dst, bytes, transferred);
+}
+
+Status BufferPool::ClientHandle::TransferReservationTo(ClientHandle* dst, int64_t bytes,
+    bool* transferred) {
+  DCHECK(dst->is_registered());
+  return TransferReservationTo(dst->impl_->reservation(), bytes, transferred);
 }
 
 void BufferPool::ClientHandle::SaveReservation(SubReservation* dst, int64_t bytes) {
@@ -625,17 +632,43 @@ Status BufferPool::Client::DecreaseReservationTo(
   return Status::OK();
 }
 
-Status BufferPool::Client::CleanPages(unique_lock<mutex>* client_lock, int64_t len) {
+Status BufferPool::Client::TransferReservationTo(
+    ReservationTracker* dst, int64_t bytes, bool* transferred) {
+  unique_lock<mutex> lock(lock_);
+  // Only flush pages if necessary, to avoid propagating write errors unnecessarily.
+  RETURN_IF_ERROR(CleanPages(&lock, bytes, /*lazy_flush=*/true));
+  *transferred = reservation_.TransferReservationTo(dst, bytes);
+  return Status::OK();
+}
+
+Status BufferPool::Client::CleanPages(
+    unique_lock<mutex>* client_lock, int64_t len, bool lazy_flush) {
+  DCHECK_GE(len, 0);
+  DCHECK_LE(len, reservation_.GetReservation());
   DCheckHoldsLock(*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"
+              << DebugStringLocked();
+  }
   // Start enough writes to ensure that the loop condition below will eventually become
   // false (or a write error will be encountered).
   int64_t min_bytes_to_write =
       max<int64_t>(0, dirty_unpinned_pages_.bytes() - target_dirty_bytes);
+  if (lazy_flush
+      && dirty_unpinned_pages_.bytes() + in_flight_write_pages_.bytes()
+          <= target_dirty_bytes) {
+    return Status::OK();
+  }
   WriteDirtyPagesAsync(min_bytes_to_write);
 
   // One of the writes we initiated, or an earlier in-flight write may have hit an error.
@@ -719,7 +752,7 @@ void BufferPool::Client::WriteCompleteCallback(Page* page, const Status& write_s
     in_flight_write_pages_.Remove(page);
     // Move to clean pages list even if an error was encountered - the buffer can be
     // repurposed by other clients and 'write_status_' must be checked by this client
-    // before reading back the bad data.
+    // before it can be re-pinned.
     pool_->allocator_->AddCleanPage(cl, page);
     WriteDirtyPagesAsync(); // Start another asynchronous write if needed.
 
diff --git a/be/src/runtime/bufferpool/buffer-pool.h b/be/src/runtime/bufferpool/buffer-pool.h
index d7c74db..43f457e 100644
--- a/be/src/runtime/bufferpool/buffer-pool.h
+++ b/be/src/runtime/bufferpool/buffer-pool.h
@@ -381,8 +381,13 @@ class BufferPool::ClientHandle {
   bool TransferReservationFrom(ReservationTracker* src, int64_t bytes);
 
   /// Transfer 'bytes' of reservation from this client to 'dst' using
-  /// ReservationTracker::TransferReservationTo().
-  bool TransferReservationTo(ReservationTracker* dst, int64_t bytes);
+  /// ReservationTracker::TransferReservationTo(). The client must have at least 'bytes'
+  /// of unused reservation. May fail if transferring the reservation requires flushing
+  /// 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.
+  Status TransferReservationTo(ReservationTracker* dst, int64_t bytes, bool* transferred);
+  Status TransferReservationTo(ClientHandle* dst, int64_t bytes, bool* transferred);
 
   /// Call SetDebugDenyIncreaseReservation() on this client's ReservationTracker.
   void SetDebugDenyIncreaseReservation(double probability);
diff --git a/be/src/runtime/coordinator-backend-state.cc b/be/src/runtime/coordinator-backend-state.cc
index 4a4243c..825732b 100644
--- a/be/src/runtime/coordinator-backend-state.cc
+++ b/be/src/runtime/coordinator-backend-state.cc
@@ -144,6 +144,7 @@ void Coordinator::BackendState::SetRpcParams(const DebugOptions& debug_options,
     instance_ctx.__set_per_exch_num_senders(
         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 (debug_options.enabled()
         && (debug_options.instance_idx() == -1
             || debug_options.instance_idx() == GetInstanceIdx(params.instance_id))) {
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index ca24d9b..06bc265 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -167,7 +167,6 @@ Status Coordinator::Exec() {
   exec_summary_.Init(schedule_);
 
   if (filter_mode_ != TRuntimeFilterMode::OFF) {
-    DCHECK_EQ(request.plan_exec_info.size(), 1);
     // Populate the runtime filter routing table. This should happen before starting the
     // fragment instances. This code anticipates the indices of the instance states
     // created later on in ExecRemoteFragment()
@@ -320,15 +319,23 @@ void Coordinator::ExecSummary::Init(const QuerySchedule& schedule) {
         node_summary.exec_stats.resize(num_instances);
       }
 
-      if (fragment.__isset.output_sink
-          && fragment.output_sink.type == TDataSinkType::DATA_STREAM_SINK) {
-        const TDataStreamSink& sink = fragment.output_sink.stream_sink;
-        int exch_idx = node_id_to_idx_map[sink.dest_node_id];
-        if (sink.output_partition.type == TPartitionType::UNPARTITIONED) {
-          thrift_exec_summary.nodes[exch_idx].__set_is_broadcast(true);
+      if (fragment.__isset.output_sink &&
+          (fragment.output_sink.type == TDataSinkType::DATA_STREAM_SINK
+           || IsJoinBuildSink(fragment.output_sink.type))) {
+        int dst_node_idx;
+        if (fragment.output_sink.type == TDataSinkType::DATA_STREAM_SINK) {
+          const TDataStreamSink& sink = fragment.output_sink.stream_sink;
+          dst_node_idx = node_id_to_idx_map[sink.dest_node_id];
+          if (sink.output_partition.type == TPartitionType::UNPARTITIONED) {
+            thrift_exec_summary.nodes[dst_node_idx].__set_is_broadcast(true);
+          }
+        } else {
+          DCHECK(IsJoinBuildSink(fragment.output_sink.type));
+          const TJoinBuildSink& sink = fragment.output_sink.join_build_sink;
+          dst_node_idx = node_id_to_idx_map[sink.dest_node_id];
         }
         thrift_exec_summary.__isset.exch_to_sender_map = true;
-        thrift_exec_summary.exch_to_sender_map[exch_idx] = root_node_idx;
+        thrift_exec_summary.exch_to_sender_map[dst_node_idx] = root_node_idx;
       }
     }
   }
@@ -347,14 +354,26 @@ void Coordinator::InitFilterRoutingTable() {
     int num_backends = fragment_params.GetNumBackends();
     DCHECK_GT(num_backends, 0);
 
-    // TODO: IMPALA-4224: also call AddFilterSource for build sinks that produce filters.
+    // Hash join build sinks can produce filters in mt_dop > 0 plans.
+    if (fragment_params.fragment.output_sink.__isset.join_build_sink) {
+      const TJoinBuildSink& join_sink =
+          fragment_params.fragment.output_sink.join_build_sink;
+      for (const TRuntimeFilterDesc& filter: join_sink.runtime_filters) {
+        // The join node ID is used to identify the join that produces the filter, even
+        // though the builder is separate from the actual node.
+        DCHECK_EQ(filter.src_node_id, join_sink.dest_node_id);
+        AddFilterSource(
+            fragment_params, num_instances, num_backends, filter, filter.src_node_id);
+      }
+    }
     for (const TPlanNode& plan_node: fragment_params.fragment.plan.nodes) {
       if (!plan_node.__isset.runtime_filters) continue;
       for (const TRuntimeFilterDesc& filter: plan_node.runtime_filters) {
         DCHECK(filter_mode_ == TRuntimeFilterMode::GLOBAL || filter.has_local_targets);
         // Currently hash joins are the only filter sources. Otherwise it must be
         // a filter consumer.
-        if (plan_node.__isset.hash_join_node) {
+        if (plan_node.__isset.join_node &&
+            plan_node.join_node.__isset.hash_join_node) {
           AddFilterSource(
               fragment_params, num_instances, num_backends, filter, plan_node.node_id);
         } else if (plan_node.__isset.hdfs_scan_node || plan_node.__isset.kudu_scan_node) {
diff --git a/be/src/runtime/fragment-instance-state.cc b/be/src/runtime/fragment-instance-state.cc
index 099e98b..6aa4a86 100644
--- a/be/src/runtime/fragment-instance-state.cc
+++ b/be/src/runtime/fragment-instance-state.cc
@@ -29,6 +29,9 @@
 #include "exec/exchange-node.h"
 #include "exec/exec-node.h"
 #include "exec/hdfs-scan-node-base.h"
+#include "exec/join-builder.h"
+#include "exec/nested-loop-join-builder.h"
+#include "exec/partitioned-hash-join-builder.h"
 #include "exec/plan-root-sink.h"
 #include "exec/scan-node.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
@@ -191,7 +194,7 @@ Status FragmentInstanceState::Prepare() {
     DCHECK_EQ(exch_node->type(), TPlanNodeType::EXCHANGE_NODE);
     int num_senders =
         FindWithDefault(instance_ctx_.per_exch_num_senders, exch_node->id(), 0);
-    DCHECK_GT(num_senders, 0);
+    DCHECK_GT(num_senders, 0) << exch_node->id();
     static_cast<ExchangeNode*>(exch_node)->set_num_senders(num_senders);
   }
 
@@ -389,8 +392,16 @@ Status FragmentInstanceState::ExecInternal() {
     RETURN_IF_ERROR(sink_->Send(runtime_state_, row_batch_.get()));
     UpdateState(StateEvent::BATCH_SENT);
   } while (!exec_tree_complete);
+  // Release resources from final row batch.
+  row_batch_->Reset();
 
   UpdateState(StateEvent::LAST_BATCH_SENT);
+
+  // Close the tree before the sink is flushed to release 'exec_tree_' resources.
+  // This can significantly reduce resource consumption if 'sink_' is a join
+  // build, where FlushFinal() blocks until the consuming fragment is finished.
+  exec_tree_->Close(runtime_state_);
+
   // Flush the sink as a final step.
   RETURN_IF_ERROR(sink_->FlushFinal(runtime_state()));
   return Status::OK();
@@ -417,11 +428,7 @@ void FragmentInstanceState::Close() {
   // Stop updating profile counters in background.
   profile()->StopPeriodicCounters();
 
-  // We need to delete row_batch_ here otherwise we can't delete the instance_mem_tracker_
-  // in runtime_state_->ReleaseResources().
-  // TODO: do not delete mem trackers in Close()/ReleaseResources(), they are part of
-  // the control structures we need to preserve until the underlying QueryState
-  // disappears.
+  // Delete row_batch_ to free resources associated with it.
   row_batch_.reset();
   if (exec_tree_ != nullptr) exec_tree_->Close(runtime_state_);
   runtime_state_->ReleaseResources();
@@ -556,6 +563,14 @@ PlanRootSink* FragmentInstanceState::GetRootSink() const {
       nullptr;
 }
 
+bool FragmentInstanceState::HasJoinBuildSink() const {
+  return IsJoinBuildSink(fragment_ctx_.fragment.output_sink.type);
+}
+
+JoinBuilder* FragmentInstanceState::GetJoinBuildSink() const {
+  return HasJoinBuildSink() ? static_cast<JoinBuilder*>(sink_) : nullptr;
+}
+
 const TQueryCtx& FragmentInstanceState::query_ctx() const {
   return query_state_->query_ctx();
 }
diff --git a/be/src/runtime/fragment-instance-state.h b/be/src/runtime/fragment-instance-state.h
index 395a519..376faee 100644
--- a/be/src/runtime/fragment-instance-state.h
+++ b/be/src/runtime/fragment-instance-state.h
@@ -58,6 +58,7 @@ class Thread;
 class DataSink;
 class DataSinkConfig;
 class RuntimeState;
+class JoinBuilder;
 
 /// FragmentInstanceState handles all aspects of the execution of a single plan fragment
 /// instance, including setup and finalization, both in the success and error case.
@@ -110,9 +111,13 @@ class FragmentInstanceState {
   void ReportSuccessful(const FragmentInstanceExecStatusPB& instance_status);
   void ReportFailed(const FragmentInstanceExecStatusPB& instance_status);
 
-  /// Returns fragment instance's sink if this is the root fragment instance. Valid after
-  /// the Prepare phase. May be nullptr.
+  /// Accessor functions for this fragment instance's sink. Valid after the Prepare
+  /// phase. Returns nullptr if this fragment has a different sink type.
   PlanRootSink* GetRootSink() const;
+  JoinBuilder* GetJoinBuildSink() const;
+
+  /// Return true if this finstance's sink is a join builder.
+  bool HasJoinBuildSink() const;
 
   /// Returns a string description of 'state'.
   static const string& ExecStateToString(FInstanceExecStatePB state);
diff --git a/be/src/runtime/initial-reservations.cc b/be/src/runtime/initial-reservations.cc
index 02728fa..300f3c2 100644
--- a/be/src/runtime/initial-reservations.cc
+++ b/be/src/runtime/initial-reservations.cc
@@ -78,9 +78,12 @@ void InitialReservations::Claim(BufferPool::ClientHandle* dst, int64_t bytes) {
 
 void InitialReservations::Return(BufferPool::ClientHandle* src, int64_t bytes) {
   lock_guard<SpinLock> l(lock_);
-  bool success = src->TransferReservationTo(&initial_reservations_, bytes);
+  bool success;
+  Status status = src->TransferReservationTo(&initial_reservations_, bytes, &success);
+  DCHECK(status.ok()) << status.GetDetail() << " no dirty pages to flush, can't fail "
+                      << src->DebugString();
   // No limits on our tracker - no way this should fail.
-  DCHECK(success);
+  DCHECK(success) << initial_reservations_.DebugString();
   // Check to see if we can release any reservation.
   int64_t excess_reservation =
     initial_reservations_.GetReservation() - remaining_initial_reservation_claims_;
diff --git a/be/src/runtime/row-batch.cc b/be/src/runtime/row-batch.cc
index b5c51bb..7c53ac2 100644
--- a/be/src/runtime/row-batch.cc
+++ b/be/src/runtime/row-batch.cc
@@ -488,7 +488,8 @@ int64_t RowBatch::GetSerializedSize(const OutboundRowBatch& batch) {
 }
 
 void RowBatch::AcquireState(RowBatch* src) {
-  DCHECK(row_desc_->LayoutEquals(*src->row_desc_));
+  DCHECK(row_desc_->LayoutEquals(*src->row_desc_)) << row_desc_->DebugString() << "\n"
+    << src->row_desc_->DebugString();
   DCHECK_EQ(num_tuples_per_row_, src->num_tuples_per_row_);
   DCHECK_EQ(tuple_ptrs_size_, src->tuple_ptrs_size_);
 
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index e04adef..56c36f7 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -231,6 +231,19 @@ Status RuntimeState::LogOrReturnError(const ErrorMsg& message) {
 
 void RuntimeState::Cancel() {
   is_cancelled_.Store(true);
+  {
+    lock_guard<SpinLock> l(cancellation_cvs_lock_);
+    for (ConditionVariable* cv : cancellation_cvs_) cv->NotifyAll();
+  }
+}
+
+void RuntimeState::AddCancellationCV(ConditionVariable* cv) {
+  lock_guard<SpinLock> l(cancellation_cvs_lock_);
+  for (ConditionVariable* cv2 : cancellation_cvs_) {
+    // Don't add if already present.
+    if (cv == cv2) return;
+  }
+  cancellation_cvs_.push_back(cv);
 }
 
 double RuntimeState::ComputeExchangeScanRatio() const {
diff --git a/be/src/runtime/runtime-state.h b/be/src/runtime/runtime-state.h
index e29c15a..f69a62a 100644
--- a/be/src/runtime/runtime-state.h
+++ b/be/src/runtime/runtime-state.h
@@ -55,6 +55,7 @@ class HBaseTableFactory;
 class TPlanFragmentCtx;
 class TPlanFragmentInstanceCtx;
 class QueryState;
+class ConditionVariable;
 
 namespace io {
   class DiskIoMgr;
@@ -238,6 +239,11 @@ class RuntimeState {
 
   bool is_cancelled() const { return is_cancelled_.Load(); }
   void Cancel();
+  /// Add a condition variable to be signalled when this RuntimeState is cancelled.
+  /// Adding a condition variable multiple times is a no-op. Each distinct 'cv' will be
+  /// signalled once with NotifyAll() when is_cancelled() becomes true.
+  /// The condition variable must have query lifetime.
+  void AddCancellationCV(ConditionVariable* cv);
 
   RuntimeProfile::Counter* total_storage_wait_timer() {
     return total_storage_wait_timer_;
@@ -418,6 +424,11 @@ class RuntimeState {
   /// status once it notices is_cancelled_ == true.
   AtomicBool is_cancelled_{false};
 
+  /// Condition variables that will be signalled by Cancel(). Protected by
+  /// 'cancellation_cvs_lock_'.
+  std::vector<ConditionVariable*> cancellation_cvs_;
+  SpinLock cancellation_cvs_lock_;
+
   /// if true, ReleaseResources() was called.
   bool released_resources_ = false;
 
@@ -427,11 +438,9 @@ class RuntimeState {
   SpinLock query_status_lock_;
   Status query_status_;
 
-  /// This is the node id of the root node for this plan fragment. This is used as the
-  /// hash seed and has two useful properties:
-  /// 1) It is the same for all exec nodes in a fragment, so the resulting hash values
-  /// can be shared.
-  /// 2) It is different between different fragments, so we do not run into hash
+  /// This is the node id of the root node for this plan fragment.
+  ///
+  /// This is used as the hash seed within the fragment so we do not run into hash
   /// collisions after data partitioning (across fragments). See IMPALA-219 for more
   /// details.
   PlanNodeId root_node_id_ = -1;
diff --git a/be/src/runtime/spillable-row-batch-queue.h b/be/src/runtime/spillable-row-batch-queue.h
index 97c60d8..a7da32c 100644
--- a/be/src/runtime/spillable-row-batch-queue.h
+++ b/be/src/runtime/spillable-row-batch-queue.h
@@ -73,7 +73,8 @@ class SpillableRowBatchQueue {
   /// achieved because there is no more available reserved memory, this method will unpin
   /// the stream and then add the RowBatch. If the batch still cannot be added, this
   /// method returns an error Status. It is not valid to call this method if the queue is
-  /// full or closed.
+  /// full or closed. After this returns, 'batch' can be safely destroyed (i.e. the
+  /// queue makes copies of all the data from 'batch' that it needs).
   Status AddBatch(RowBatch* batch);
 
   /// Returns and removes the RowBatch at the head of the queue. Returns Status::OK() if
diff --git a/be/src/util/summary-util.cc b/be/src/util/summary-util.cc
index aded383..cdb945b 100644
--- a/be/src/util/summary-util.cc
+++ b/be/src/util/summary-util.cc
@@ -95,9 +95,11 @@ void PrintExecSummary(const TExecSummary& exec_summary, int indent_level,
   map<int, int>::const_iterator child_fragment_idx_it =
       exec_summary.exch_to_sender_map.find(*node_idx);
   if (child_fragment_idx_it != exec_summary.exch_to_sender_map.end()) {
-    DCHECK_EQ(node.num_children, 0);
+    int child_fragment_indent_level = indent_level + node.num_children;
+    bool child_fragment_new_indent_level = node.num_children > 0;
     int child_fragment_id = child_fragment_idx_it->second;
-    PrintExecSummary(exec_summary, indent_level, false, &child_fragment_id, result);
+    PrintExecSummary(exec_summary, child_fragment_indent_level,
+        child_fragment_new_indent_level, &child_fragment_id, result);
   }
   ++*node_idx;
   if (node.num_children == 0) return;
diff --git a/bin/run-all-tests.sh b/bin/run-all-tests.sh
index 6d4919b..37eee42 100755
--- a/bin/run-all-tests.sh
+++ b/bin/run-all-tests.sh
@@ -164,7 +164,7 @@ do
 
   run-step "Starting Impala cluster" start-impala-cluster.log \
       "${IMPALA_HOME}/bin/start-impala-cluster.py" --log_dir="${IMPALA_EE_TEST_LOGS_DIR}" \
-      ${TEST_START_CLUSTER_ARGS}
+      ${TEST_START_CLUSTER_ARGS} --impalad_args=--unlock_mt_dop=true
 
   if [[ "$BE_TEST" == true ]]; then
     if [[ "$TARGET_FILESYSTEM" == "local" ]]; then
diff --git a/common/thrift/DataSinks.thrift b/common/thrift/DataSinks.thrift
index fdf8f6c..81c1a67 100644
--- a/common/thrift/DataSinks.thrift
+++ b/common/thrift/DataSinks.thrift
@@ -23,13 +23,15 @@ include "Exprs.thrift"
 include "Types.thrift"
 include "Descriptors.thrift"
 include "Partitions.thrift"
+include "PlanNodes.thrift"
 include "ResourceProfile.thrift"
 
 enum TDataSinkType {
   DATA_STREAM_SINK = 0
   TABLE_SINK = 1
-  JOIN_BUILD_SINK = 2
+  HASH_JOIN_BUILDER = 2
   PLAN_ROOT_SINK = 3
+  NESTED_LOOP_JOIN_BUILDER = 4
 }
 
 enum TSinkAction {
@@ -103,8 +105,18 @@ struct TJoinBuildSink {
   // destination join node id
   1: required Types.TPlanNodeId dest_node_id
 
-  // only set for hash join build sinks
-  2: required list<Exprs.TExpr> build_exprs
+  // Join operation implemented by the JoinNode
+  2: required PlanNodes.TJoinOp join_op
+
+  // Equi-join conjunctions. Only set for hash join builds.
+  3: optional list<PlanNodes.TEqJoinCondition> eq_join_conjuncts
+
+  // Runtime filters produced by this sink.
+  4: optional list<PlanNodes.TRuntimeFilterDesc> runtime_filters
+
+  // 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
 }
 
 struct TPlanRootSink {
@@ -137,4 +149,7 @@ struct TDataSink {
   // Exprs that produce values for slots of output tuple (one expr per slot).
   // Only set by the DataSink implementations that require it.
   8: optional list<Exprs.TExpr> output_exprs
+
+  // Resource profile for this data sink. Always set.
+  9: optional ResourceProfile.TBackendResourceProfile resource_profile
 }
diff --git a/common/thrift/ExecStats.thrift b/common/thrift/ExecStats.thrift
index 30c1fcd..d059d24 100644
--- a/common/thrift/ExecStats.thrift
+++ b/common/thrift/ExecStats.thrift
@@ -98,8 +98,9 @@ struct TExecSummary {
   // Flattened execution summary of the plan tree.
   3: optional list<TPlanNodeExecSummary> nodes
 
-  // For each exch node in 'nodes', contains the index to the root node of the sending
-  // fragment for this exch. Both the key and value are indices into 'nodes'.
+  // For each node in 'nodes' that consumes input from the root of a different fragment,
+  // i.e. an exchange or join node with a separate build, contains the index to the root
+  // node of the source fragment. Both the key and value are indices into 'nodes'.
   4: optional map<i32, i32> exch_to_sender_map
 
   // List of errors that were encountered during execution. This can be non-empty
diff --git a/common/thrift/PlanNodes.thrift b/common/thrift/PlanNodes.thrift
index 4d5a556..17e353e 100644
--- a/common/thrift/PlanNodes.thrift
+++ b/common/thrift/PlanNodes.thrift
@@ -341,6 +341,7 @@ struct TEqJoinCondition {
   3: required bool is_not_distinct_from;
 }
 
+// Different join operations supported by backend join operations.
 enum TJoinOp {
   INNER_JOIN = 0
   LEFT_OUTER_JOIN = 1
@@ -361,21 +362,38 @@ enum TJoinOp {
 }
 
 struct THashJoinNode {
-  1: required TJoinOp join_op
-
   // equi-join predicates
-  2: required list<TEqJoinCondition> eq_join_conjuncts
+  1: required list<TEqJoinCondition> eq_join_conjuncts
 
   // non equi-join predicates
-  3: optional list<Exprs.TExpr> other_join_conjuncts
+  2: optional list<Exprs.TExpr> other_join_conjuncts
+
+  // Hash seed to use. Must be the same as the join builder's hash seed, if there is
+  // a separate join build. Must be positive.
+  3: optional i32 hash_seed
 }
 
 struct TNestedLoopJoinNode {
-  1: required TJoinOp join_op
-
   // Join conjuncts (both equi-join and non equi-join). All other conjuncts that are
   // evaluated at the join node are stored in TPlanNode.conjuncts.
-  2: optional list<Exprs.TExpr> join_conjuncts
+  1: optional list<Exprs.TExpr> join_conjuncts
+}
+
+// Top-level struct for a join node. Elements that are shared between the different
+// join implementations are top-level variables and elements that are specific to a
+// join implementation live in a specialized struct.
+struct TJoinNode {
+  1: required TJoinOp join_op
+
+  // Tuples in build row.
+  2: optional list<Types.TTupleId> build_tuples
+
+  // Nullable tuples in build row.
+  3: optional list<bool> nullable_build_tuples
+
+  // One of these must be set.
+  4: optional THashJoinNode hash_join_node
+  5: optional TNestedLoopJoinNode nested_loop_join_node
 }
 
 struct TAggregator {
@@ -599,8 +617,7 @@ struct TPlanNode {
   10: optional THBaseScanNode hbase_scan_node
   11: optional TKuduScanNode kudu_scan_node
   12: optional TDataSourceScanNode data_source_node
-  13: optional THashJoinNode hash_join_node
-  14: optional TNestedLoopJoinNode nested_loop_join_node
+  13: optional TJoinNode join_node
   15: optional TAggregationNode agg_node
   16: optional TSortNode sort_node
   17: optional TUnionNode union_node
diff --git a/common/thrift/Types.thrift b/common/thrift/Types.thrift
index 0c7ff71..0485bf2 100644
--- a/common/thrift/Types.thrift
+++ b/common/thrift/Types.thrift
@@ -29,7 +29,6 @@ typedef i32 TDataSinkId
 typedef i32 TTupleId
 typedef i32 TSlotId
 typedef i32 TTableId
-typedef i32 TJoinTableId
 
 // TODO: Consider moving unrelated enums to better locations.
 
diff --git a/fe/src/main/java/org/apache/impala/planner/DataSink.java b/fe/src/main/java/org/apache/impala/planner/DataSink.java
index b78e67c..2cacb4d 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSink.java
@@ -17,15 +17,19 @@
 
 package org.apache.impala.planner;
 
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.impala.analysis.Expr;
+import org.apache.impala.planner.RuntimeFilterGenerator.RuntimeFilter;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExecStats;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TQueryOptions;
 
+import com.google.common.base.Preconditions;
+
 /**
  * A DataSink describes the destination of a plan fragment's output rows.
  * The destination could be another plan fragment on a remote machine,
@@ -68,6 +72,13 @@ public abstract class DataSink {
    */
   abstract protected String getLabel();
 
+
+  /**
+   * Return runtime filters produced by this sink. Subclasses that use runtime filters
+   * must override.
+   */
+  public List<RuntimeFilter> getRuntimeFilters() { return Collections.emptyList(); }
+
   /**
    * Construct a thrift representation of the sink.
    */
@@ -77,6 +88,8 @@ public abstract class DataSink {
     TExecStats estimatedStats = new TExecStats();
     estimatedStats.setMemory_used(resourceProfile_.getMemEstimateBytes());
     tsink.setEstimated_stats(estimatedStats);
+    Preconditions.checkState(resourceProfile_.isValid());
+    tsink.resource_profile = resourceProfile_.toThrift();
     toThriftImpl(tsink);
     return tsink;
   }
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 85bc8bd..9e568ad 100644
--- a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
@@ -29,6 +29,7 @@ import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.InternalException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.thrift.TEqJoinCondition;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.THashJoinNode;
@@ -134,19 +135,30 @@ public class HashJoinNode extends JoinNode {
   @Override
   protected void toThrift(TPlanNode msg) {
     msg.node_type = TPlanNodeType.HASH_JOIN_NODE;
-    msg.hash_join_node = new THashJoinNode();
-    msg.hash_join_node.join_op = joinOp_.toThrift();
-    for (Expr entry: eqJoinConjuncts_) {
+    msg.join_node = joinNodeToThrift();
+    msg.join_node.hash_join_node = new THashJoinNode();
+    msg.join_node.hash_join_node.setEq_join_conjuncts(getThriftEquiJoinConjuncts());
+    for (Expr e: otherJoinConjuncts_) {
+      msg.join_node.hash_join_node.addToOther_join_conjuncts(e.treeToThrift());
+    }
+    msg.join_node.hash_join_node.setHash_seed(getFragment().getHashSeed());
+  }
+
+  /**
+   * Helper to get the equi-join conjuncts in a thrift representation.
+   */
+  public List<TEqJoinCondition> getThriftEquiJoinConjuncts() {
+    List<TEqJoinCondition> equiJoinConjuncts = new ArrayList<>(eqJoinConjuncts_.size());
+    for (Expr entry : eqJoinConjuncts_) {
       BinaryPredicate bp = (BinaryPredicate)entry;
       TEqJoinCondition eqJoinCondition =
           new TEqJoinCondition(bp.getChild(0).treeToThrift(),
               bp.getChild(1).treeToThrift(),
               bp.getOp() == BinaryPredicate.Operator.NOT_DISTINCT);
-      msg.hash_join_node.addToEq_join_conjuncts(eqJoinCondition);
-    }
-    for (Expr e: otherJoinConjuncts_) {
-      msg.hash_join_node.addToOther_join_conjuncts(e.treeToThrift());
+
+      equiJoinConjuncts.add(eqJoinCondition);
     }
+    return equiJoinConjuncts;
   }
 
   @Override
@@ -203,7 +215,8 @@ public class HashJoinNode extends JoinNode {
   }
 
   @Override
-  public void computeNodeResourceProfile(TQueryOptions queryOptions) {
+  public Pair<ResourceProfile, ResourceProfile> computeJoinResourceProfile(
+      TQueryOptions queryOptions) {
     long perInstanceMemEstimate;
     long perInstanceDataBytes;
     int numInstances = fragment_.getNumInstances(queryOptions.getMt_dop());
@@ -227,8 +240,8 @@ public class HashJoinNode extends JoinNode {
 
     // Must be kept in sync with PartitionedHashJoinBuilder::MinReservation() in be.
     final int PARTITION_FANOUT = 16;
-    long minBuffers = PARTITION_FANOUT + 1
-        + (joinOp_ == JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN ? 3 : 0);
+    long minBuildBuffers = PARTITION_FANOUT + 1
+        + (joinOp_ == JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN ? 1 : 0);
 
     long bufferSize = queryOptions.getDefault_spillable_buffer_size();
     if (perInstanceDataBytes != -1) {
@@ -244,12 +257,32 @@ public class HashJoinNode extends JoinNode {
     // to serve as input and output buffers while repartitioning.
     long maxRowBufferSize =
         computeMaxSpillableBufferSize(bufferSize, queryOptions.getMax_row_size());
-    long perInstanceMinMemReservation =
-        bufferSize * (minBuffers - 2) + maxRowBufferSize * 2;
-    nodeResourceProfile_ = new ResourceProfileBuilder()
+    long perInstanceBuildMinMemReservation =
+        bufferSize * (minBuildBuffers - 2) + maxRowBufferSize * 2;
+    // 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.
+    long perInstanceProbeMinMemReservation = 0;
+    if (joinOp_ == JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN) {
+      // Only one of the NAAJ probe streams is written at a time, so it needs a max-sized
+      // buffer. If the build is integrated, we already have a max sized buffer accounted
+      // for in the build reservation.
+      perInstanceProbeMinMemReservation =
+          hasSeparateBuild() ? maxRowBufferSize + bufferSize : bufferSize * 2;
+    }
+
+    // Almost all resource consumption is in the build, or shared between the build and
+    // the probe. These are accounted for in the build profile.
+    ResourceProfile probeProfile = new ResourceProfileBuilder()
+        .setMemEstimateBytes(0)
+        .setMinMemReservationBytes(perInstanceProbeMinMemReservation)
+        .setSpillableBufferBytes(bufferSize)
+        .setMaxRowBufferBytes(maxRowBufferSize).build();
+    ResourceProfile buildProfile = new ResourceProfileBuilder()
         .setMemEstimateBytes(perInstanceMemEstimate)
-        .setMinMemReservationBytes(perInstanceMinMemReservation)
+        .setMinMemReservationBytes(perInstanceBuildMinMemReservation)
         .setSpillableBufferBytes(bufferSize)
         .setMaxRowBufferBytes(maxRowBufferSize).build();
+    return Pair.create(probeProfile, buildProfile);
   }
 }
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 63e733f..75af109 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.Expr;
+import org.apache.impala.planner.RuntimeFilterGenerator.RuntimeFilter;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExplainLevel;
@@ -43,6 +44,8 @@ public class JoinBuildSink extends DataSink {
 
   private final List<Expr> buildExprs_ = new ArrayList<>();
 
+  private final List<RuntimeFilter> runtimeFilters_ = new ArrayList<>();
+
   /**
    * Creates sink for build side of 'joinNode' (extracts buildExprs_ from joinNode).
    */
@@ -58,23 +61,37 @@ public class JoinBuildSink extends DataSink {
       // by convention the build exprs are the rhs of the join conjuncts
       buildExprs_.add(p.getChild(1).clone());
     }
+    runtimeFilters_.addAll(joinNode.getRuntimeFilters());
   }
 
   public JoinTableId getJoinTableId() { return joinTableId_; }
+  @Override
+  public List<RuntimeFilter> getRuntimeFilters() { return runtimeFilters_; }
 
   @Override
   protected void toThriftImpl(TDataSink tsink) {
     TJoinBuildSink tBuildSink = new TJoinBuildSink();
     tBuildSink.setDest_node_id(joinNode_.getId().asInt());
-    for (Expr buildExpr: buildExprs_) {
-      tBuildSink.addToBuild_exprs(buildExpr.treeToThrift());
+    tBuildSink.setJoin_op(joinNode_.getJoinOp().toThrift());
+    if (joinNode_ instanceof HashJoinNode) {
+      tBuildSink.setEq_join_conjuncts(
+          ((HashJoinNode)joinNode_).getThriftEquiJoinConjuncts());
+      tBuildSink.setHash_seed(joinNode_.getFragment().getHashSeed());
+    }
+    for (RuntimeFilter filter : runtimeFilters_) {
+      tBuildSink.addToRuntime_filters(filter.toThrift());
     }
     tsink.setJoin_build_sink(tBuildSink);
   }
 
   @Override
   protected TDataSinkType getSinkType() {
-    return TDataSinkType.JOIN_BUILD_SINK;
+    if (joinNode_ instanceof HashJoinNode) {
+      return TDataSinkType.HASH_JOIN_BUILDER;
+    } else {
+      Preconditions.checkState(joinNode_ instanceof NestedLoopJoinNode);
+      return TDataSinkType.NESTED_LOOP_JOIN_BUILDER;
+    }
   }
 
   @Override
@@ -90,6 +107,11 @@ public class JoinBuildSink extends DataSink {
         output.append(detailPrefix + "build expressions: ")
             .append(Expr.toSql(buildExprs_, DEFAULT) + "\n");
       }
+      if (!runtimeFilters_.isEmpty()) {
+        output.append(detailPrefix + "runtime filters: ");
+        output.append(PlanNode.getRuntimeFilterExplainString(
+            runtimeFilters_, true, joinNode_.getId(), detailLevel));
+      }
     }
   }
 
@@ -100,8 +122,7 @@ public class JoinBuildSink extends DataSink {
 
   @Override
   public void computeResourceProfile(TQueryOptions queryOptions) {
-    // The memory consumption is counted against the join PlanNode.
-    resourceProfile_ = ResourceProfile.noReservation(0);
+    resourceProfile_ = joinNode_.computeJoinResourceProfile(queryOptions).second;
   }
 
   @Override
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 944fe51..a3d6042 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
@@ -36,6 +36,7 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.Pair;
 import org.apache.impala.thrift.TExecNodePhase;
 import org.apache.impala.thrift.TJoinDistributionMode;
+import org.apache.impala.thrift.TJoinNode;
 import org.apache.impala.thrift.TQueryOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -198,6 +199,7 @@ public abstract class JoinNode extends PlanNode {
   public void setDistributionMode(DistributionMode distrMode) { distrMode_ = distrMode; }
   public JoinTableId getJoinTableId() { return joinTableId_; }
   public void setJoinTableId(JoinTableId id) { joinTableId_ = id; }
+  public boolean hasSeparateBuild() { return joinTableId_ != JoinTableId.INVALID; }
   /// True if this consumes all of its right input before outputting any rows.
   abstract public boolean isBlockingJoinNode();
 
@@ -657,39 +659,43 @@ public abstract class JoinNode extends PlanNode {
       TQueryOptions queryOptions) {
     Preconditions.checkState(isBlockingJoinNode(), "Only blocking join nodes supported");
 
-    ExecPhaseResourceProfiles buildSideProfile =
-        getChild(1).computeTreeResourceProfiles(queryOptions);
     ExecPhaseResourceProfiles probeSideProfile =
         getChild(0).computeTreeResourceProfiles(queryOptions);
 
-    // The peak resource consumption of the build phase is either during the Open() of
-    // the build side or while we're doing the join build and calling GetNext() on the
-    // build side.
-    ResourceProfile buildPhaseProfile = buildSideProfile.duringOpenProfile.max(
-        buildSideProfile.postOpenProfile.sum(nodeResourceProfile_));
-
-    ResourceProfile finishedBuildProfile = nodeResourceProfile_;
-    if (this instanceof NestedLoopJoinNode) {
-      // These exec node implementations may hold references into the build side, which
-      // prevents closing of the build side in a timely manner. This means we have to
-      // count the post-open resource consumption of the build side in the same way as
-      // the other in-memory data structures.
-      // TODO: IMPALA-4179: remove this workaround
-      finishedBuildProfile = buildSideProfile.postOpenProfile.sum(nodeResourceProfile_);
-    }
-
-    // Peak resource consumption of this subtree during Open().
-    ResourceProfile duringOpenProfile;
-    if (queryOptions.getMt_dop() == 0) {
-      // The build and probe side can be open and therefore consume resources
-      // simultaneously when mt_dop = 0 because of the async build thread.
-      duringOpenProfile = buildPhaseProfile.sum(probeSideProfile.duringOpenProfile);
+    ResourceProfile buildPhaseProfile;
+    ResourceProfile finishedBuildProfile;
+    if (hasSeparateBuild()) {
+      // Memory consumption is accounted for in the build fragment, except for the probe
+      // buffers accounted for in nodeResourceProfile_.
+      buildPhaseProfile = nodeResourceProfile_;
+      finishedBuildProfile = nodeResourceProfile_;
     } else {
-      // Open() of the probe side happens after the build completes.
-      duringOpenProfile = buildPhaseProfile.max(
-          finishedBuildProfile.sum(probeSideProfile.duringOpenProfile));
+      ExecPhaseResourceProfiles buildSideProfile =
+          getChild(1).computeTreeResourceProfiles(queryOptions);
+      // The peak resource consumption of the build phase is either during the Open() of
+      // the build side or while we're doing the join build and calling GetNext() on the
+      // build side.
+      buildPhaseProfile = buildSideProfile.duringOpenProfile.max(
+          buildSideProfile.postOpenProfile.sum(nodeResourceProfile_));
+
+      finishedBuildProfile = nodeResourceProfile_;
+      if (this instanceof NestedLoopJoinNode) {
+        // These exec node implementations may hold references into the build side, which
+        // prevents closing of the build side in a timely manner. This means we have to
+        // count the post-open resource consumption of the build side in the same way as
+        // the other in-memory data structures.
+        // TODO: IMPALA-4179: remove this workaround
+        finishedBuildProfile =
+            buildSideProfile.postOpenProfile.sum(nodeResourceProfile_);
+      }
     }
 
+    // Compute peak resource consumption of this subtree during Open().
+    // The build and probe side can be open and therefore consume resources
+    // simultaneously when mt_dop = 0 because of the async build thread.
+    ResourceProfile duringOpenProfile =
+        buildPhaseProfile.sum(probeSideProfile.duringOpenProfile);
+
     // After Open(), the probe side remains open and the join build remain in memory.
     ResourceProfile probePhaseProfile =
         finishedBuildProfile.sum(probeSideProfile.postOpenProfile);
@@ -714,4 +720,39 @@ public abstract class JoinNode extends PlanNode {
       }
     }
   }
+
+  /** Helper to construct TJoinNode. */
+  protected TJoinNode joinNodeToThrift() {
+    TJoinNode result = new TJoinNode(joinOp_.toThrift());
+    List<TupleId> buildTupleIds = getChild(1).getTupleIds();
+    result.setBuild_tuples(new ArrayList<>(buildTupleIds.size()));
+    result.setNullable_build_tuples(new ArrayList<>(buildTupleIds.size()));
+    for (TupleId tid : buildTupleIds) {
+      result.addToBuild_tuples(tid.asInt());
+      result.addToNullable_build_tuples(getChild(1).getNullableTupleIds().contains(tid));
+    }
+    return result;
+  }
+
+  @Override
+  public void computeNodeResourceProfile(TQueryOptions queryOptions) {
+    Pair<ResourceProfile, ResourceProfile> profiles =
+        computeJoinResourceProfile(queryOptions);
+    if (hasSeparateBuild()) {
+      // All build resource consumption is accounted for in the separate builder.
+      nodeResourceProfile_ = profiles.first;
+    } else {
+      // Both build and profile resources are accounted for in the node.
+      nodeResourceProfile_ = profiles.first.combine(profiles.second);
+    }
+  }
+
+  /**
+   * Helper method to compute the resource requirements for the join that can be
+   * called from the builder or the join node. Returns a pair of the probe
+   * resource requirements and the build resource requirements.
+   * Does not modify the state of this node.
+   */
+  public abstract Pair<ResourceProfile, ResourceProfile> computeJoinResourceProfile(
+      TQueryOptions queryOptions);
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
index 6f2f4f9..da5d1ff 100644
--- a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
@@ -25,6 +25,7 @@ import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.JoinOperator;
 import org.apache.impala.common.ImpalaException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TNestedLoopJoinNode;
 import org.apache.impala.thrift.TPlanNode;
@@ -75,7 +76,8 @@ public class NestedLoopJoinNode extends JoinNode {
   }
 
   @Override
-  public void computeNodeResourceProfile(TQueryOptions queryOptions) {
+  public Pair<ResourceProfile, ResourceProfile> computeJoinResourceProfile(
+      TQueryOptions queryOptions) {
     long perInstanceMemEstimate;
     if (getChild(1).getCardinality() == -1 || getChild(1).getAvgRowSize() == -1
         || numNodes_ == 0) {
@@ -84,7 +86,10 @@ public class NestedLoopJoinNode extends JoinNode {
       perInstanceMemEstimate =
           (long) Math.ceil(getChild(1).cardinality_ * getChild(1).avgRowSize_);
     }
-    nodeResourceProfile_ = ResourceProfile.noReservation(perInstanceMemEstimate);
+    ResourceProfile buildProfile = ResourceProfile.noReservation(perInstanceMemEstimate);
+    // Memory requirements for the probe side are minimal - batches are just streamed
+    // through.
+    return Pair.create(ResourceProfile.noReservation(0), buildProfile);
   }
 
   @Override
@@ -118,10 +123,10 @@ public class NestedLoopJoinNode extends JoinNode {
   @Override
   protected void toThrift(TPlanNode msg) {
     msg.node_type = TPlanNodeType.NESTED_LOOP_JOIN_NODE;
-    msg.nested_loop_join_node = new TNestedLoopJoinNode();
-    msg.nested_loop_join_node.join_op = joinOp_.toThrift();
-    for (Expr e: otherJoinConjuncts_) {
-      msg.nested_loop_join_node.addToJoin_conjuncts(e.treeToThrift());
+    msg.join_node = joinNodeToThrift();
+    msg.join_node.nested_loop_join_node = new TNestedLoopJoinNode();
+    for (Expr e : otherJoinConjuncts_) {
+      msg.join_node.nested_loop_join_node.addToJoin_conjuncts(e.treeToThrift());
     }
   }
 
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 e5cd9b8..d0b83e9 100644
--- a/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
@@ -55,9 +55,10 @@ public class ParallelPlanner {
   public ParallelPlanner(PlannerContext ctx) { ctx_ = ctx; }
 
   /**
-   * Given a distributed plan, return list of plans ready for parallel execution.
-   * The last plan in the sequence materializes the query result, the preceding
-   * plans materialize the build sides of joins.
+   * Given a distributed plan, return list of plans ready for parallel execution. Each
+   * returned fragment has a PlanRootSink or JoinBuildSink as their sink. The first
+   * plan in the list materializes the query result and subsequent plans materialize
+   * the build sides of joins. Each plan appears before its dependencies in the list.
    * Assigns cohortId and planId for all fragments.
    * TODO: create class DistributedPlan with a PlanFragment member, so we don't
    * need to distinguish PlanFragment and Plan through comments?
@@ -163,6 +164,9 @@ public class ParallelPlanner {
     JoinBuildSink buildSink =
         new JoinBuildSink(joinTableIdGenerator_.getNextId(), join);
     join.setJoinTableId(buildSink.getJoinTableId());
+    // Filters will be produced by the build sink. Remove them from the join node since
+    // it now not responsible for producing them.
+    join.getRuntimeFilters().clear();
     // c'tor fixes up PlanNode.fragment_
     PlanFragment buildFragment = new PlanFragment(ctx_.getNextFragmentId(),
         join.getChild(1), join.getFragment().getDataPartition());
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 bdc1de5..fb23f0e 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -37,6 +37,8 @@ import org.apache.impala.thrift.TPlanFragmentTree;
 import org.apache.impala.thrift.TQueryOptions;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 
 /**
  * PlanFragments form a tree structure via their ExchangeNodes. A tree of fragments
@@ -48,11 +50,16 @@ import com.google.common.base.Preconditions;
  * plans that materialize intermediate results for a particular consumer plan
  * are grouped into a single cohort.
  *
- * A PlanFragment encapsulates the specific tree of execution nodes that
+ * A PlanFragment encapsulates the specific tree of execution nodes (PlanNodes) that
  * are used to produce the output of the plan fragment, as well as output exprs,
  * destination node, etc. If there are no output exprs, the full row that is
  * is produced by the plan root is marked as materialized.
  *
+ * PlanNode trees are connected across fragments where the parent fragment consumes the
+ * output of the child fragment. In this case the PlanNode and DataSink of the two
+ * fragments must match, e.g. ExchangeNode and DataStreamSink or a JoinNode and a
+ * JoinBuildSink.
+ *
  * A plan fragment can have one or many instances, each of which in turn is executed by
  * an individual node and the output sent to a specific instance of the destination
  * fragment (or, in the case of the root fragment, is materialized in some form).
@@ -64,13 +71,10 @@ import com.google.common.base.Preconditions;
  *
  * The sequence of calls is:
  * - c'tor
- * - assemble with getters, etc.
- * - finalize()
+ * - assemble with getters, etc. setSink() must be called so that the fragment has a sink.
+ * - finalizeExchanges()
+ * - computeResourceProfile()
  * - toThrift()
- *
- * TODO: the tree of PlanNodes is connected across fragment boundaries, which makes
- *   it impossible search for things within a fragment (using TreeNode functions);
- *   fix that
  */
 public class PlanFragment extends TreeNode<PlanFragment> {
   private final PlanFragmentId fragmentId_;
@@ -157,15 +161,30 @@ public class PlanFragment extends TreeNode<PlanFragment> {
    */
   public List<PlanNode> collectPlanNodes() {
     List<PlanNode> nodes = new ArrayList<>();
-    collectPlanNodesHelper(planRoot_, nodes);
+    collectPlanNodesHelper(planRoot_, Predicates.alwaysTrue(), nodes);
     return nodes;
   }
 
-  private void collectPlanNodesHelper(PlanNode root, List<PlanNode> nodes) {
+  /**
+   * Collect PlanNodes that belong to the exec tree of this fragment and for which
+   * 'predicate' is true. Collected nodes are added to 'node'. Nodes are cast to
+   * T.
+   */
+  public <T extends PlanNode> void collectPlanNodes(
+      Predicate<? super PlanNode> predicate, List<T> nodes) {
+    collectPlanNodesHelper(planRoot_, predicate, nodes);
+  }
+
+  @SuppressWarnings("unchecked")
+  private  <T extends PlanNode> void collectPlanNodesHelper(
+      PlanNode root, Predicate<? super PlanNode> predicate, List<T> nodes) {
     if (root == null) return;
-    nodes.add(root);
-    if (root instanceof ExchangeNode) return;
-    for (PlanNode child: root.getChildren()) collectPlanNodesHelper(child, nodes);
+    if (predicate.apply(root)) nodes.add((T)root);
+    for (PlanNode child: root.getChildren()) {
+      if (child.getFragment() == this) {
+        collectPlanNodesHelper(child, predicate, nodes);
+      }
+    }
   }
 
   /**
@@ -238,17 +257,19 @@ public class PlanFragment extends TreeNode<PlanFragment> {
    * runtime filters that are stored at the fragment level.
    */
   public void computeResourceProfile(Analyzer analyzer) {
+    Preconditions.checkState(sink_ != null);
     // Compute resource profiles for all plan nodes and sinks in the fragment.
     sink_.computeResourceProfile(analyzer.getQueryOptions());
     computeRuntimeFilterResources(analyzer);
 
-    if (sink_ instanceof JoinBuildSink) {
-      // Resource consumption of fragments with join build sinks is included in the
-      // parent fragment because the join node blocks waiting for the join build to
-      // finish - see JoinNode.computeTreeResourceProfiles().
-      perBackendResourceProfile_ = ResourceProfile.invalid();
-      perInstanceResourceProfile_ = ResourceProfile.invalid();
-      return;
+    perBackendInitialMemReservationTotalClaims_ =
+        consumedGlobalRuntimeFiltersMemReservationBytes_;
+    perInstanceInitialMemReservationTotalClaims_ =
+        sink_.getResourceProfile().getMinMemReservationBytes()
+        + producedRuntimeFiltersMemReservationBytes_;
+    for (PlanNode node: collectPlanNodes()) {
+      perInstanceInitialMemReservationTotalClaims_ +=
+          node.getNodeResourceProfile().getMinMemReservationBytes();
     }
 
     ExecPhaseResourceProfiles planTreeProfile =
@@ -270,15 +291,20 @@ public class PlanFragment extends TreeNode<PlanFragment> {
             .setMinMemReservationBytes(consumedGlobalRuntimeFiltersMemReservationBytes_)
             .setThreadReservation(0)
             .build();
-    perBackendInitialMemReservationTotalClaims_ =
-        consumedGlobalRuntimeFiltersMemReservationBytes_;
-    perInstanceInitialMemReservationTotalClaims_ =
-        sink_.getResourceProfile().getMinMemReservationBytes()
-        + producedRuntimeFiltersMemReservationBytes_;
-    for (PlanNode node : collectPlanNodes()) {
-      perInstanceInitialMemReservationTotalClaims_ +=
-          node.getNodeResourceProfile().getMinMemReservationBytes();
-    }
+    validateResourceProfiles();
+  }
+
+  /**
+   * Validates that the resource profiles for this PlanFragment are complete and valid,
+   * i.e. that computeResourceProfile() was called and that it filled out the profiles
+   * correctly. Raises an exception if an invariant is violated. */
+  private void validateResourceProfiles() {
+    Preconditions.checkState(perInstanceResourceProfile_.isValid());
+    Preconditions.checkState(perBackendResourceProfile_.isValid());
+    Preconditions.checkArgument(perInstanceInitialMemReservationTotalClaims_ > -1);
+    Preconditions.checkArgument(perBackendInitialMemReservationTotalClaims_ > -1);
+    Preconditions.checkArgument(producedRuntimeFiltersMemReservationBytes_ > -1);
+    Preconditions.checkArgument(consumedGlobalRuntimeFiltersMemReservationBytes_ > -1);
   }
 
   /**
@@ -289,6 +315,14 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   private void computeRuntimeFilterResources(Analyzer analyzer) {
     Map<RuntimeFilterId, RuntimeFilter> consumedFilters = new HashMap<>();
     Map<RuntimeFilterId, RuntimeFilter> producedFilters = new HashMap<>();
+    // Visit all sinks and nodes to identify filters produced or consumed by fragment.
+    sink_.computeResourceProfile(analyzer.getQueryOptions());
+    Preconditions.checkState(
+        sink_.getRuntimeFilters().isEmpty() || sink_ instanceof JoinBuildSink);
+    for (RuntimeFilter filter : sink_.getRuntimeFilters()) {
+      // Join build sinks are always runtime filter producers, not consumers.
+      producedFilters.put(filter.getFilterId(), filter);
+    }
     for (PlanNode node : collectPlanNodes()) {
       node.computeNodeResourceProfile(analyzer.getQueryOptions());
       boolean isFilterProducer = node instanceof JoinNode;
@@ -382,36 +416,26 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   }
 
   public TPlanFragment toThrift() {
+    validateResourceProfiles();
     TPlanFragment result = new TPlanFragment();
     result.setDisplay_name(fragmentId_.toString());
     if (planRoot_ != null) result.setPlan(planRoot_.treeToThrift());
     if (sink_ != null) result.setOutput_sink(sink_.toThrift());
     result.setPartition(dataPartition_.toThrift());
-    if (perInstanceResourceProfile_.isValid()) {
-      Preconditions.checkState(perBackendResourceProfile_.isValid());
-      Preconditions.checkArgument(perInstanceInitialMemReservationTotalClaims_ > -1);
-      Preconditions.checkArgument(perBackendInitialMemReservationTotalClaims_ > -1);
-      result.setInstance_min_mem_reservation_bytes(
-          perInstanceResourceProfile_.getMinMemReservationBytes());
-      result.setBackend_min_mem_reservation_bytes(
-          perBackendResourceProfile_.getMinMemReservationBytes());
-      result.setInstance_initial_mem_reservation_total_claims(
-          perInstanceInitialMemReservationTotalClaims_);
-      result.setBackend_initial_mem_reservation_total_claims(
-          perBackendInitialMemReservationTotalClaims_);
-      result.setProduced_runtime_filters_reservation_bytes(
-          producedRuntimeFiltersMemReservationBytes_);
-      result.setConsumed_runtime_filters_reservation_bytes(
-          consumedGlobalRuntimeFiltersMemReservationBytes_);
-      result.setThread_reservation(perInstanceResourceProfile_.getThreadReservation());
-    } else {
-      result.setBackend_min_mem_reservation_bytes(0);
-      result.setInstance_min_mem_reservation_bytes(0);
-      result.setBackend_initial_mem_reservation_total_claims(0);
-      result.setInstance_initial_mem_reservation_total_claims(0);
-      result.setProduced_runtime_filters_reservation_bytes(0);
-      result.setConsumed_runtime_filters_reservation_bytes(0);
-    }
+
+    result.setInstance_initial_mem_reservation_total_claims(
+        perInstanceInitialMemReservationTotalClaims_);
+    result.setBackend_initial_mem_reservation_total_claims(
+        perBackendInitialMemReservationTotalClaims_);
+    result.setProduced_runtime_filters_reservation_bytes(
+        producedRuntimeFiltersMemReservationBytes_);
+    result.setConsumed_runtime_filters_reservation_bytes(
+        consumedGlobalRuntimeFiltersMemReservationBytes_);
+    result.setInstance_min_mem_reservation_bytes(
+        perInstanceResourceProfile_.getMinMemReservationBytes());
+    result.setBackend_min_mem_reservation_bytes(
+        perBackendResourceProfile_.getMinMemReservationBytes());
+    result.setThread_reservation(perInstanceResourceProfile_.getThreadReservation());
     return result;
   }
 
@@ -491,10 +515,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
         "Per-Host Resources: " : "Per-Host Shared Resources: ";
     String perHostExplainString = null;
     String perInstanceExplainString = null;
-    if (sink_ instanceof JoinBuildSink) {
-      perHostExplainString = "included in parent fragment";
-      perInstanceExplainString = mt_dop == 0 ? null : "included in parent fragment";
-    } else if (mt_dop == 0) {
+    if (mt_dop == 0) {
       // There is no point separating out per-host and per-instance resources when there
       // is only a single instance per host so combine them together.
       ResourceProfile perHostProfile = getTotalPerBackendResourceProfile(mt_dop);
@@ -602,6 +623,29 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   }
 
   /**
+   * Return all fragments in the current plan (i.e. ancestors of this root reachable
+   * via exchanges but not via join builds).
+   * Only valid to call once all fragments have sinks created.
+   */
+  public List<PlanFragment> getFragmentsInPlanPreorder() {
+    List<PlanFragment> result = new ArrayList<>();
+    getFragmentsInPlanPreorderAux(result);
+    return result;
+  }
+
+  /**
+   * Helper for getFragmentsInPlanPreorder().
+   */
+  protected void getFragmentsInPlanPreorderAux(List<PlanFragment> result) {
+    result.add(this);
+    for (PlanFragment child: children_) {
+      if (child.getSink() instanceof DataStreamSink) {
+        child.getFragmentsInPlanPreorderAux(result);
+      }
+    }
+  }
+
+  /**
    * Verify that the tree of PlanFragments and their contained tree of
    * PlanNodes is constructed correctly.
    */
@@ -628,4 +672,12 @@ public class PlanFragment extends TreeNode<PlanFragment> {
 
     for (PlanFragment child: getChildren()) child.verifyTree();
   }
+
+  /// Returns a seed value to use when hashing tuples for nodes within this fragment.
+  /// Also see RuntimeState::fragment_hash_seed().
+  public int getHashSeed() {
+    // IMPALA-219: we should use different seeds for different fragment.
+    // We add one to prevent having a hash seed of 0.
+    return planRoot_.getId().asInt() + 1;
+  }
 }
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 cfde915..2fc883e 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -435,7 +435,8 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     return result;
   }
 
-  // Append a flattened version of this plan node, including all children, to 'container'.
+  // Append a flattened version of this plan node, including all children in the same
+  // fragment, to 'container'.
   private void treeToThriftHelper(TPlan container) {
     TPlanNode msg = new TPlanNode();
     msg.node_id = id_.asInt();
@@ -471,16 +472,15 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     }
     toThrift(msg);
     container.addToNodes(msg);
-    // For the purpose of the BE consider ExchangeNodes to have no children.
-    if (this instanceof ExchangeNode) {
-      msg.num_children = 0;
-      return;
-    } else {
-      msg.num_children = children_.size();
-      for (PlanNode child: children_) {
-        child.treeToThriftHelper(container);
-      }
+    // For the purpose of the BE consider cross-fragment children (i.e.
+    // ExchangeNodes and separated join builds) to have no children.
+    int numChildren = 0;
+    for (PlanNode child: children_) {
+      if (child.getFragment() != getFragment()) continue;
+      child.treeToThriftHelper(container);
+      ++numChildren;
     }
+    msg.num_children = numChildren;
   }
 
   /**
@@ -866,9 +866,14 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
 
   protected String getRuntimeFilterExplainString(
       boolean isBuildNode, TExplainLevel detailLevel) {
-    if (runtimeFilters_.isEmpty()) return "";
+    return getRuntimeFilterExplainString(runtimeFilters_, isBuildNode, id_, detailLevel);
+  }
+
+  public static String getRuntimeFilterExplainString(List<RuntimeFilter> filters,
+      boolean isBuildNode, PlanNodeId nodeId, TExplainLevel detailLevel) {
+    if (filters.isEmpty()) return "";
     List<String> filtersStr = new ArrayList<>();
-    for (RuntimeFilter filter: runtimeFilters_) {
+    for (RuntimeFilter filter: filters) {
       StringBuilder filterStr = new StringBuilder();
       filterStr.append(filter.getFilterId());
       if (detailLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
@@ -881,7 +886,7 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
         filterStr.append(filter.getSrcExpr().toSql());
       } else {
         filterStr.append(" -> ");
-        filterStr.append(filter.getTargetExpr(getId()).toSql());
+        filterStr.append(filter.getTargetExpr(nodeId).toSql());
       }
       filtersStr.add(filterStr.toString());
     }
diff --git a/fe/src/main/java/org/apache/impala/planner/Planner.java b/fe/src/main/java/org/apache/impala/planner/Planner.java
index 0647d51..47c36f0 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -112,7 +112,7 @@ public class Planner {
    *    that such an expr substitution during plan generation never fails. If it does,
    *    that typically means there is a bug in analysis, or a broken/missing smap.
    */
-  public List<PlanFragment> createPlan() throws ImpalaException {
+  private List<PlanFragment> createPlanFragments() throws ImpalaException {
     SingleNodePlanner singleNodePlanner = new SingleNodePlanner(ctx_);
     DistributedPlanner distributedPlanner = new DistributedPlanner(ctx_);
     PlanNode singleNodePlan = singleNodePlanner.createSingleNodePlan();
@@ -125,11 +125,9 @@ public class Planner {
     invertJoins(singleNodePlan, ctx_.isSingleNodeExec());
     singleNodePlan = useNljForSingularRowBuilds(singleNodePlan, ctx_.getRootAnalyzer());
 
-    // MT_DOP > 0 is not supported by default for plans with base table joins or table
-    // sinks: we only allow MT_DOP > 0 with such plans if --unlock_mt_dop=true is
-    // specified. We allow single node plans with mt_dop since there is no actual
-    // parallelism.
-    if (!ctx_.isSingleNodeExec() && ctx_.getQueryOptions().mt_dop > 0
+    // Parallel plans are not supported by default for plans with base table joins or
+    // table sinks: we only allow such plans if --unlock_mt_dop=true is specified.
+    if (useParallelPlan()
         && (!RuntimeEnv.INSTANCE.isTestEnv()
                || RuntimeEnv.INSTANCE.isMtDopValidationEnabled())
         && !BackendConfig.INSTANCE.isMtDopUnlocked()
@@ -258,26 +256,32 @@ public class Planner {
   }
 
   /**
-   * Return a list of plans, each represented by the root of their fragment trees.
-   * TODO: roll into createPlan()
+   * Return a list of plans, each represented by the root of their fragment trees. May
+   * return a single-node, distributed, or parallel plan depending on the query and
+   * configuration.
    */
-  public List<PlanFragment> createParallelPlans() throws ImpalaException {
-    Preconditions.checkState(ctx_.getQueryOptions().mt_dop > 0);
-    List<PlanFragment> distrPlan = createPlan();
+  public List<PlanFragment> createPlans() throws ImpalaException {
+    List<PlanFragment> distrPlan = createPlanFragments();
     Preconditions.checkNotNull(distrPlan);
-    List<PlanFragment> parallelPlans;
-    // TODO: IMPALA-4224: Parallel plans are not executable
-    if (RuntimeEnv.INSTANCE.isTestEnv()) {
-      ParallelPlanner planner = new ParallelPlanner(ctx_);
-      parallelPlans = planner.createPlans(distrPlan.get(0));
-    } else {
-      parallelPlans = Collections.singletonList(distrPlan.get(0));
+    if (!useParallelPlan()) {
+      return Collections.singletonList(distrPlan.get(0));
     }
+    ParallelPlanner planner = new ParallelPlanner(ctx_);
+    List<PlanFragment> parallelPlans = planner.createPlans(distrPlan.get(0));
     ctx_.getTimeline().markEvent("Parallel plans created");
     return parallelPlans;
   }
 
   /**
+   * Return true if we should generate a parallel plan for this query, based on the
+   * current mt_dop value and whether a single-node plan was chosen.
+   */
+  private boolean useParallelPlan() {
+    Preconditions.checkState(ctx_.getQueryOptions().isSetMt_dop());
+    return ctx_.getQueryOptions().mt_dop > 0 && !ctx_.isSingleNodeExec();
+  }
+
+  /**
    * Return combined explain string for all plan fragments.
    * Includes the estimated resource requirements from the request if set.
    * Uses a default level of EXTENDED, unless overriden by the
@@ -436,6 +440,7 @@ public class Planner {
       // instances run on all backends with max DOP, and can consume their peak resources
       // at the same time, i.e. that the query-wide peak resources is the sum of the
       // per-fragment-instance peak resources.
+      // TODO: IMPALA-9255: take into account parallel plan dependencies.
       maxPerHostPeakResources =
           maxPerHostPeakResources.sum(fragment.getTotalPerBackendResourceProfile(mtDop));
       // Coordinator has to have a copy of each of the runtime filters to perform filter
diff --git a/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java b/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java
index 3855c99..cc13e41 100644
--- a/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java
+++ b/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java
@@ -125,18 +125,21 @@ public class ResourceProfile {
     return output.toString();
   }
 
-  // Returns a profile with the max of each value in 'this' and 'other'.
+  // Returns a profile with the max of each aggregate value in 'this' and 'other'.
+  // Values which don't aggregate (like buffer sizes) are invalid in the result.
   public ResourceProfile max(ResourceProfile other) {
     if (!isValid()) return other;
     if (!other.isValid()) return this;
     return new ResourceProfile(true,
         Math.max(getMemEstimateBytes(), other.getMemEstimateBytes()),
         Math.max(getMinMemReservationBytes(), other.getMinMemReservationBytes()),
-        Math.max(getMaxMemReservationBytes(), other.getMaxMemReservationBytes()), -1, -1,
+        Math.max(getMaxMemReservationBytes(), other.getMaxMemReservationBytes()),
+        -1, -1,
         Math.max(getThreadReservation(), other.getThreadReservation()));
   }
 
-  // Returns a profile with the sum of each value in 'this' and 'other'.
+  // Returns a profile with the sum of each aggregate value in 'this' and 'other'.
+  // Values which don't aggregate (like buffer sizes) are invalid in the result.
   public ResourceProfile sum(ResourceProfile other) {
     if (!isValid()) return other;
     if (!other.isValid()) return this;
@@ -145,12 +148,29 @@ public class ResourceProfile {
         MathUtil.saturatingAdd(
             getMinMemReservationBytes(),other.getMinMemReservationBytes()),
         MathUtil.saturatingAdd(
+            getMaxMemReservationBytes(), other.getMaxMemReservationBytes()), -1, -1,
+        MathUtil.saturatingAdd(getThreadReservation(), other.getThreadReservation()));
+  }
+
+  // Returns a profile with the sum of each aggregate value in 'this' and 'other'.
+  // For buffer sizes, where summing the values doesn't make sense, returns the max.
+  public ResourceProfile combine(ResourceProfile other) {
+    if (!isValid()) return other;
+    if (!other.isValid()) return this;
+    return new ResourceProfile(true,
+        MathUtil.saturatingAdd(getMemEstimateBytes(), other.getMemEstimateBytes()),
+        MathUtil.saturatingAdd(
+            getMinMemReservationBytes(),other.getMinMemReservationBytes()),
+        MathUtil.saturatingAdd(
             getMaxMemReservationBytes(), other.getMaxMemReservationBytes()),
-        -1, -1,
+        Math.max(getSpillableBufferBytes(), other.getSpillableBufferBytes()),
+        Math.max(getMaxRowBufferBytes(), other.getMaxRowBufferBytes()),
         MathUtil.saturatingAdd(getThreadReservation(), other.getThreadReservation()));
   }
 
-  // Returns a profile with all values multiplied by 'factor'.
+  // Returns a profile with each aggregate value multiplied by 'factor'.
+  // For buffer sizes, where multiplying the values doesn't make sense, invalid values
+  // are present in the returned profile.
   public ResourceProfile multiply(int factor) {
     if (!isValid()) return this;
     return new ResourceProfile(true,
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 3e1431d..1144ca7 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -1355,13 +1355,13 @@ public class Frontend {
   private TPlanExecInfo createPlanExecInfo(PlanFragment planRoot, Planner planner,
       TQueryCtx queryCtx, TQueryExecRequest queryExecRequest) {
     TPlanExecInfo result = new TPlanExecInfo();
-    List<PlanFragment> fragments = planRoot.getNodesPreOrder();
+    List<PlanFragment> fragments = planRoot.getFragmentsInPlanPreorder();
 
     // collect ScanNodes
     List<ScanNode> scanNodes = Lists.newArrayList();
-    for (PlanFragment fragment: fragments) {
-      Preconditions.checkNotNull(fragment.getPlanRoot());
-      fragment.getPlanRoot().collect(Predicates.instanceOf(ScanNode.class), scanNodes);
+    for (PlanFragment fragment : fragments) {
+      fragment.collectPlanNodes(
+        Predicates.instanceOf(ScanNode.class), scanNodes);
     }
 
     // Set scan ranges/locations for scan nodes.
@@ -1410,25 +1410,13 @@ public class Frontend {
   private TQueryExecRequest createExecRequest(
       Planner planner, PlanCtx planCtx) throws ImpalaException {
     TQueryCtx queryCtx = planner.getQueryCtx();
-    AnalysisResult analysisResult = planner.getAnalysisResult();
-    boolean isMtExec = (analysisResult.isQueryStmt() || analysisResult.isDmlStmt())
-        && queryCtx.client_request.query_options.isSetMt_dop()
-        && queryCtx.client_request.query_options.mt_dop > 0;
-
-    List<PlanFragment> planRoots = Lists.newArrayList();
-    TQueryExecRequest result = new TQueryExecRequest();
-    if (isMtExec) {
-      LOG.trace("create mt plan");
-      planRoots.addAll(planner.createParallelPlans());
-    } else {
-      LOG.trace("create plan");
-      planRoots.add(planner.createPlan().get(0));
-    }
+    List<PlanFragment> planRoots = planner.createPlans();
     if (planCtx.planCaptureRequested()) {
       planCtx.plan_ = planRoots;
     }
 
     // Compute resource requirements of the final plans.
+    TQueryExecRequest result = new TQueryExecRequest();
     planner.computeResourceReqs(planRoots, queryCtx, result);
 
     // create per-plan exec info;
@@ -1445,7 +1433,7 @@ public class Frontend {
         queryCtx.client_request.query_options.isDisable_unsafe_spills()
           && queryCtx.isSetTables_missing_stats()
           && !queryCtx.tables_missing_stats.isEmpty()
-          && !analysisResult.getAnalyzer().hasPlanHints();
+          && !planner.getAnalysisResult().getAnalyzer().hasPlanHints();
     queryCtx.setDisable_spilling(disableSpilling);
 
     // assign fragment idx
diff --git a/shell/impala_client.py b/shell/impala_client.py
index aaccf61..6040925 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -589,9 +589,12 @@ class ImpalaClient(object):
     output.append(row)
     try:
       sender_idx = summary.exch_to_sender_map[idx]
-      # This is an exchange node, so the sender is a fragment root, and should be printed
-      # next.
-      self.build_summary_table(summary, sender_idx, True, indent_level, False, output)
+      # This is an exchange node or a join node with a separate builder, so the source
+      # is a fragment root, and should be printed next.
+      sender_indent_level = indent_level + node.num_children
+      sender_new_indent_level = node.num_children > 0
+      self.build_summary_table(
+          summary, sender_idx, True, sender_indent_level, sender_new_indent_level, output)
     except (KeyError, TypeError):
       # Fall through if idx not in map, or if exch_to_sender_map itself is not set
       pass
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 98c940a..6d097f2 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
@@ -58,12 +58,12 @@ PLAN-ROOT SINK
 |
 02:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: c_nationkey = n_nationkey
-|  runtime filters: RF000 <- n_nationkey
 |  row-size=327B cardinality=150.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: n_nationkey
+|  |  runtime filters: RF000 <- n_nationkey
 |  |
 |  04:EXCHANGE [HASH(n_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 4bafdf6..ee04783 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
@@ -21,19 +21,11 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
 02:NESTED LOOP JOIN [CROSS JOIN]
-|  join table id: 00
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1 row-size=0B cardinality=550.56K
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  |  Per-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
-|  JOIN BUILD
-|  |  join-table-id=00 plan-id=01 cohort-id=01
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |
-|  01:SCAN HDFS [functional_parquet.alltypestiny b]
+|--01:SCAN HDFS [functional_parquet.alltypestiny b]
 |     HDFS partitions=4/4 files=4 size=11.67KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
@@ -64,7 +56,7 @@ functional_parquet.alltypestiny b
 where a.id = b.id
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=28.94MB mem-reservation=2.95MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=34.94MB mem-reservation=2.95MB thread-reservation=1 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -76,7 +68,6 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
 02:HASH JOIN [INNER JOIN]
-|  hash-table-id=00
 |  hash predicates: a.id = b.id
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF000[bloom] <- b.id
@@ -84,15 +75,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=8B cardinality=742
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  |  Per-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
-|  JOIN BUILD
-|  |  join-table-id=00 plan-id=01 cohort-id=01
-|  |  build expressions: b.id
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |
-|  01:SCAN HDFS [functional_parquet.alltypestiny b]
+|--01:SCAN HDFS [functional_parquet.alltypestiny b]
 |     HDFS partitions=4/4 files=4 size=11.67KB
 |     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 de67fb9..4e53644 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
@@ -16,19 +16,11 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
 02:NESTED LOOP JOIN [CROSS JOIN]
-|  join table id: 00
 |  mem-estimate=2.00GB mem-reservation=0B thread-reservation=0
 |  tuple-ids=0,1 row-size=0B cardinality=unavailable
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  |  Per-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
-|  JOIN BUILD
-|  |  join-table-id=00 plan-id=01 cohort-id=01
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |
-|  01:SCAN HDFS [functional_parquet.alltypestiny b]
+|--01:SCAN HDFS [functional_parquet.alltypestiny b]
 |     HDFS partitions=4/4 files=4 size=11.67KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
@@ -59,7 +51,7 @@ functional_parquet.alltypestiny b
 where a.id = b.id
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=2.03GB mem-reservation=35.01MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=2.03GB mem-reservation=35.02MB thread-reservation=1 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -71,7 +63,6 @@ PLAN-ROOT SINK
 |  in pipelines: 03(GETNEXT), 00(OPEN)
 |
 02:HASH JOIN [INNER JOIN]
-|  hash-table-id=00
 |  hash predicates: a.id = b.id
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF000[bloom] <- b.id
@@ -79,15 +70,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=8B cardinality=unavailable
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
-|--F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  |  Per-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
-|  JOIN BUILD
-|  |  join-table-id=00 plan-id=01 cohort-id=01
-|  |  build expressions: b.id
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |
-|  01:SCAN HDFS [functional_parquet.alltypestiny b]
+|--01:SCAN HDFS [functional_parquet.alltypestiny b]
 |     HDFS partitions=4/4 files=4 size=11.67KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
index 0ea5f35..de2827d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -2536,8 +2536,8 @@ 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=7
-Per-Host Resource Estimates: Memory=480MB
+Max Per-Host Resource Reservation: Memory=103.00MB Threads=9
+Per-Host Resource Estimates: Memory=501MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey
 = o_orderkey
 
@@ -2553,23 +2553,22 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=57.20MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
 02:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = 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=0B mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=56.51MB 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
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- o_orderkey
+|  |  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
@@ -2712,8 +2711,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=7
-Per-Host Resource Estimates: Memory=480MB
+Max Per-Host Resource Reservation: Memory=103.00MB Threads=9
+Per-Host Resource Estimates: Memory=501MB
 Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN /* +shuffle */
 tpch.orders ON l_orderkey = o_orderkey
 
@@ -2729,23 +2728,22 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=57.20MB mem-reservation=35.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
 02:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = 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=0B mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=56.51MB 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
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- o_orderkey
+|  |  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
@@ -2871,7 +2869,7 @@ 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=5
+Max Per-Host Resource Reservation: Memory=32.00MB Threads=7
 Per-Host Resource Estimates: Memory=875MB
 Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders
 
@@ -2887,19 +2885,18 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=343.18MB mem-reservation=8.00MB thread-reservation=1
+Per-Instance Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  join table id: 00
-|  mem-estimate=244.49MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=255.18MB mem-reservation=0B thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=244.49MB mem-reservation=0B thread-reservation=0
 |  |
 |  03:EXCHANGE [BROADCAST]
 |  |  mem-estimate=10.68MB mem-reservation=0B thread-reservation=0
@@ -3645,8 +3642,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=257.75MB Threads=15
-Per-Host Resource Estimates: Memory=833MB
+Max Per-Host Resource Reservation: Memory=264.50MB Threads=21
+Per-Host Resource Estimates: Memory=879MB
 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
@@ -3668,7 +3665,8 @@ PLAN-ROOT SINK
 |  in pipelines: 14(GETNEXT), 05(GETNEXT), 08(GETNEXT)
 |
 F09:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=99.00MB mem-reservation=43.00MB thread-reservation=1 runtime-filters-memory=2.00MB
+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
 00:UNION
 |  pass-through-operands: 14
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -3679,18 +3677,17 @@ Per-Instance Resources: mem-estimate=99.00MB mem-reservation=43.00MB thread-rese
 |  |  hash-table-id=01
 |  |  hash predicates: l_orderkey = o_orderkey
 |  |  fk/pk conjuncts: l_orderkey = 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=0B mem-reservation=0B spill-buffer=128.00KB 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-Host Shared Resources: included in parent fragment
-|  |  |  Per-Instance Resources: included in parent fragment
+|  |  |  Per-Instance Resources: mem-estimate=9.64MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  |  build expressions: o_orderkey
-|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  runtime filters: RF004[bloom] <- o_orderkey
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  |  |
 |  |  17:EXCHANGE [HASH(o_orderkey)]
 |  |  |  mem-estimate=5.77MB mem-reservation=0B thread-reservation=0
@@ -3735,18 +3732,17 @@ Per-Instance Resources: mem-estimate=99.00MB mem-reservation=43.00MB thread-rese
 |  |  hash-table-id=00
 |  |  hash predicates: l_orderkey = o_orderkey
 |  |  fk/pk conjuncts: l_orderkey = 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=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
 |  |  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-Host Shared Resources: included in parent fragment
-|  |  |  Per-Instance Resources: included in parent fragment
+|  |  |  Per-Instance Resources: mem-estimate=26.25MB 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
-|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  runtime filters: RF002[bloom] <- o_orderkey
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |  |
 |  |  15:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=8.25MB mem-reservation=0B thread-reservation=0
@@ -3791,7 +3787,7 @@ Per-Instance Resources: mem-estimate=99.00MB mem-reservation=43.00MB thread-rese
 |  in pipelines: 01(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=31.33MB mem-reservation=20.88MB thread-reservation=1 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=27.46MB mem-reservation=17.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
@@ -3802,18 +3798,17 @@ Per-Instance Resources: mem-estimate=31.33MB mem-reservation=20.88MB thread-rese
 |  hash-table-id=02
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = 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=0B mem-reservation=0B spill-buffer=128.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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=9.64MB mem-reservation=3.88MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: o_orderkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- o_orderkey
+|  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
 |  |
 |  12:EXCHANGE [HASH(o_orderkey)]
 |  |  mem-estimate=5.77MB mem-reservation=0B thread-reservation=0
@@ -4152,8 +4147,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=264.88MB Threads=13
-Per-Host Resource Estimates: Memory=842MB
+Max Per-Host Resource Reservation: Memory=298.88MB Threads=19
+Per-Host Resource Estimates: Memory=924MB
 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)
@@ -4196,7 +4191,7 @@ Per-Instance Resources: mem-estimate=27.56MB mem-reservation=17.00MB thread-rese
 |  in pipelines: 02(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=57.63MB mem-reservation=47.44MB thread-reservation=1 runtime-filters-memory=3.00MB
+Per-Instance Resources: mem-estimate=27.12MB mem-reservation=17.00MB thread-reservation=1
 08:AGGREGATE [STREAMING]
 |  output: sum(l_quantity)
 |  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
@@ -4207,18 +4202,17 @@ Per-Instance Resources: mem-estimate=57.63MB mem-reservation=47.44MB thread-rese
 07:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: o_orderkey = l_orderkey
-|  runtime filters: RF000[bloom] <- l_orderkey
-|  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
 |  tuple-ids=2,1,0 row-size=100B cardinality=600.12K
 |  in pipelines: 02(GETNEXT), 14(OPEN)
 |
 |--F08:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-|  |  Per-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=28.16MB mem-reservation=19.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: l_orderkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- l_orderkey
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  14:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(l_quantity)
@@ -4256,18 +4250,17 @@ Per-Instance Resources: mem-estimate=57.63MB mem-reservation=47.44MB thread-rese
 |  hash-table-id=01
 |  hash predicates: o_custkey = c_custkey
 |  fk/pk conjuncts: o_custkey = 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=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=15.02MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: c_custkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF002[bloom] <- c_custkey
+|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |  mem-estimate=5.52MB mem-reservation=0B thread-reservation=0
@@ -4290,18 +4283,17 @@ Per-Instance Resources: mem-estimate=57.63MB mem-reservation=47.44MB thread-rese
 |  hash-table-id=02
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
-|  runtime filters: RF004[bloom] <- o_orderkey
-|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=28.20MB 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
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF004[bloom] <- o_orderkey
+|  |  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
@@ -5385,8 +5377,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=132.00MB Threads=11
-Per-Host Resource Estimates: Memory=402MB
+Max Per-Host Resource Reservation: Memory=141.50MB Threads=17
+Per-Host Resource Estimates: Memory=456MB
 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,
@@ -5406,57 +5398,54 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F04:PLAN FRAGMENT [HASH(t3.o_orderkey)] hosts=2 instances=4
-Per-Instance Resources: mem-estimate=34.27MB mem-reservation=28.50MB thread-reservation=1 runtime-filters-memory=3.00MB
+Per-Instance Resources: mem-estimate=10.68MB mem-reservation=0B thread-reservation=1
 06:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: t1.o_orderkey = t3.o_orderkey
 |  fk/pk conjuncts: t1.o_orderkey = 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=0B mem-reservation=0B spill-buffer=1.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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=23.77MB 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: t3.o_orderkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- t3.o_orderkey
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
 |  |
 |  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
-|  |  runtime filters: RF002[bloom] <- t3.o_orderkey
-|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB 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-Host Shared Resources: included in parent fragment
-|  |  |  Per-Instance Resources: included in parent fragment
+|  |  |  Per-Instance Resources: mem-estimate=15.27MB mem-reservation=9.50MB 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
-|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  runtime filters: RF002[bloom] <- t3.o_orderkey
+|  |  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB 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
-|  |  |  runtime filters: RF004[bloom] <- t4.o_orderkey
-|  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.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-Host Shared Resources: included in parent fragment
-|  |  |  |  Per-Instance Resources: included in parent fragment
+|  |  |  |  Per-Instance Resources: mem-estimate=11.52MB mem-reservation=5.75MB 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
-|  |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  |  runtime filters: RF004[bloom] <- t4.o_orderkey
+|  |  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
 |  |  |  |
 |  |  |  08:EXCHANGE [HASH(t4.o_orderkey)]
 |  |  |  |  mem-estimate=5.77MB mem-reservation=0B thread-reservation=0
@@ -5714,7 +5703,7 @@ 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=9
+Max Per-Host Resource Reservation: Memory=352.00KB Threads=15
 Per-Host Resource Estimates: Memory=311MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.nation t1 INNER
 JOIN (SELECT /* +straight_join */ t2.n_nationkey k2, k3, k4 FROM
@@ -5734,19 +5723,18 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-Per-Instance Resources: mem-estimate=97.57MB mem-reservation=32.00KB thread-reservation=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
-|  mem-estimate=71.53MB mem-reservation=0B thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=81.57MB mem-reservation=0B thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=71.53MB mem-reservation=0B thread-reservation=0
 |  |
 |  09:EXCHANGE [BROADCAST]
 |  |  mem-estimate=10.05MB mem-reservation=0B thread-reservation=0
@@ -5754,19 +5742,18 @@ Per-Instance Resources: mem-estimate=97.57MB mem-reservation=32.00KB thread-rese
 |  |  in pipelines: 01(GETNEXT)
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  Per-Instance Resources: mem-estimate=20.80MB mem-reservation=8.00KB thread-reservation=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
-|  |  mem-estimate=2.38MB mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  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-Host Shared Resources: included in parent fragment
-|  |  |  Per-Instance Resources: included in parent fragment
+|  |  |  Per-Instance Resources: mem-estimate=4.80MB mem-reservation=0B thread-reservation=1
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
-|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=2.38MB mem-reservation=0B thread-reservation=0
 |  |  |
 |  |  08:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=2.42MB mem-reservation=0B thread-reservation=0
@@ -5774,19 +5761,18 @@ Per-Instance Resources: mem-estimate=97.57MB mem-reservation=32.00KB thread-rese
 |  |  |  in pipelines: 02(GETNEXT)
 |  |  |
 |  |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  |  Per-Instance Resources: mem-estimate=16.18MB mem-reservation=8.00KB thread-reservation=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
-|  |  |  mem-estimate=78.12KB mem-reservation=0B thread-reservation=0
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  |  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-Host Shared Resources: included in parent fragment
-|  |  |  |  Per-Instance Resources: included in parent fragment
+|  |  |  |  Per-Instance Resources: mem-estimate=180.25KB mem-reservation=0B thread-reservation=1
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
-|  |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  |  mem-estimate=78.12KB mem-reservation=0B thread-reservation=0
 |  |  |  |
 |  |  |  07:EXCHANGE [BROADCAST]
 |  |  |  |  mem-estimate=102.12KB mem-reservation=0B thread-reservation=0
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 414fb6b..274085c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
@@ -57,8 +57,8 @@ 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=37.94MB Threads=5
-Per-Host Resource Estimates: Memory=97MB
+Max Per-Host Resource Reservation: Memory=38.94MB Threads=7
+Per-Host Resource Estimates: Memory=98MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.customer INNER
 JOIN tpch_parquet.nation ON c_nationkey = n_nationkey
 
@@ -74,23 +74,23 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-Per-Instance Resources: mem-estimate=26.94MB mem-reservation=18.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+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]
 |  hash-table-id=00
 |  hash predicates: c_nationkey = n_nationkey
 |  fk/pk conjuncts: c_nationkey = 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=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=2.95MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: n_nationkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- n_nationkey
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |
 |  03:EXCHANGE [BROADCAST]
 |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
@@ -177,8 +177,8 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=74.00MB thread-reserva
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=196.00MB Threads=7
-Per-Host Resource Estimates: Memory=365MB
+Max Per-Host Resource Reservation: Memory=196.00MB Threads=9
+Per-Host Resource Estimates: Memory=386MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.lineitem LEFT
 OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey
 
@@ -194,22 +194,21 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=56.20MB mem-reservation=34.00MB thread-reservation=1
+Per-Instance Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
 02:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
-|  mem-estimate=44.82MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=55.51MB mem-reservation=34.00MB thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: o_orderkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  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
@@ -311,8 +310,8 @@ 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=7
-Per-Host Resource Estimates: Memory=181MB
+Max Per-Host Resource Reservation: Memory=100.00MB Threads=9
+Per-Host Resource Estimates: Memory=202MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER
 JOIN /* +shuffle */ tpch_parquet.customer ON o_custkey = c_custkey
 
@@ -328,23 +327,22 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(o_custkey)] hosts=2 instances=4
-Per-Instance Resources: mem-estimate=20.24MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=10.68MB 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
-|  runtime filters: RF000[bloom] <- c_custkey
-|  mem-estimate=8.56MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB 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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=19.99MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: c_custkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- c_custkey
+|  |  mem-estimate=8.56MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |
 |  04:EXCHANGE [HASH(c_custkey)]
 |  |  mem-estimate=10.43MB mem-reservation=0B thread-reservation=0
@@ -441,8 +439,8 @@ 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=150.00MB Threads=5
-Per-Host Resource Estimates: Memory=210MB
+Max Per-Host Resource Reservation: Memory=151.00MB Threads=7
+Per-Host Resource Estimates: Memory=232MB
 Analyzed query: SELECT /* +straight_join */ * FROM tpch_parquet.orders INNER
 JOIN /* +broadcast */ tpch_parquet.customer ON o_custkey = c_custkey
 
@@ -458,23 +456,23 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-Per-Instance Resources: mem-estimate=75.23MB mem-reservation=59.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+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]
 |  hash-table-id=00
 |  hash predicates: o_custkey = c_custkey
 |  fk/pk conjuncts: o_custkey = 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=0B mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
 |  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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=45.67MB 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: c_custkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- c_custkey
+|  |  mem-estimate=34.23MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  |
 |  03:EXCHANGE [BROADCAST]
 |  |  mem-estimate=10.43MB mem-reservation=0B thread-reservation=0
@@ -565,7 +563,7 @@ 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=5
+Max Per-Host Resource Reservation: Memory=68.34MB Threads=7
 Per-Host Resource Estimates: Memory=4.06GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypestiny
@@ -584,22 +582,21 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT)
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=2.02GB mem-reservation=34.09MB thread-reservation=1
+Per-Instance Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=1
 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: alltypes.id = alltypestiny.id
 |  fk/pk conjuncts: assumed fk/pk
-|  mem-estimate=2.00GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  mem-estimate=0B mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
 |  tuple-ids=0,1N row-size=160B cardinality=unavailable
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  |  Per-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=2.00GB mem-reservation=34.00MB thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: alltypestiny.id
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  mem-estimate=2.00GB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
 |  |
 |  03:EXCHANGE [BROADCAST]
 |  |  mem-estimate=503.92KB mem-reservation=0B thread-reservation=0
@@ -825,8 +822,8 @@ Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservati
    tuple-ids=0 row-size=8B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=129.00MB Threads=9
-Per-Host Resource Estimates: Memory=399MB
+Max Per-Host Resource Reservation: Memory=129.00MB Threads=11
+Per-Host Resource Estimates: Memory=419MB
 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
@@ -859,7 +856,7 @@ Per-Instance Resources: mem-estimate=20.19MB mem-reservation=8.50MB thread-reser
 |  in pipelines: 00(GETNEXT)
 |
 F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=53.60MB mem-reservation=43.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=44.07MB mem-reservation=34.00MB thread-reservation=1
 03:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: l_orderkey, o_orderstatus
@@ -871,18 +868,17 @@ Per-Instance Resources: mem-estimate=53.60MB mem-reservation=43.50MB thread-rese
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = 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=0B mem-reservation=0B spill-buffer=512.00KB 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-Host Shared Resources: included in parent fragment
-|  |  Per-Instance Resources: included in parent fragment
+|  |  Per-Instance Resources: mem-estimate=19.60MB mem-reservation=9.50MB thread-reservation=1 runtime-filters-memory=1.00MB
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: o_orderkey
-|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  runtime filters: RF000[bloom] <- o_orderkey
+|  |  mem-estimate=8.50MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
 |  |
 |  05:EXCHANGE [HASH(o_orderkey)]
 |  |  mem-estimate=10.10MB mem-reservation=0B thread-reservation=0
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
index 5d1a427..773fd07 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=68.62MB Threads=11
-Per-Host Resource Estimates: Memory=166MB
+Max Per-Host Resource Reservation: Memory=69.62MB Threads=15
+Per-Host Resource Estimates: Memory=168MB
 PLAN-ROOT SINK
 |
 12:MERGING-EXCHANGE [UNPARTITIONED]
@@ -142,12 +142,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: store_sales.ss_sold_date_sk = dt.d_date_sk
-|  runtime filters: RF000 <- dt.d_date_sk
 |  row-size=72B cardinality=3.04K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: dt.d_date_sk
+|  |  runtime filters: RF000 <- dt.d_date_sk
 |  |
 |  09:EXCHANGE [HASH(dt.d_date_sk)]
 |  |
@@ -160,12 +160,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_item_sk = item.i_item_sk
-|  runtime filters: RF002 <- item.i_item_sk
 |  row-size=60B cardinality=3.04K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: item.i_item_sk
+|  |  runtime filters: RF002 <- item.i_item_sk
 |  |
 |  07:EXCHANGE [BROADCAST]
 |  |
@@ -342,8 +342,8 @@ 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=98.50MB Threads=15
-Per-Host Resource Estimates: Memory=405MB
+Max Per-Host Resource Reservation: Memory=99.50MB Threads=23
+Per-Host Resource Estimates: Memory=417MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -368,12 +368,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF000 <- i_item_sk
 |  row-size=162B cardinality=261.60K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF000 <- i_item_sk
 |  |
 |  15:EXCHANGE [BROADCAST]
 |  |
@@ -383,12 +383,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_promo_sk = p_promo_sk
-|  runtime filters: RF002 <- p_promo_sk
 |  row-size=126B cardinality=261.60K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: p_promo_sk
+|  |  runtime filters: RF002 <- p_promo_sk
 |  |
 |  14:EXCHANGE [BROADCAST]
 |  |
@@ -399,12 +399,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_cdemo_sk = cd_demo_sk
-|  runtime filters: RF004 <- cd_demo_sk
 |  row-size=96B cardinality=263.34K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: cd_demo_sk
+|  |  runtime filters: RF004 <- cd_demo_sk
 |  |
 |  13:EXCHANGE [HASH(cd_demo_sk)]
 |  |
@@ -417,12 +417,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF006 <- d_date_sk
 |  row-size=44B cardinality=589.03K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF006 <- d_date_sk
 |  |
 |  11:EXCHANGE [BROADCAST]
 |  |
@@ -664,8 +664,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> store_sales.ss_store_sk
    row-size=8B cardinality=84.40K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=91.27MB Threads=17
-Per-Host Resource Estimates: Memory=285MB
+Max Per-Host Resource Reservation: Memory=92.27MB Threads=25
+Per-Host Resource Estimates: Memory=288MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -690,23 +690,23 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: substr(store.s_zip, 1, 2) = substr(substr(ca_zip, 1, 5), 1, 2)
-|  runtime filters: RF000 <- substr(substr(ca_zip, 1, 5), 1, 2)
 |  row-size=65B cardinality=84.40K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: substr(substr(ca_zip, 1, 5), 1, 2)
+|  |  runtime filters: RF000 <- substr(substr(ca_zip, 1, 5), 1, 2)
 |  |
 |  18:EXCHANGE [BROADCAST]
 |  |
 |  07:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  |  hash predicates: substr(ca_zip, 1, 5) = substr(ca_zip, 1, 5)
-|  |  runtime filters: RF004 <- substr(ca_zip, 1, 5)
 |  |  row-size=20B cardinality=396
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: substr(ca_zip, 1, 5)
+|  |  |  runtime filters: RF004 <- substr(ca_zip, 1, 5)
 |  |  |
 |  |  16:AGGREGATE [FINALIZE]
 |  |  |  output: count:merge(*)
@@ -723,12 +723,12 @@ PLAN-ROOT SINK
 |  |  |
 |  |  04:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  |  hash predicates: customer_address.ca_address_sk = customer.c_current_addr_sk
-|  |  |  runtime filters: RF006 <- customer.c_current_addr_sk
 |  |  |  row-size=38B cardinality=51.30K
 |  |  |
 |  |  |--JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: customer.c_current_addr_sk
+|  |  |  |  runtime filters: RF006 <- customer.c_current_addr_sk
 |  |  |  |
 |  |  |  14:EXCHANGE [HASH(customer.c_current_addr_sk)]
 |  |  |  |
@@ -754,12 +754,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_store_sk = store.s_store_sk
-|  runtime filters: RF002 <- store.s_store_sk
 |  row-size=45B cardinality=84.40K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: store.s_store_sk
+|  |  runtime filters: RF002 <- store.s_store_sk
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -974,8 +974,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=105.27MB Threads=19
-Per-Host Resource Estimates: Memory=300MB
+Max Per-Host Resource Reservation: Memory=107.27MB Threads=29
+Per-Host Resource Estimates: Memory=302MB
 PLAN-ROOT SINK
 |
 22:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1001,12 +1001,12 @@ PLAN-ROOT SINK
 10:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
 |  other predicates: substr(ca_zip, 1, 5) != substr(s_zip, 1, 5)
-|  runtime filters: RF000 <- s_store_sk
 |  row-size=158B cardinality=24
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF000 <- s_store_sk
 |  |
 |  19:EXCHANGE [BROADCAST]
 |  |
@@ -1016,12 +1016,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: c_current_addr_sk = ca_address_sk
-|  runtime filters: RF002 <- ca_address_sk
 |  row-size=137B cardinality=24
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: ca_address_sk
+|  |  runtime filters: RF002 <- ca_address_sk
 |  |
 |  18:EXCHANGE [HASH(ca_address_sk)]
 |  |
@@ -1033,12 +1033,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_customer_sk = c_customer_sk
-|  runtime filters: RF004 <- c_customer_sk
 |  row-size=116B cardinality=24
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: c_customer_sk
+|  |  runtime filters: RF004 <- c_customer_sk
 |  |
 |  16:EXCHANGE [HASH(c_customer_sk)]
 |  |
@@ -1051,12 +1051,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF006 <- d_date_sk
 |  row-size=108B cardinality=24
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF006 <- d_date_sk
 |  |
 |  14:EXCHANGE [BROADCAST]
 |  |
@@ -1067,12 +1067,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF008 <- i_item_sk
 |  row-size=96B cardinality=873
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF008 <- i_item_sk
 |  |
 |  13:EXCHANGE [BROADCAST]
 |  |
@@ -1252,8 +1252,8 @@ 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=98.39MB Threads=15
-Per-Host Resource Estimates: Memory=405MB
+Max Per-Host Resource Reservation: Memory=99.39MB Threads=23
+Per-Host Resource Estimates: Memory=417MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1278,12 +1278,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF000 <- i_item_sk
 |  row-size=150B cardinality=263.34K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF000 <- i_item_sk
 |  |
 |  15:EXCHANGE [BROADCAST]
 |  |
@@ -1293,12 +1293,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF002 <- s_store_sk
 |  row-size=114B cardinality=263.34K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF002 <- s_store_sk
 |  |
 |  14:EXCHANGE [BROADCAST]
 |  |
@@ -1309,12 +1309,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_cdemo_sk = cd_demo_sk
-|  runtime filters: RF004 <- cd_demo_sk
 |  row-size=96B cardinality=263.34K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: cd_demo_sk
+|  |  runtime filters: RF004 <- cd_demo_sk
 |  |
 |  13:EXCHANGE [HASH(cd_demo_sk)]
 |  |
@@ -1327,12 +1327,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF006 <- d_date_sk
 |  row-size=44B cardinality=589.03K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF006 <- d_date_sk
 |  |
 |  11:EXCHANGE [BROADCAST]
 |  |
@@ -1529,8 +1529,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=86.52MB Threads=15
-Per-Host Resource Estimates: Memory=277MB
+Max Per-Host Resource Reservation: Memory=89.52MB Threads=23
+Per-Host Resource Estimates: Memory=280MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1543,12 +1543,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_customer_sk = c_customer_sk
-|  runtime filters: RF000 <- c_customer_sk
 |  row-size=88B cardinality=3.15K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: c_customer_sk
+|  |  runtime filters: RF000 <- c_customer_sk
 |  |
 |  17:EXCHANGE [HASH(c_customer_sk)]
 |  |
@@ -1573,12 +1573,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_store_sk = store.s_store_sk
-|  runtime filters: RF002 <- store.s_store_sk
 |  row-size=101B cardinality=31.53K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: store.s_store_sk
+|  |  runtime filters: RF002 <- store.s_store_sk
 |  |
 |  13:EXCHANGE [BROADCAST]
 |  |
@@ -1589,12 +1589,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
-|  runtime filters: RF004 <- date_dim.d_date_sk
 |  row-size=68B cardinality=31.53K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: date_dim.d_date_sk
+|  |  runtime filters: RF004 <- date_dim.d_date_sk
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -1605,12 +1605,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-|  runtime filters: RF006 <- household_demographics.hd_demo_sk
 |  row-size=56B cardinality=162.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: household_demographics.hd_demo_sk
+|  |  runtime filters: RF006 <- household_demographics.hd_demo_sk
 |  |
 |  11:EXCHANGE [BROADCAST]
 |  |
@@ -1742,8 +1742,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=67.62MB Threads=9
-Per-Host Resource Estimates: Memory=164MB
+Max Per-Host Resource Reservation: Memory=69.62MB Threads=13
+Per-Host Resource Estimates: Memory=166MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1768,12 +1768,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_sold_date_sk = dt.d_date_sk
-|  runtime filters: RF000 <- dt.d_date_sk
 |  row-size=62B cardinality=1.73K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: dt.d_date_sk
+|  |  runtime filters: RF000 <- dt.d_date_sk
 |  |
 |  08:EXCHANGE [BROADCAST]
 |  |
@@ -1784,12 +1784,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_item_sk = item.i_item_sk
-|  runtime filters: RF002 <- item.i_item_sk
 |  row-size=50B cardinality=29.12K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: item.i_item_sk
+|  |  runtime filters: RF002 <- item.i_item_sk
 |  |
 |  07:EXCHANGE [BROADCAST]
 |  |
@@ -1929,8 +1929,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=51.64MB Threads=9
-Per-Host Resource Estimates: Memory=164MB
+Max Per-Host Resource Reservation: Memory=53.64MB Threads=13
+Per-Host Resource Estimates: Memory=166MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1955,12 +1955,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF000 <- s_store_sk
 |  row-size=91B cardinality=589.03K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF000 <- s_store_sk
 |  |
 |  08:EXCHANGE [BROADCAST]
 |  |
@@ -1971,12 +1971,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF002 <- d_date_sk
 |  row-size=39B cardinality=589.03K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF002 <- d_date_sk
 |  |
 |  07:EXCHANGE [BROADCAST]
 |  |
@@ -2221,8 +2221,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=141.02MB Threads=19
-Per-Host Resource Estimates: Memory=359MB
+Max Per-Host Resource Reservation: Memory=145.02MB Threads=31
+Per-Host Resource Estimates: Memory=377MB
 PLAN-ROOT SINK
 |
 24:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2236,12 +2236,12 @@ PLAN-ROOT SINK
 13:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: customer.c_current_addr_sk = current_addr.ca_address_sk
 |  other predicates: current_addr.ca_city != ca_city
-|  runtime filters: RF000 <- current_addr.ca_address_sk
 |  row-size=138B cardinality=230.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: current_addr.ca_address_sk
+|  |  runtime filters: RF000 <- current_addr.ca_address_sk
 |  |
 |  23:EXCHANGE [BROADCAST]
 |  |
@@ -2251,12 +2251,12 @@ PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_customer_sk = c_customer_sk
-|  runtime filters: RF002 <- c_customer_sk
 |  row-size=113B cardinality=230.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: c_customer_sk
+|  |  runtime filters: RF002 <- c_customer_sk
 |  |
 |  22:EXCHANGE [HASH(c_customer_sk)]
 |  |
@@ -2281,12 +2281,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_addr_sk = customer_address.ca_address_sk
-|  runtime filters: RF004 <- customer_address.ca_address_sk
 |  row-size=107B cardinality=230.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: customer_address.ca_address_sk
+|  |  runtime filters: RF004 <- customer_address.ca_address_sk
 |  |
 |  18:EXCHANGE [BROADCAST]
 |  |
@@ -2296,12 +2296,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_store_sk = store.s_store_sk
-|  runtime filters: RF006 <- store.s_store_sk
 |  row-size=83B cardinality=230.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: store.s_store_sk
+|  |  runtime filters: RF006 <- store.s_store_sk
 |  |
 |  17:EXCHANGE [BROADCAST]
 |  |
@@ -2312,12 +2312,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
-|  runtime filters: RF008 <- date_dim.d_date_sk
 |  row-size=60B cardinality=230.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: date_dim.d_date_sk
+|  |  runtime filters: RF008 <- date_dim.d_date_sk
 |  |
 |  16:EXCHANGE [BROADCAST]
 |  |
@@ -2328,12 +2328,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-|  runtime filters: RF010 <- household_demographics.hd_demo_sk
 |  row-size=48B cardinality=702.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  build expressions: household_demographics.hd_demo_sk
+|  |  runtime filters: RF010 <- household_demographics.hd_demo_sk
 |  |
 |  15:EXCHANGE [BROADCAST]
 |  |
@@ -2464,8 +2464,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=67.62MB Threads=9
-Per-Host Resource Estimates: Memory=164MB
+Max Per-Host Resource Reservation: Memory=69.62MB Threads=13
+Per-Host Resource Estimates: Memory=167MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2490,12 +2490,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_sold_date_sk = dt.d_date_sk
-|  runtime filters: RF000 <- dt.d_date_sk
 |  row-size=72B cardinality=1.73K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: dt.d_date_sk
+|  |  runtime filters: RF000 <- dt.d_date_sk
 |  |
 |  08:EXCHANGE [BROADCAST]
 |  |
@@ -2506,12 +2506,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_item_sk = item.i_item_sk
-|  runtime filters: RF002 <- item.i_item_sk
 |  row-size=60B cardinality=29.12K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: item.i_item_sk
+|  |  runtime filters: RF002 <- item.i_item_sk
 |  |
 |  07:EXCHANGE [BROADCAST]
 |  |
@@ -2668,8 +2668,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=75.52MB Threads=13
-Per-Host Resource Estimates: Memory=204MB
+Max Per-Host Resource Reservation: Memory=76.52MB Threads=19
+Per-Host Resource Estimates: Memory=205MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2694,12 +2694,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF000 <- s_store_sk
 |  row-size=114B cardinality=96
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF000 <- s_store_sk
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -2709,12 +2709,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF002 <- d_date_sk
 |  row-size=110B cardinality=96
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF002 <- d_date_sk
 |  |
 |  11:EXCHANGE [HASH(d_date_sk)]
 |  |
@@ -2727,12 +2727,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF004 <- i_item_sk
 |  row-size=98B cardinality=481
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF004 <- i_item_sk
 |  |
 |  09:EXCHANGE [BROADCAST]
 |  |
@@ -2860,8 +2860,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=67.62MB Threads=9
-Per-Host Resource Estimates: Memory=164MB
+Max Per-Host Resource Reservation: Memory=69.62MB Threads=13
+Per-Host Resource Estimates: Memory=167MB
 PLAN-ROOT SINK
 |
 11:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2886,12 +2886,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF000 <- d_date_sk
 |  row-size=72B cardinality=1.73K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF000 <- d_date_sk
 |  |
 |  08:EXCHANGE [BROADCAST]
 |  |
@@ -2902,12 +2902,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF002 <- i_item_sk
 |  row-size=60B cardinality=29.12K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF002 <- i_item_sk
 |  |
 |  07:EXCHANGE [BROADCAST]
 |  |
@@ -3224,8 +3224,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=195.53MB Threads=23
-Per-Host Resource Estimates: Memory=476MB
+Max Per-Host Resource Reservation: Memory=197.53MB Threads=37
+Per-Host Resource Estimates: Memory=494MB
 PLAN-ROOT SINK
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3238,23 +3238,23 @@ PLAN-ROOT SINK
 |
 16:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: d_week_seq = d_week_seq - 52, s_store_id = s_store_id
-|  runtime filters: RF000 <- d_week_seq - 52, RF001 <- s_store_id
 |  row-size=336B cardinality=419.05M
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: d_week_seq - 52, s_store_id
+|  |  runtime filters: RF000 <- d_week_seq - 52, RF001 <- s_store_id
 |  |
 |  29:EXCHANGE [HASH(d_week_seq - 52,s_store_id)]
 |  |
 |  15:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_store_sk = s_store_sk
-|  |  runtime filters: RF010 <- s_store_sk
 |  |  row-size=160B cardinality=50.14K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: s_store_sk
+|  |  |  runtime filters: RF010 <- s_store_sk
 |  |  |
 |  |  27:EXCHANGE [BROADCAST]
 |  |  |
@@ -3264,12 +3264,12 @@ PLAN-ROOT SINK
 |  |
 |  14:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: d_week_seq = d.d_week_seq
-|  |  runtime filters: RF012 <- d.d_week_seq
 |  |  row-size=128B cardinality=50.14K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  build expressions: d.d_week_seq
+|  |  |  runtime filters: RF012 <- d.d_week_seq
 |  |  |
 |  |  26:EXCHANGE [BROADCAST]
 |  |  |
@@ -3292,12 +3292,12 @@ PLAN-ROOT SINK
 |  |
 |  10:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF014 <- d_date_sk
 |  |  row-size=39B cardinality=2.88M
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=02
 |  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF014 <- d_date_sk
 |  |  |
 |  |  23:EXCHANGE [BROADCAST]
 |  |  |
@@ -3315,12 +3315,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF004 <- s_store_sk
 |  row-size=176B cardinality=50.14K
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF004 <- s_store_sk
 |  |
 |  22:EXCHANGE [BROADCAST]
 |  |
@@ -3331,12 +3331,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: d_week_seq = d.d_week_seq
-|  runtime filters: RF006 <- d.d_week_seq
 |  row-size=128B cardinality=50.14K
 |
 |--JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  build expressions: d.d_week_seq
+|  |  runtime filters: RF006 <- d.d_week_seq
 |  |
 |  21:EXCHANGE [BROADCAST]
 |  |
@@ -3360,12 +3360,12 @@ PLAN-ROOT SINK
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF008 <- d_date_sk
 |  row-size=39B cardinality=2.88M
 |
 |--JOIN BUILD
 |  |  join-table-id=06 plan-id=07 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF008 <- d_date_sk
 |  |
 |  18:EXCHANGE [BROADCAST]
 |  |
@@ -3555,8 +3555,8 @@ 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=80.52MB Threads=15
-Per-Host Resource Estimates: Memory=224MB
+Max Per-Host Resource Reservation: Memory=81.52MB Threads=21
+Per-Host Resource Estimates: Memory=225MB
 PLAN-ROOT SINK
 |
 19:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3594,12 +3594,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF000 <- s_store_sk
 |  row-size=114B cardinality=6
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF000 <- s_store_sk
 |  |
 |  16:EXCHANGE [HASH(s_store_sk)]
 |  |
@@ -3611,12 +3611,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF002 <- d_date_sk
 |  row-size=110B cardinality=6
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF002 <- d_date_sk
 |  |
 |  14:EXCHANGE [HASH(d_date_sk)]
 |  |
@@ -3629,12 +3629,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF004 <- i_item_sk
 |  row-size=98B cardinality=92
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF004 <- i_item_sk
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -3898,8 +3898,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=149.02MB Threads=21
-Per-Host Resource Estimates: Memory=388MB
+Max Per-Host Resource Reservation: Memory=151.02MB Threads=31
+Per-Host Resource Estimates: Memory=394MB
 PLAN-ROOT SINK
 |
 27:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3913,12 +3913,12 @@ PLAN-ROOT SINK
 13:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = ss_store_sk
 |  other predicates: sum(ss_sales_price) <= 0.1 * avg(revenue)
-|  runtime filters: RF000 <- ss_store_sk
 |  row-size=225B cardinality=647.10K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: ss_store_sk
+|  |  runtime filters: RF000 <- ss_store_sk
 |  |
 |  26:EXCHANGE [BROADCAST]
 |  |
@@ -3948,12 +3948,12 @@ PLAN-ROOT SINK
 |  |
 |  04:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF008 <- d_date_sk
 |  |  row-size=28B cardinality=2.88M
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF008 <- d_date_sk
 |  |  |
 |  |  21:EXCHANGE [BROADCAST]
 |  |  |
@@ -3969,12 +3969,12 @@ PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF002 <- i_item_sk
 |  row-size=205B cardinality=107.85K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF002 <- i_item_sk
 |  |
 |  20:EXCHANGE [HASH(i_item_sk)]
 |  |
@@ -3986,12 +3986,12 @@ PLAN-ROOT SINK
 |
 11:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF004 <- s_store_sk
 |  row-size=48B cardinality=107.85K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF004 <- s_store_sk
 |  |
 |  18:EXCHANGE [BROADCAST]
 |  |
@@ -4014,12 +4014,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF006 <- d_date_sk
 |  row-size=28B cardinality=2.88M
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF006 <- d_date_sk
 |  |
 |  15:EXCHANGE [BROADCAST]
 |  |
@@ -4260,8 +4260,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=202.64MB Threads=17
-Per-Host Resource Estimates: Memory=399MB
+Max Per-Host Resource Reservation: Memory=206.64MB Threads=29
+Per-Host Resource Estimates: Memory=417MB
 PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4275,12 +4275,12 @@ PLAN-ROOT SINK
 13:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: customer.c_current_addr_sk = current_addr.ca_address_sk
 |  other predicates: current_addr.ca_city != ca_city
-|  runtime filters: RF000 <- current_addr.ca_address_sk
 |  row-size=154B cardinality=702.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: current_addr.ca_address_sk
+|  |  runtime filters: RF000 <- current_addr.ca_address_sk
 |  |
 |  22:EXCHANGE [BROADCAST]
 |  |
@@ -4290,12 +4290,12 @@ PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_customer_sk = c_customer_sk
-|  runtime filters: RF002 <- c_customer_sk
 |  row-size=129B cardinality=702.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: c_customer_sk
+|  |  runtime filters: RF002 <- c_customer_sk
 |  |
 |  21:EXCHANGE [BROADCAST]
 |  |
@@ -4318,12 +4318,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_addr_sk = customer_address.ca_address_sk
-|  runtime filters: RF004 <- customer_address.ca_address_sk
 |  row-size=125B cardinality=702.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: customer_address.ca_address_sk
+|  |  runtime filters: RF004 <- customer_address.ca_address_sk
 |  |
 |  18:EXCHANGE [BROADCAST]
 |  |
@@ -4333,12 +4333,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_store_sk = store.s_store_sk
-|  runtime filters: RF006 <- store.s_store_sk
 |  row-size=101B cardinality=702.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: store.s_store_sk
+|  |  runtime filters: RF006 <- store.s_store_sk
 |  |
 |  17:EXCHANGE [BROADCAST]
 |  |
@@ -4349,12 +4349,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
-|  runtime filters: RF008 <- date_dim.d_date_sk
 |  row-size=78B cardinality=702.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: date_dim.d_date_sk
+|  |  runtime filters: RF008 <- date_dim.d_date_sk
 |  |
 |  16:EXCHANGE [BROADCAST]
 |  |
@@ -4365,12 +4365,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-|  runtime filters: RF010 <- household_demographics.hd_demo_sk
 |  row-size=52B cardinality=702.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  build expressions: household_demographics.hd_demo_sk
+|  |  runtime filters: RF010 <- household_demographics.hd_demo_sk
 |  |
 |  15:EXCHANGE [BROADCAST]
 |  |
@@ -4559,8 +4559,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=86.52MB Threads=15
-Per-Host Resource Estimates: Memory=278MB
+Max Per-Host Resource Reservation: Memory=89.52MB Threads=23
+Per-Host Resource Estimates: Memory=282MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4573,12 +4573,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_customer_sk = c_customer_sk
-|  runtime filters: RF000 <- c_customer_sk
 |  row-size=88B cardinality=16.25K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: c_customer_sk
+|  |  runtime filters: RF000 <- c_customer_sk
 |  |
 |  17:EXCHANGE [HASH(c_customer_sk)]
 |  |
@@ -4603,12 +4603,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_store_sk = store.s_store_sk
-|  runtime filters: RF002 <- store.s_store_sk
 |  row-size=93B cardinality=162.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: store.s_store_sk
+|  |  runtime filters: RF002 <- store.s_store_sk
 |  |
 |  13:EXCHANGE [BROADCAST]
 |  |
@@ -4619,12 +4619,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
-|  runtime filters: RF004 <- date_dim.d_date_sk
 |  row-size=60B cardinality=162.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: date_dim.d_date_sk
+|  |  runtime filters: RF004 <- date_dim.d_date_sk
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -4634,12 +4634,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-|  runtime filters: RF006 <- household_demographics.hd_demo_sk
 |  row-size=56B cardinality=162.45K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: household_demographics.hd_demo_sk
+|  |  runtime filters: RF006 <- household_demographics.hd_demo_sk
 |  |
 |  11:EXCHANGE [BROADCAST]
 |  |
@@ -4833,8 +4833,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=82.77MB Threads=15
-Per-Host Resource Estimates: Memory=269MB
+Max Per-Host Resource Reservation: Memory=85.77MB Threads=23
+Per-Host Resource Estimates: Memory=275MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4847,12 +4847,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_customer_sk = c_customer_sk
-|  runtime filters: RF000 <- c_customer_sk
 |  row-size=106B cardinality=46.86K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: c_customer_sk
+|  |  runtime filters: RF000 <- c_customer_sk
 |  |
 |  17:EXCHANGE [HASH(c_customer_sk)]
 |  |
@@ -4876,12 +4876,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
-|  runtime filters: RF002 <- date_dim.d_date_sk
 |  row-size=101B cardinality=46.86K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: date_dim.d_date_sk
+|  |  runtime filters: RF002 <- date_dim.d_date_sk
 |  |
 |  13:EXCHANGE [BROADCAST]
 |  |
@@ -4892,12 +4892,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_store_sk = store.s_store_sk
-|  runtime filters: RF004 <- store.s_store_sk
 |  row-size=75B cardinality=46.86K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: store.s_store_sk
+|  |  runtime filters: RF004 <- store.s_store_sk
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -4908,12 +4908,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
-|  runtime filters: RF006 <- household_demographics.hd_demo_sk
 |  row-size=48B cardinality=281.17K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: household_demographics.hd_demo_sk
+|  |  runtime filters: RF006 <- household_demographics.hd_demo_sk
 |  |
 |  11:EXCHANGE [BROADCAST]
 |  |
@@ -5091,8 +5091,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=105.52MB Threads=11
-Per-Host Resource Estimates: Memory=234MB
+Max Per-Host Resource Reservation: Memory=108.52MB Threads=17
+Per-Host Resource Estimates: Memory=237MB
 PLAN-ROOT SINK
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -5130,12 +5130,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF000 <- s_store_sk
 |  row-size=145B cardinality=10.67K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF000 <- s_store_sk
 |  |
 |  14:EXCHANGE [BROADCAST]
 |  |
@@ -5145,12 +5145,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF002 <- d_date_sk
 |  row-size=106B cardinality=10.67K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF002 <- d_date_sk
 |  |
 |  13:EXCHANGE [BROADCAST]
 |  |
@@ -5161,12 +5161,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF004 <- i_item_sk
 |  row-size=94B cardinality=52.17K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF004 <- i_item_sk
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -5294,8 +5294,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=50.14MB Threads=9
-Per-Host Resource Estimates: Memory=176MB
+Max Per-Host Resource Reservation: Memory=53.14MB Threads=15
+Per-Host Resource Estimates: Memory=179MB
 PLAN-ROOT SINK
 |
 12:AGGREGATE [FINALIZE]
@@ -5310,12 +5310,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss.ss_store_sk = s.s_store_sk
-|  runtime filters: RF000 <- s.s_store_sk
 |  row-size=52B cardinality=2.27K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: s.s_store_sk
+|  |  runtime filters: RF000 <- s.s_store_sk
 |  |
 |  10:EXCHANGE [BROADCAST]
 |  |
@@ -5326,12 +5326,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss.ss_hdemo_sk = hd.hd_demo_sk
-|  runtime filters: RF002 <- hd.hd_demo_sk
 |  row-size=32B cardinality=6.82K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: hd.hd_demo_sk
+|  |  runtime filters: RF002 <- hd.hd_demo_sk
 |  |
 |  09:EXCHANGE [BROADCAST]
 |  |
@@ -5342,12 +5342,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss.ss_sold_time_sk = td.t_time_sk
-|  runtime filters: RF004 <- td.t_time_sk
 |  row-size=24B cardinality=69.82K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: td.t_time_sk
+|  |  runtime filters: RF004 <- td.t_time_sk
 |  |
 |  08:EXCHANGE [BROADCAST]
 |  |
@@ -5507,8 +5507,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=11
-Per-Host Resource Estimates: Memory=201MB
+Max Per-Host Resource Reservation: Memory=113.75MB Threads=15
+Per-Host Resource Estimates: Memory=202MB
 PLAN-ROOT SINK
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
@@ -5542,12 +5542,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF000 <- d_date_sk
 |  row-size=232B cardinality=29.80K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF000 <- d_date_sk
 |  |
 |  11:EXCHANGE [BROADCAST]
 |  |
@@ -5558,12 +5558,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF002 <- i_item_sk
 |  row-size=206B cardinality=29.80K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF002 <- i_item_sk
 |  |
 |  10:EXCHANGE [HASH(i_item_sk)]
 |  |
@@ -5822,8 +5822,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=173.88MB Threads=22
-Per-Host Resource Estimates: Memory=430MB
+Max Per-Host Resource Reservation: Memory=176.88MB Threads=34
+Per-Host Resource Estimates: Memory=439MB
 PLAN-ROOT SINK
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
@@ -5850,12 +5850,12 @@ PLAN-ROOT SINK
 14:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  hash predicates: i.i_category = j.i_category
 |  other join predicates: i.i_current_price > 1.2 * avg(j.i_current_price)
-|  runtime filters: RF000 <- j.i_category
 |  row-size=80B cardinality=1.19K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: j.i_category
+|  |  runtime filters: RF000 <- j.i_category
 |  |
 |  27:EXCHANGE [BROADCAST]
 |  |
@@ -5877,12 +5877,12 @@ PLAN-ROOT SINK
 |
 13:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  hash predicates: d.d_month_seq = (d_month_seq)
-|  runtime filters: RF002 <- (d_month_seq)
 |  row-size=80B cardinality=1.19K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: (d_month_seq)
+|  |  runtime filters: RF002 <- (d_month_seq)
 |  |
 |  24:EXCHANGE [BROADCAST]
 |  |
@@ -5907,12 +5907,12 @@ PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s.ss_item_sk = i.i_item_sk
-|  runtime filters: RF004 <- i.i_item_sk
 |  row-size=80B cardinality=2.88M
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: i.i_item_sk
+|  |  runtime filters: RF004 <- i.i_item_sk
 |  |
 |  20:EXCHANGE [BROADCAST]
 |  |
@@ -5923,12 +5923,12 @@ PLAN-ROOT SINK
 |
 11:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s.ss_sold_date_sk = d.d_date_sk
-|  runtime filters: RF006 <- d.d_date_sk
 |  row-size=50B cardinality=2.88M
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: d.d_date_sk
+|  |  runtime filters: RF006 <- d.d_date_sk
 |  |
 |  19:EXCHANGE [BROADCAST]
 |  |
@@ -5939,12 +5939,12 @@ PLAN-ROOT SINK
 |
 10:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c.c_current_addr_sk = a.ca_address_sk
-|  runtime filters: RF008 <- a.ca_address_sk
 |  row-size=42B cardinality=2.88M
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: a.ca_address_sk
+|  |  runtime filters: RF008 <- a.ca_address_sk
 |  |
 |  18:EXCHANGE [BROADCAST]
 |  |
@@ -5954,12 +5954,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s.ss_customer_sk = c.c_customer_sk
-|  runtime filters: RF010 <- c.c_customer_sk
 |  row-size=24B cardinality=2.88M
 |
 |--JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  build expressions: c.c_customer_sk
+|  |  runtime filters: RF010 <- c.c_customer_sk
 |  |
 |  17:EXCHANGE [BROADCAST]
 |  |
@@ -6425,8 +6425,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=746.80MB Threads=33
-Per-Host Resource Estimates: Memory=1.03GB
+Max Per-Host Resource Reservation: Memory=755.80MB Threads=55
+Per-Host Resource Estimates: Memory=1.07GB
 PLAN-ROOT SINK
 |
 54:MERGING-EXCHANGE [UNPARTITIONED]
@@ -6472,12 +6472,12 @@ PLAN-ROOT SINK
 |  |
 |  29:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_store_sk = s_store_sk
-|  |  runtime filters: RF012 <- s_store_sk
 |  |  row-size=125B cardinality=685.36K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: s_store_sk
+|  |  |  runtime filters: RF012 <- s_store_sk
 |  |  |
 |  |  49:EXCHANGE [BROADCAST]
 |  |  |
@@ -6487,12 +6487,12 @@ PLAN-ROOT SINK
 |  |
 |  28:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_item_sk = i_item_sk
-|  |  runtime filters: RF014 <- i_item_sk
 |  |  row-size=86B cardinality=685.36K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF014 <- i_item_sk
 |  |  |
 |  |  48:EXCHANGE [BROADCAST]
 |  |  |
@@ -6502,12 +6502,12 @@ PLAN-ROOT SINK
 |  |
 |  27:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF016 <- d_date_sk
 |  |  row-size=32B cardinality=685.36K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=02
 |  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF016 <- d_date_sk
 |  |  |
 |  |  47:EXCHANGE [BROADCAST]
 |  |  |
@@ -6571,12 +6571,12 @@ PLAN-ROOT SINK
 |  |
 |  06:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_store_sk = s_store_sk
-|  |  runtime filters: RF006 <- s_store_sk
 |  |  row-size=125B cardinality=685.36K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=03
 |  |  |  build expressions: s_store_sk
+|  |  |  runtime filters: RF006 <- s_store_sk
 |  |  |
 |  |  43:EXCHANGE [BROADCAST]
 |  |  |
@@ -6586,12 +6586,12 @@ PLAN-ROOT SINK
 |  |
 |  05:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_item_sk = i_item_sk
-|  |  runtime filters: RF008 <- i_item_sk
 |  |  row-size=86B cardinality=685.36K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=06 plan-id=07 cohort-id=03
 |  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF008 <- i_item_sk
 |  |  |
 |  |  42:EXCHANGE [BROADCAST]
 |  |  |
@@ -6601,12 +6601,12 @@ PLAN-ROOT SINK
 |  |
 |  04:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF010 <- d_date_sk
 |  |  row-size=32B cardinality=685.36K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=03
 |  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF010 <- d_date_sk
 |  |  |
 |  |  41:EXCHANGE [BROADCAST]
 |  |  |
@@ -6645,12 +6645,12 @@ PLAN-ROOT SINK
 |
 19:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF000 <- s_store_sk
 |  row-size=125B cardinality=685.36K
 |
 |--JOIN BUILD
 |  |  join-table-id=08 plan-id=09 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF000 <- s_store_sk
 |  |
 |  38:EXCHANGE [BROADCAST]
 |  |
@@ -6660,12 +6660,12 @@ PLAN-ROOT SINK
 |
 18:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF002 <- i_item_sk
 |  row-size=86B cardinality=685.36K
 |
 |--JOIN BUILD
 |  |  join-table-id=09 plan-id=10 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF002 <- i_item_sk
 |  |
 |  37:EXCHANGE [BROADCAST]
 |  |
@@ -6675,12 +6675,12 @@ PLAN-ROOT SINK
 |
 17:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF004 <- d_date_sk
 |  row-size=32B cardinality=685.36K
 |
 |--JOIN BUILD
 |  |  join-table-id=10 plan-id=11 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF004 <- d_date_sk
 |  |
 |  36:EXCHANGE [BROADCAST]
 |  |
@@ -7052,8 +7052,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=232.78MB Threads=32
-Per-Host Resource Estimates: Memory=617MB
+Max Per-Host Resource Reservation: Memory=238.78MB Threads=55
+Per-Host Resource Estimates: Memory=627MB
 PLAN-ROOT SINK
 |
 27:SORT
@@ -7081,12 +7081,12 @@ PLAN-ROOT SINK
 |  |
 |  24:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: c_current_addr_sk = ca_address_sk
-|  |  runtime filters: RF012 <- ca_address_sk
 |  |  row-size=86B cardinality=3.30K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: ca_address_sk
+|  |  |  runtime filters: RF012 <- ca_address_sk
 |  |  |
 |  |  42:EXCHANGE [BROADCAST]
 |  |  |
@@ -7097,12 +7097,12 @@ PLAN-ROOT SINK
 |  |
 |  23:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash predicates: ss_customer_sk = c_customer_sk
-|  |  runtime filters: RF014 <- c_customer_sk
 |  |  row-size=78B cardinality=17.06K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  build expressions: c_customer_sk
+|  |  |  runtime filters: RF014 <- c_customer_sk
 |  |  |
 |  |  41:EXCHANGE [HASH(c_customer_sk)]
 |  |  |
@@ -7130,12 +7130,12 @@ PLAN-ROOT SINK
 |  |
 |  21:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_item_sk = i_item_sk
-|  |  runtime filters: RF018 <- i_item_sk
 |  |  row-size=62B cardinality=17.06K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=02
 |  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF018 <- i_item_sk
 |  |  |
 |  |  38:EXCHANGE [BROADCAST]
 |  |  |
@@ -7146,12 +7146,12 @@ PLAN-ROOT SINK
 |  |
 |  20:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_date_sk = d_date_sk
-|  |  runtime filters: RF020 <- d_date_sk
 |  |  row-size=36B cardinality=170.55K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=02
 |  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF020 <- d_date_sk
 |  |  |
 |  |  37:EXCHANGE [BROADCAST]
 |  |  |
@@ -7177,12 +7177,12 @@ PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c_current_addr_sk = ca_address_sk
-|  runtime filters: RF000 <- ca_address_sk
 |  row-size=133B cardinality=3.28K
 |
 |--JOIN BUILD
 |  |  join-table-id=06 plan-id=07 cohort-id=01
 |  |  build expressions: ca_address_sk
+|  |  runtime filters: RF000 <- ca_address_sk
 |  |
 |  34:EXCHANGE [BROADCAST]
 |  |
@@ -7193,12 +7193,12 @@ PLAN-ROOT SINK
 |
 11:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: ss_customer_sk = c_customer_sk
-|  runtime filters: RF002 <- c_customer_sk
 |  row-size=125B cardinality=16.94K
 |
 |--JOIN BUILD
 |  |  join-table-id=07 plan-id=08 cohort-id=01
 |  |  build expressions: c_customer_sk
+|  |  runtime filters: RF002 <- c_customer_sk
 |  |
 |  33:EXCHANGE [HASH(c_customer_sk)]
 |  |
@@ -7211,12 +7211,12 @@ PLAN-ROOT SINK
 |
 10:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_store_sk = s_store_sk
-|  runtime filters: RF004 <- s_store_sk
 |  row-size=117B cardinality=16.94K
 |
 |--JOIN BUILD
 |  |  join-table-id=08 plan-id=09 cohort-id=01
 |  |  build expressions: s_store_sk
+|  |  runtime filters: RF004 <- s_store_sk
 |  |
 |  31:EXCHANGE [BROADCAST]
 |  |
@@ -7227,12 +7227,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_promo_sk = p_promo_sk
-|  runtime filters: RF006 <- p_promo_sk
 |  row-size=109B cardinality=16.94K
 |
 |--JOIN BUILD
 |  |  join-table-id=09 plan-id=10 cohort-id=01
 |  |  build expressions: p_promo_sk
+|  |  runtime filters: RF006 <- p_promo_sk
 |  |
 |  30:EXCHANGE [BROADCAST]
 |  |
@@ -7243,12 +7243,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i_item_sk
-|  runtime filters: RF008 <- i_item_sk
 |  row-size=66B cardinality=17.06K
 |
 |--JOIN BUILD
 |  |  join-table-id=10 plan-id=11 cohort-id=01
 |  |  build expressions: i_item_sk
+|  |  runtime filters: RF008 <- i_item_sk
 |  |
 |  29:EXCHANGE [BROADCAST]
 |  |
@@ -7259,12 +7259,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_date_sk = d_date_sk
-|  runtime filters: RF010 <- d_date_sk
 |  row-size=40B cardinality=170.55K
 |
 |--JOIN BUILD
 |  |  join-table-id=11 plan-id=12 cohort-id=01
 |  |  build expressions: d_date_sk
+|  |  runtime filters: RF010 <- d_date_sk
 |  |
 |  28:EXCHANGE [BROADCAST]
 |  |
@@ -8126,8 +8126,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=373.12MB Threads=72
-Per-Host Resource Estimates: Memory=1.34GB
+Max Per-Host Resource Reservation: Memory=383.12MB Threads=127
+Per-Host Resource Estimates: Memory=1.36GB
 PLAN-ROOT SINK
 |
 70:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
@@ -8181,12 +8181,12 @@ PLAN-ROOT SINK
 |  |
 |  60:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_time_sk = time_dim.t_time_sk
-|  |  runtime filters: RF046 <- time_dim.t_time_sk
 |  |  row-size=24B cardinality=69.82K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=02
 |  |  |  build expressions: time_dim.t_time_sk
+|  |  |  runtime filters: RF046 <- time_dim.t_time_sk
 |  |  |
 |  |  112:EXCHANGE [BROADCAST]
 |  |  |
@@ -8251,12 +8251,12 @@ PLAN-ROOT SINK
 |  |
 |  52:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_time_sk = time_dim.t_time_sk
-|  |  runtime filters: RF040 <- time_dim.t_time_sk
 |  |  row-size=24B cardinality=69.82K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=03
 |  |  |  build expressions: time_dim.t_time_sk
+|  |  |  runtime filters: RF040 <- time_dim.t_time_sk
 |  |  |
 |  |  106:EXCHANGE [BROADCAST]
 |  |  |
@@ -8321,12 +8321,12 @@ PLAN-ROOT SINK
 |  |
 |  44:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_time_sk = time_dim.t_time_sk
-|  |  runtime filters: RF034 <- time_dim.t_time_sk
 |  |  row-size=24B cardinality=69.82K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=11 plan-id=12 cohort-id=04
 |  |  |  build expressions: time_dim.t_time_sk
+|  |  |  runtime filters: RF034 <- time_dim.t_time_sk
 |  |  |
 |  |  100:EXCHANGE [BROADCAST]
 |  |  |
@@ -8391,12 +8391,12 @@ PLAN-ROOT SINK
 |  |
 |  36:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_time_sk = time_dim.t_time_sk
-|  |  runtime filters: RF028 <- time_dim.t_time_sk
 |  |  row-size=24B cardinality=69.82K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=15 plan-id=16 cohort-id=05
 |  |  |  build expressions: time_dim.t_time_sk
+|  |  |  runtime filters: RF028 <- time_dim.t_time_sk
 |  |  |
 |  |  94:EXCHANGE [BROADCAST]
 |  |  |
@@ -8461,12 +8461,12 @@ PLAN-ROOT SINK
 |  |
 |  28:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_time_sk = time_dim.t_time_sk
-|  |  runtime filters: RF022 <- time_dim.t_time_sk
 |  |  row-size=24B cardinality=69.82K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=19 plan-id=20 cohort-id=06
 |  |  |  build expressions: time_dim.t_time_sk
+|  |  |  runtime filters: RF022 <- time_dim.t_time_sk
 |  |  |
 |  |  88:EXCHANGE [BROADCAST]
 |  |  |
@@ -8531,12 +8531,12 @@ PLAN-ROOT SINK
 |  |
 |  20:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_time_sk = time_dim.t_time_sk
-|  |  runtime filters: RF016 <- time_dim.t_time_sk
 |  |  row-size=24B cardinality=69.82K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=23 plan-id=24 cohort-id=07
 |  |  |  build expressions: time_dim.t_time_sk
+|  |  |  runtime filters: RF016 <- time_dim.t_time_sk
 |  |  |
 |  |  82:EXCHANGE [BROADCAST]
 |  |  |
@@ -8586,12 +8586,12 @@ PLAN-ROOT SINK
 |  |
 |  13:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_hdemo_sk = household_demographics.hd_demo_sk
-|  |  runtime filters: RF008 <- household_demographics.hd_demo_sk
 |  |  row-size=36B cardinality=6.82K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=26 plan-id=27 cohort-id=08
 |  |  |  build expressions: household_demographics.hd_demo_sk
+|  |  |  runtime filters: RF008 <- household_demographics.hd_demo_sk
 |  |  |
 |  |  77:EXCHANGE [BROADCAST]
 |  |  |
@@ -8602,12 +8602,12 @@ PLAN-ROOT SINK
 |  |
 |  12:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ss_sold_time_sk = time_dim.t_time_sk
-|  |  runtime filters: RF010 <- time_dim.t_time_sk
 |  |  row-size=24B cardinality=69.82K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=27 plan-id=28 cohort-id=08
 |  |  |  build expressions: time_dim.t_time_sk
+|  |  |  runtime filters: RF010 <- time_dim.t_time_sk
 |  |  |
 |  |  76:EXCHANGE [BROADCAST]
 |  |  |
@@ -8648,12 +8648,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_hdemo_sk = household_demographics.hd_demo_sk
-|  runtime filters: RF002 <- household_demographics.hd_demo_sk
 |  row-size=36B cardinality=6.82K
 |
 |--JOIN BUILD
 |  |  join-table-id=29 plan-id=30 cohort-id=01
 |  |  build expressions: household_demographics.hd_demo_sk
+|  |  runtime filters: RF002 <- household_demographics.hd_demo_sk
 |  |
 |  72:EXCHANGE [BROADCAST]
 |  |
@@ -8664,12 +8664,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_sold_time_sk = time_dim.t_time_sk
-|  runtime filters: RF004 <- time_dim.t_time_sk
 |  row-size=24B cardinality=69.82K
 |
 |--JOIN BUILD
 |  |  join-table-id=30 plan-id=31 cohort-id=01
 |  |  build expressions: time_dim.t_time_sk
+|  |  runtime filters: RF004 <- time_dim.t_time_sk
 |  |
 |  71:EXCHANGE [BROADCAST]
 |  |
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
index 9ff4342..f4313b0 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=118.81MB Threads=23
-Per-Host Resource Estimates: Memory=709MB
+Max Per-Host Resource Reservation: Memory=122.81MB Threads=39
+Per-Host Resource Estimates: Memory=717MB
 PLAN-ROOT SINK
 |
 30:MERGING-EXCHANGE [UNPARTITIONED]
@@ -382,23 +382,23 @@ PLAN-ROOT SINK
 |
 17:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  hash predicates: ps_partkey = p_partkey, min(ps_supplycost) = ps_supplycost
-|  runtime filters: RF000 <- p_partkey
 |  row-size=325B cardinality=1.01K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: p_partkey, ps_supplycost
+|  |  runtime filters: RF000 <- p_partkey
 |  |
 |  29:EXCHANGE [HASH(p_partkey,ps_supplycost)]
 |  |
 |  16:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: n_regionkey = r_regionkey
-|  |  runtime filters: RF010 <- r_regionkey
 |  |  row-size=325B cardinality=1.01K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: r_regionkey
+|  |  |  runtime filters: RF010 <- r_regionkey
 |  |  |
 |  |  27:EXCHANGE [BROADCAST]
 |  |  |
@@ -409,12 +409,12 @@ PLAN-ROOT SINK
 |  |
 |  15:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: s_nationkey = n_nationkey
-|  |  runtime filters: RF012 <- n_nationkey
 |  |  row-size=304B cardinality=5.05K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  build expressions: n_nationkey
+|  |  |  runtime filters: RF012 <- n_nationkey
 |  |  |
 |  |  26:EXCHANGE [BROADCAST]
 |  |  |
@@ -425,23 +425,23 @@ PLAN-ROOT SINK
 |  |
 |  14:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: s_suppkey = ps_suppkey
-|  |  runtime filters: RF014 <- ps_suppkey
 |  |  row-size=281B cardinality=5.05K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=02
 |  |  |  build expressions: ps_suppkey
+|  |  |  runtime filters: RF014 <- ps_suppkey
 |  |  |
 |  |  25:EXCHANGE [BROADCAST]
 |  |  |
 |  |  13:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  |  hash predicates: ps_partkey = p_partkey
-|  |  |  runtime filters: RF016 <- p_partkey
 |  |  |  row-size=95B cardinality=5.05K
 |  |  |
 |  |  |--JOIN BUILD
 |  |  |  |  join-table-id=04 plan-id=05 cohort-id=03
 |  |  |  |  build expressions: p_partkey
+|  |  |  |  runtime filters: RF016 <- p_partkey
 |  |  |  |
 |  |  |  24:EXCHANGE [BROADCAST]
 |  |  |  |
@@ -476,12 +476,12 @@ PLAN-ROOT SINK
 |
 11:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: n_regionkey = r_regionkey
-|  runtime filters: RF004 <- r_regionkey
 |  row-size=59B cardinality=160.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  build expressions: r_regionkey
+|  |  runtime filters: RF004 <- r_regionkey
 |  |
 |  21:EXCHANGE [BROADCAST]
 |  |
@@ -492,12 +492,12 @@ PLAN-ROOT SINK
 |
 10:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s_nationkey = n_nationkey
-|  runtime filters: RF006 <- n_nationkey
 |  row-size=38B cardinality=800.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=06 plan-id=07 cohort-id=01
 |  |  build expressions: n_nationkey
+|  |  runtime filters: RF006 <- n_nationkey
 |  |
 |  20:EXCHANGE [BROADCAST]
 |  |
@@ -508,12 +508,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ps_suppkey = s_suppkey
-|  runtime filters: RF008 <- s_suppkey
 |  row-size=34B cardinality=800.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=07 plan-id=08 cohort-id=01
 |  |  build expressions: s_suppkey
+|  |  runtime filters: RF008 <- s_suppkey
 |  |
 |  19:EXCHANGE [BROADCAST]
 |  |
@@ -648,8 +648,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> l_orderkey
    row-size=46B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=71.50MB Threads=13
-Per-Host Resource Estimates: Memory=495MB
+Max Per-Host Resource Reservation: Memory=71.50MB Threads=17
+Per-Host Resource Estimates: Memory=503MB
 PLAN-ROOT SINK
 |
 13:MERGING-EXCHANGE [UNPARTITIONED]
@@ -674,12 +674,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: o_custkey = c_custkey
-|  runtime filters: RF000 <- c_custkey
 |  row-size=117B cardinality=17.56K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: c_custkey
+|  |  runtime filters: RF000 <- c_custkey
 |  |
 |  10:EXCHANGE [HASH(c_custkey)]
 |  |
@@ -692,12 +692,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF002 <- o_orderkey
 |  row-size=88B cardinality=57.58K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF002 <- o_orderkey
 |  |
 |  08:EXCHANGE [HASH(o_orderkey)]
 |  |
@@ -811,8 +811,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=9
-Per-Host Resource Estimates: Memory=445MB
+Max Per-Host Resource Reservation: Memory=72.62MB Threads=11
+Per-Host Resource Estimates: Memory=453MB
 PLAN-ROOT SINK
 |
 09:MERGING-EXCHANGE [UNPARTITIONED]
@@ -836,12 +836,12 @@ PLAN-ROOT SINK
 |
 02:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF000 <- o_orderkey
 |  row-size=50B cardinality=150.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF000 <- o_orderkey
 |  |
 |  06:EXCHANGE [HASH(o_orderkey)]
 |  |
@@ -1043,8 +1043,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=135.28MB Threads=15
-Per-Host Resource Estimates: Memory=627MB
+Max Per-Host Resource Reservation: Memory=137.28MB Threads=25
+Per-Host Resource Estimates: Memory=644MB
 PLAN-ROOT SINK
 |
 20:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1068,12 +1068,12 @@ PLAN-ROOT SINK
 |
 10:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: n_regionkey = r_regionkey
-|  runtime filters: RF000 <- r_regionkey
 |  row-size=134B cardinality=115.16K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: r_regionkey
+|  |  runtime filters: RF000 <- r_regionkey
 |  |
 |  17:EXCHANGE [BROADCAST]
 |  |
@@ -1084,12 +1084,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s_nationkey = n_nationkey
-|  runtime filters: RF002 <- n_nationkey
 |  row-size=113B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: n_nationkey
+|  |  runtime filters: RF002 <- n_nationkey
 |  |
 |  16:EXCHANGE [BROADCAST]
 |  |
@@ -1100,12 +1100,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c_nationkey = s_nationkey, l_suppkey = s_suppkey
-|  runtime filters: RF004 <- s_nationkey, RF005 <- s_suppkey
 |  row-size=90B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: s_nationkey, s_suppkey
+|  |  runtime filters: RF004 <- s_nationkey, RF005 <- s_suppkey
 |  |
 |  15:EXCHANGE [BROADCAST]
 |  |
@@ -1116,12 +1116,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: o_custkey = c_custkey
-|  runtime filters: RF008 <- c_custkey
 |  row-size=80B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: c_custkey
+|  |  runtime filters: RF008 <- c_custkey
 |  |
 |  14:EXCHANGE [BROADCAST]
 |  |
@@ -1132,12 +1132,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF010 <- o_orderkey
 |  row-size=70B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF010 <- o_orderkey
 |  |
 |  13:EXCHANGE [BROADCAST]
 |  |
@@ -1414,8 +1414,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=17
-Per-Host Resource Estimates: Memory=650MB
+Max Per-Host Resource Reservation: Memory=139.78MB Threads=27
+Per-Host Resource Estimates: Memory=673MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1440,12 +1440,12 @@ PLAN-ROOT SINK
 10:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c_nationkey = n2.n_nationkey
 |  other predicates: ((n1.n_name = 'FRANCE' AND n2.n_name = 'GERMANY') OR (n1.n_name = 'GERMANY' AND n2.n_name = 'FRANCE'))
-|  runtime filters: RF000 <- n2.n_nationkey
 |  row-size=132B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: n2.n_nationkey
+|  |  runtime filters: RF000 <- n2.n_nationkey
 |  |
 |  18:EXCHANGE [BROADCAST]
 |  |
@@ -1455,12 +1455,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s_nationkey = n1.n_nationkey
-|  runtime filters: RF002 <- n1.n_nationkey
 |  row-size=111B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: n1.n_nationkey
+|  |  runtime filters: RF002 <- n1.n_nationkey
 |  |
 |  17:EXCHANGE [BROADCAST]
 |  |
@@ -1470,12 +1470,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: o_custkey = c_custkey
-|  runtime filters: RF004 <- c_custkey
 |  row-size=90B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: c_custkey
+|  |  runtime filters: RF004 <- c_custkey
 |  |
 |  16:EXCHANGE [BROADCAST]
 |  |
@@ -1486,12 +1486,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_suppkey = s_suppkey
-|  runtime filters: RF006 <- s_suppkey
 |  row-size=80B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: s_suppkey
+|  |  runtime filters: RF006 <- s_suppkey
 |  |
 |  15:EXCHANGE [BROADCAST]
 |  |
@@ -1502,12 +1502,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF008 <- o_orderkey
 |  row-size=70B cardinality=575.77K
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF008 <- o_orderkey
 |  |
 |  14:EXCHANGE [HASH(o_orderkey)]
 |  |
@@ -1770,8 +1770,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=135.05MB Threads=25
-Per-Host Resource Estimates: Memory=718MB
+Max Per-Host Resource Reservation: Memory=136.05MB Threads=39
+Per-Host Resource Estimates: Memory=724MB
 PLAN-ROOT SINK
 |
 29:MERGING-EXCHANGE [UNPARTITIONED]
@@ -1795,12 +1795,12 @@ PLAN-ROOT SINK
 |
 14:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s_nationkey = n2.n_nationkey
-|  runtime filters: RF000 <- n2.n_nationkey
 |  row-size=184B cardinality=761
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: n2.n_nationkey
+|  |  runtime filters: RF000 <- n2.n_nationkey
 |  |
 |  26:EXCHANGE [BROADCAST]
 |  |
@@ -1810,12 +1810,12 @@ PLAN-ROOT SINK
 |
 13:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: n1.n_regionkey = r_regionkey
-|  runtime filters: RF002 <- r_regionkey
 |  row-size=163B cardinality=761
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: r_regionkey
+|  |  runtime filters: RF002 <- r_regionkey
 |  |
 |  25:EXCHANGE [BROADCAST]
 |  |
@@ -1826,12 +1826,12 @@ PLAN-ROOT SINK
 |
 12:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c_nationkey = n1.n_nationkey
-|  runtime filters: RF004 <- n1.n_nationkey
 |  row-size=143B cardinality=3.81K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: n1.n_nationkey
+|  |  runtime filters: RF004 <- n1.n_nationkey
 |  |
 |  24:EXCHANGE [BROADCAST]
 |  |
@@ -1842,12 +1842,12 @@ PLAN-ROOT SINK
 |
 11:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: o_custkey = c_custkey
-|  runtime filters: RF006 <- c_custkey
 |  row-size=139B cardinality=3.81K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: c_custkey
+|  |  runtime filters: RF006 <- c_custkey
 |  |
 |  23:EXCHANGE [HASH(c_custkey)]
 |  |
@@ -1860,12 +1860,12 @@ PLAN-ROOT SINK
 |
 10:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_suppkey = s_suppkey
-|  runtime filters: RF008 <- s_suppkey
 |  row-size=129B cardinality=3.81K
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: s_suppkey
+|  |  runtime filters: RF008 <- s_suppkey
 |  |
 |  21:EXCHANGE [HASH(s_suppkey)]
 |  |
@@ -1878,12 +1878,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF010 <- o_orderkey
 |  row-size=119B cardinality=3.81K
 |
 |--JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF010 <- o_orderkey
 |  |
 |  19:EXCHANGE [HASH(o_orderkey)]
 |  |
@@ -1897,12 +1897,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_partkey = p_partkey
-|  runtime filters: RF012 <- p_partkey
 |  row-size=81B cardinality=39.66K
 |
 |--JOIN BUILD
 |  |  join-table-id=06 plan-id=07 cohort-id=01
 |  |  build expressions: p_partkey
+|  |  runtime filters: RF012 <- p_partkey
 |  |
 |  17:EXCHANGE [BROADCAST]
 |  |
@@ -2104,8 +2104,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> l_partkey, RF003 -> l_suppkey, RF006 -> l_suppkey, RF008 -> l_orderkey, RF010 -> l_partkey
    row-size=48B cardinality=6.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=164.89MB Threads=19
-Per-Host Resource Estimates: Memory=823MB
+Max Per-Host Resource Reservation: Memory=165.89MB Threads=29
+Per-Host Resource Estimates: Memory=867MB
 PLAN-ROOT SINK
 |
 22:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2129,12 +2129,12 @@ PLAN-ROOT SINK
 |
 10:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s_nationkey = n_nationkey
-|  runtime filters: RF000 <- n_nationkey
 |  row-size=186B cardinality=574.29K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: n_nationkey
+|  |  runtime filters: RF000 <- n_nationkey
 |  |
 |  19:EXCHANGE [BROADCAST]
 |  |
@@ -2144,12 +2144,12 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_partkey = ps_partkey, l_suppkey = ps_suppkey
-|  runtime filters: RF002 <- ps_partkey, RF003 <- ps_suppkey
 |  row-size=165B cardinality=574.29K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: ps_partkey, ps_suppkey
+|  |  runtime filters: RF002 <- ps_partkey, RF003 <- ps_suppkey
 |  |
 |  18:EXCHANGE [HASH(ps_partkey,ps_suppkey)]
 |  |
@@ -2161,12 +2161,12 @@ PLAN-ROOT SINK
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_suppkey = s_suppkey
-|  runtime filters: RF006 <- s_suppkey
 |  row-size=141B cardinality=574.29K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: s_suppkey
+|  |  runtime filters: RF006 <- s_suppkey
 |  |
 |  16:EXCHANGE [BROADCAST]
 |  |
@@ -2177,12 +2177,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF008 <- o_orderkey
 |  row-size=131B cardinality=574.29K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF008 <- o_orderkey
 |  |
 |  15:EXCHANGE [HASH(o_orderkey)]
 |  |
@@ -2194,12 +2194,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_partkey = p_partkey
-|  runtime filters: RF010 <- p_partkey
 |  row-size=101B cardinality=598.58K
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: p_partkey
+|  |  runtime filters: RF010 <- p_partkey
 |  |
 |  13:EXCHANGE [BROADCAST]
 |  |
@@ -2366,8 +2366,8 @@ PLAN-ROOT SINK
    runtime filters: RF004 -> l_orderkey
    row-size=37B cardinality=2.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=161.89MB Threads=13
-Per-Host Resource Estimates: Memory=605MB
+Max Per-Host Resource Reservation: Memory=162.89MB Threads=19
+Per-Host Resource Estimates: Memory=628MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2392,12 +2392,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c_nationkey = n_nationkey
-|  runtime filters: RF000 <- n_nationkey
 |  row-size=293B cardinality=191.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: n_nationkey
+|  |  runtime filters: RF000 <- n_nationkey
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -2407,12 +2407,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: o_custkey = c_custkey
-|  runtime filters: RF002 <- c_custkey
 |  row-size=272B cardinality=191.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: c_custkey
+|  |  runtime filters: RF002 <- c_custkey
 |  |
 |  11:EXCHANGE [HASH(c_custkey)]
 |  |
@@ -2425,12 +2425,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF004 <- o_orderkey
 |  row-size=75B cardinality=191.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF004 <- o_orderkey
 |  |
 |  09:EXCHANGE [BROADCAST]
 |  |
@@ -2650,8 +2650,8 @@ PLAN-ROOT SINK
    runtime filters: RF002 -> ps_suppkey
    row-size=28B cardinality=800.00K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=85.41MB Threads=16
-Per-Host Resource Estimates: Memory=588MB
+Max Per-Host Resource Reservation: Memory=87.41MB Threads=26
+Per-Host Resource Estimates: Memory=591MB
 PLAN-ROOT SINK
 |
 23:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2683,12 +2683,12 @@ PLAN-ROOT SINK
 |  |
 |  10:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: s_nationkey = n_nationkey
-|  |  runtime filters: RF004 <- n_nationkey
 |  |  row-size=51B cardinality=32.00K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: n_nationkey
+|  |  |  runtime filters: RF004 <- n_nationkey
 |  |  |
 |  |  19:EXCHANGE [BROADCAST]
 |  |  |
@@ -2699,12 +2699,12 @@ PLAN-ROOT SINK
 |  |
 |  09:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: ps_suppkey = s_suppkey
-|  |  runtime filters: RF006 <- s_suppkey
 |  |  row-size=30B cardinality=800.00K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  build expressions: s_suppkey
+|  |  |  runtime filters: RF006 <- s_suppkey
 |  |  |
 |  |  18:EXCHANGE [BROADCAST]
 |  |  |
@@ -2732,12 +2732,12 @@ PLAN-ROOT SINK
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: s_nationkey = n_nationkey
-|  runtime filters: RF000 <- n_nationkey
 |  row-size=59B cardinality=32.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: n_nationkey
+|  |  runtime filters: RF000 <- n_nationkey
 |  |
 |  15:EXCHANGE [BROADCAST]
 |  |
@@ -2748,12 +2748,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ps_suppkey = s_suppkey
-|  runtime filters: RF002 <- s_suppkey
 |  row-size=38B cardinality=800.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: s_suppkey
+|  |  runtime filters: RF002 <- s_suppkey
 |  |
 |  14:EXCHANGE [BROADCAST]
 |  |
@@ -2868,8 +2868,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=9
-Per-Host Resource Estimates: Memory=473MB
+Max Per-Host Resource Reservation: Memory=100.88MB Threads=11
+Per-Host Resource Estimates: Memory=493MB
 PLAN-ROOT SINK
 |
 09:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2893,12 +2893,12 @@ PLAN-ROOT SINK
 |
 02:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF000 <- o_orderkey
 |  row-size=119B cardinality=320.78K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF000 <- o_orderkey
 |  |
 |  06:EXCHANGE [HASH(o_orderkey)]
 |  |
@@ -3017,8 +3017,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=9
-Per-Host Resource Estimates: Memory=333MB
+Max Per-Host Resource Reservation: Memory=68.38MB Threads=11
+Per-Host Resource Estimates: Memory=335MB
 PLAN-ROOT SINK
 |
 10:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3047,12 +3047,12 @@ PLAN-ROOT SINK
 |
 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
 |  hash predicates: o_custkey = c_custkey
-|  runtime filters: RF000 <- c_custkey
 |  row-size=85B cardinality=150.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: c_custkey
+|  |  runtime filters: RF000 <- c_custkey
 |  |
 |  07:EXCHANGE [HASH(c_custkey)]
 |  |
@@ -3140,8 +3140,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=7
-Per-Host Resource Estimates: Memory=297MB
+Max Per-Host Resource Reservation: Memory=40.75MB Threads=9
+Per-Host Resource Estimates: Memory=313MB
 PLAN-ROOT SINK
 |
 07:AGGREGATE [FINALIZE]
@@ -3156,12 +3156,12 @@ PLAN-ROOT SINK
 |
 02:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_partkey = p_partkey
-|  runtime filters: RF000 <- p_partkey
 |  row-size=87B cardinality=598.58K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: p_partkey
+|  |  runtime filters: RF000 <- p_partkey
 |  |
 |  05:EXCHANGE [HASH(p_partkey)]
 |  |
@@ -3329,8 +3329,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=12
-Per-Host Resource Estimates: Memory=529MB
+Max Per-Host Resource Reservation: Memory=86.50MB Threads=16
+Per-Host Resource Estimates: Memory=531MB
 PLAN-ROOT SINK
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3379,12 +3379,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_suppkey = s_suppkey
-|  runtime filters: RF000 <- s_suppkey
 |  row-size=126B cardinality=10.00K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: s_suppkey
+|  |  runtime filters: RF000 <- s_suppkey
 |  |
 |  11:EXCHANGE [HASH(s_suppkey)]
 |  |
@@ -3545,8 +3545,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> ps_partkey
    row-size=16B cardinality=800.00K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=93.62MB Threads=11
-Per-Host Resource Estimates: Memory=392MB
+Max Per-Host Resource Reservation: Memory=95.50MB Threads=15
+Per-Host Resource Estimates: Memory=396MB
 PLAN-ROOT SINK
 |
 14:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3595,12 +3595,12 @@ PLAN-ROOT SINK
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ps_partkey = p_partkey
-|  runtime filters: RF000 <- p_partkey
 |  row-size=81B cardinality=31.92K
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: p_partkey
+|  |  runtime filters: RF000 <- p_partkey
 |  |
 |  08:EXCHANGE [BROADCAST]
 |  |
@@ -3729,8 +3729,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> tpch.lineitem.l_partkey
    row-size=16B cardinality=6.00M
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=74.62MB Threads=9
-Per-Host Resource Estimates: Memory=499MB
+Max Per-Host Resource Reservation: Memory=75.62MB Threads=13
+Per-Host Resource Estimates: Memory=502MB
 PLAN-ROOT SINK
 |
 12:AGGREGATE [FINALIZE]
@@ -3746,23 +3746,23 @@ PLAN-ROOT SINK
 05:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  hash predicates: l_partkey = p_partkey
 |  other join predicates: l_quantity < 0.2 * avg(l_quantity)
-|  runtime filters: RF000 <- p_partkey
 |  row-size=72B cardinality=29.93K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: p_partkey
+|  |  runtime filters: RF000 <- p_partkey
 |  |
 |  10:EXCHANGE [HASH(p_partkey)]
 |  |
 |  04:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: l_partkey = p_partkey
-|  |  runtime filters: RF002 <- p_partkey
 |  |  row-size=72B cardinality=29.93K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: p_partkey
+|  |  |  runtime filters: RF002 <- p_partkey
 |  |  |
 |  |  09:EXCHANGE [BROADCAST]
 |  |  |
@@ -3958,8 +3958,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=264.88MB Threads=13
-Per-Host Resource Estimates: Memory=842MB
+Max Per-Host Resource Reservation: Memory=298.88MB Threads=19
+Per-Host Resource Estimates: Memory=924MB
 PLAN-ROOT SINK
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -3984,12 +3984,12 @@ PLAN-ROOT SINK
 |
 07:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
 |  hash predicates: o_orderkey = l_orderkey
-|  runtime filters: RF000 <- l_orderkey
 |  row-size=100B cardinality=600.12K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: l_orderkey
+|  |  runtime filters: RF000 <- l_orderkey
 |  |
 |  14:AGGREGATE [FINALIZE]
 |  |  output: sum:merge(l_quantity)
@@ -4010,12 +4010,12 @@ PLAN-ROOT SINK
 |
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: o_custkey = c_custkey
-|  runtime filters: RF002 <- c_custkey
 |  row-size=100B cardinality=5.76M
 |
 |--JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: c_custkey
+|  |  runtime filters: RF002 <- c_custkey
 |  |
 |  12:EXCHANGE [BROADCAST]
 |  |
@@ -4025,12 +4025,12 @@ PLAN-ROOT SINK
 |
 05:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash predicates: l_orderkey = o_orderkey
-|  runtime filters: RF004 <- o_orderkey
 |  row-size=62B cardinality=5.76M
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: o_orderkey
+|  |  runtime filters: RF004 <- o_orderkey
 |  |
 |  11:EXCHANGE [HASH(o_orderkey)]
 |  |
@@ -4142,8 +4142,8 @@ PLAN-ROOT SINK
    runtime filters: RF000 -> l_partkey
    row-size=72B cardinality=801.95K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=37.88MB Threads=5
-Per-Host Resource Estimates: Memory=276MB
+Max Per-Host Resource Reservation: Memory=38.88MB Threads=7
+Per-Host Resource Estimates: Memory=279MB
 PLAN-ROOT SINK
 |
 06:AGGREGATE [FINALIZE]
@@ -4159,12 +4159,12 @@ PLAN-ROOT SINK
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_partkey = p_partkey
 |  other predicates: ((p_brand = 'Brand#12' AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') AND l_quantity >= 1 AND l_quantity <= 11 AND p_size <= 5) OR (p_brand = 'Brand#23' AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') AND l_quantity >= 10 AND l_quantity <= 20 AND p_size <= 10) OR (p_brand = 'Brand#34' AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') AND l_quantity >= 20 AND l_quantity <= 30 AND p_size <= 15))
-|  runtime filters: RF000 <- p_partkey
 |  row-size=124B cardinality=79.99K
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: p_partkey
+|  |  runtime filters: RF000 <- p_partkey
 |  |
 |  04:EXCHANGE [BROADCAST]
 |  |
@@ -4359,8 +4359,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=159.39MB Threads=15
-Per-Host Resource Estimates: Memory=605MB
+Max Per-Host Resource Reservation: Memory=161.39MB Threads=23
+Per-Host Resource Estimates: Memory=613MB
 PLAN-ROOT SINK
 |
 18:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4372,23 +4372,23 @@ PLAN-ROOT SINK
 |
 09:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  hash predicates: ps_suppkey = s_suppkey
-|  runtime filters: RF000 <- s_suppkey
 |  row-size=98B cardinality=400
 |
 |--JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: s_suppkey
+|  |  runtime filters: RF000 <- s_suppkey
 |  |
 |  17:EXCHANGE [HASH(s_suppkey)]
 |  |
 |  08:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash predicates: s_nationkey = n_nationkey
-|  |  runtime filters: RF008 <- n_nationkey
 |  |  row-size=98B cardinality=400
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: n_nationkey
+|  |  |  runtime filters: RF008 <- n_nationkey
 |  |  |
 |  |  15:EXCHANGE [BROADCAST]
 |  |  |
@@ -4407,23 +4407,23 @@ PLAN-ROOT SINK
 07:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  hash predicates: l_partkey = ps_partkey, l_suppkey = ps_suppkey
 |  other join predicates: ps_availqty > 0.5 * sum(l_quantity)
-|  runtime filters: RF002 <- ps_partkey, RF003 <- ps_suppkey
 |  row-size=20B cardinality=79.79K
 |
 |--JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: ps_partkey, ps_suppkey
+|  |  runtime filters: RF002 <- ps_partkey, RF003 <- ps_suppkey
 |  |
 |  14:EXCHANGE [HASH(ps_partkey,ps_suppkey)]
 |  |
 |  06:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash predicates: ps_partkey = p_partkey
-|  |  runtime filters: RF006 <- p_partkey
 |  |  row-size=20B cardinality=79.79K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  build expressions: p_partkey
+|  |  |  runtime filters: RF006 <- p_partkey
 |  |  |
 |  |  13:EXCHANGE [BROADCAST]
 |  |  |
@@ -4661,8 +4661,8 @@ PLAN-ROOT SINK
    predicates: l3.l_receiptdate > l3.l_commitdate
    row-size=60B cardinality=600.12K
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=105.27MB Threads=17
-Per-Host Resource Estimates: Memory=859MB
+Max Per-Host Resource Reservation: Memory=109.14MB Threads=27
+Per-Host Resource Estimates: Memory=915MB
 PLAN-ROOT SINK
 |
 21:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4697,21 +4697,21 @@ PLAN-ROOT SINK
 |  09:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
 |  |  hash predicates: l2.l_orderkey = l1.l_orderkey
 |  |  other join predicates: l2.l_suppkey != l1.l_suppkey
-|  |  runtime filters: RF000 <- l1.l_orderkey
 |  |  row-size=142B cardinality=7.68K
 |  |
 |  |--JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: l1.l_orderkey
+|  |  |  runtime filters: RF000 <- l1.l_orderkey
 |  |  |
 |  |  08:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  |  hash predicates: s_nationkey = n_nationkey
-|  |  |  runtime filters: RF002 <- n_nationkey
 |  |  |  row-size=142B cardinality=7.68K
 |  |  |
 |  |  |--JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: n_nationkey
+|  |  |  |  runtime filters: RF002 <- n_nationkey
 |  |  |  |
 |  |  |  16:EXCHANGE [BROADCAST]
 |  |  |  |
@@ -4722,12 +4722,12 @@ PLAN-ROOT SINK
 |  |  |
 |  |  07:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  |  hash predicates: l1.l_suppkey = s_suppkey
-|  |  |  runtime filters: RF004 <- s_suppkey
 |  |  |  row-size=121B cardinality=191.92K
 |  |  |
 |  |  |--JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  |  build expressions: s_suppkey
+|  |  |  |  runtime filters: RF004 <- s_suppkey
 |  |  |  |
 |  |  |  15:EXCHANGE [BROADCAST]
 |  |  |  |
@@ -4738,12 +4738,12 @@ PLAN-ROOT SINK
 |  |  |
 |  |  06:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  |  hash predicates: l1.l_orderkey = o_orderkey
-|  |  |  runtime filters: RF006 <- o_orderkey
 |  |  |  row-size=81B cardinality=191.92K
 |  |  |
 |  |  |--JOIN BUILD
 |  |  |  |  join-table-id=04 plan-id=05 cohort-id=03
 |  |  |  |  build expressions: o_orderkey
+|  |  |  |  runtime filters: RF006 <- o_orderkey
 |  |  |  |
 |  |  |  14:EXCHANGE [HASH(o_orderkey)]
 |  |  |  |
@@ -4911,8 +4911,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=12
-Per-Host Resource Estimates: Memory=414MB
+Max Per-Host Resource Reservation: Memory=83.75MB Threads=16
+Per-Host Resource Estimates: Memory=416MB
 PLAN-ROOT SINK
 |
 15:MERGING-EXCHANGE [UNPARTITIONED]
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-large-rows.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-large-rows.test
index 8756f72..db401f9 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling-large-rows.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-large-rows.test
@@ -63,7 +63,7 @@ from functional.alltypes atp
   join bigstrs bs on repeat(atp.string_col, 10000) = substring(bs.bigstr, 5000000, 10000) and atp.id = bs.id
 where atp.id < 100
 ---- CATCH
-Row of size 9.54 MB could not be materialized by HASH_JOIN_NODE (id=2). Increase the max_row_size query option (currently 512.00 KB) to process larger rows.
+Row of size 9.54 MB could not be materialized by Hash Join Builder (join_node_id=2). Increase the max_row_size query option (currently 512.00 KB) to process larger rows.
 ====
 ---- QUERY
 # Row is too big to process in right side of hash join.
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling.test b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
index 835dbdc..e5a43a9 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
@@ -296,12 +296,14 @@ row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
 # Test spilling join with many duplicates in join key. We don't expect this to succeed
-# with a memory constraint: see IMPALA-4857.
+# with a memory constraint: see IMPALA-4857. Limit size of probe so that query doesn't
+# bog down executing an exploding join.
 # The additional "order by" and "limit" clauses make sure that a successful
 # query does not too much data to the client.
 set buffer_pool_limit=167m;
-select *
+select straight_join *
 from lineitem l1 join lineitem l2 on l1.l_linenumber = l2.l_linenumber
+where l1.l_orderkey < 100000
 order by l1.l_orderkey desc, l1.l_linenumber desc limit 10
 ---- CATCH
 Repartitioning did not reduce the size of a spilled partition
diff --git a/tests/common/impala_test_suite.py b/tests/common/impala_test_suite.py
index 0ac21b4..e7b5c8f 100644
--- a/tests/common/impala_test_suite.py
+++ b/tests/common/impala_test_suite.py
@@ -916,6 +916,7 @@ class ImpalaTestSuite(BaseTestSuite):
     Uses a default list of valid section names if valid_section_names is None.
     """
     test_file_path = os.path.join(WORKLOAD_DIR, workload, 'queries', file_name + '.test')
+    LOG.info("Loading query test file: %s", test_file_path)
     if not os.path.isfile(test_file_path):
       assert False, 'Test file not found: %s' % file_name
     return parse_query_test_file(test_file_path, valid_section_names, encoding=encoding)
diff --git a/tests/custom_cluster/test_mt_dop.py b/tests/custom_cluster/test_mt_dop.py
index 796eb42..b187ff1 100644
--- a/tests/custom_cluster/test_mt_dop.py
+++ b/tests/custom_cluster/test_mt_dop.py
@@ -35,30 +35,6 @@ class TestMtDopFlags(CustomClusterTestSuite):
     super(TestMtDopFlags, cls).add_test_dimensions()
 
   @pytest.mark.execute_serially
-  @CustomClusterTestSuite.with_args(impalad_args="--unlock_mt_dop=true")
-  @SkipIfABFS.file_or_folder_name_ends_with_period
-  def test_mt_dop_all(self, vector, unique_database):
-    """Test joins, inserts and runtime filters with mt_dop > 0"""
-    vector = deepcopy(vector)
-    vector.get_value('exec_option')['mt_dop'] = 4
-    self.run_test_case('QueryTest/joins', vector, use_db="functional_parquet")
-    self.run_test_case('QueryTest/insert', vector)
-
-    # Runtime filter tests assume 3 fragments, which we can get with one instance per
-    # node.
-    vector.get_value('exec_option')['mt_dop'] = 1
-    self.run_test_case('QueryTest/runtime_filters', vector,
-       test_file_vars={'$RUNTIME_FILTER_WAIT_TIME_MS': str(WAIT_TIME_MS)})
-    self.run_test_case('QueryTest/runtime_row_filters', vector,
-        use_db="functional_parquet",
-        test_file_vars={'$RUNTIME_FILTER_WAIT_TIME_MS' : str(WAIT_TIME_MS)})
-
-    # Allow test to override num_nodes.
-    del vector.get_value('exec_option')['num_nodes']
-    self.run_test_case('QueryTest/joins_mt_dop', vector,
-       test_file_vars={'$RUNTIME_FILTER_WAIT_TIME_MS': str(WAIT_TIME_MS)})
-
-  @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(impalad_args="--mt_dop_auto_fallback=true")
   @SkipIfNotHdfsMinicluster.tuned_for_minicluster
   def test_mt_dop_fallback(self, vector, unique_database):
diff --git a/tests/failure/test_failpoints.py b/tests/failure/test_failpoints.py
index 7ae8e00..6d0a7db 100644
--- a/tests/failure/test_failpoints.py
+++ b/tests/failure/test_failpoints.py
@@ -100,14 +100,7 @@ class TestFailpoints(ImpalaTestSuite):
     location = vector.get_value('location')
     vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
 
-    try:
-      plan_node_ids = self.__parse_plan_nodes_from_explain(query, vector)
-    except ImpalaBeeswaxException as e:
-      if "MT_DOP not supported" in str(e):
-        pytest.xfail(reason="MT_DOP not supported.")
-      else:
-        raise e
-
+    plan_node_ids = self.__parse_plan_nodes_from_explain(query, vector)
     for node_id in plan_node_ids:
       debug_action = '%d:%s:%s' % (node_id, location, FAILPOINT_ACTION_MAP[action])
       # IMPALA-7046: add jitter to backend startup to exercise various failure paths.
diff --git a/tests/query_test/test_cancellation.py b/tests/query_test/test_cancellation.py
index e82b864..8770733 100644
--- a/tests/query_test/test_cancellation.py
+++ b/tests/query_test/test_cancellation.py
@@ -68,6 +68,9 @@ SORT_QUERY = 'select * from lineitem order by l_orderkey'
 SORT_CANCEL_DELAY = range(6, 10)
 SORT_BUFFER_POOL_LIMIT = ['0', '300m'] # Test spilling and non-spilling sorts.
 
+# Test with and without multithreading
+MT_DOP_VALUES = [0, 4]
+
 class TestCancellation(ImpalaTestSuite):
   @classmethod
   def get_workload(self):
@@ -92,6 +95,8 @@ class TestCancellation(ImpalaTestSuite):
         ImpalaTestDimension('buffer_pool_limit', 0))
     cls.ImpalaTestMatrix.add_dimension(
         ImpalaTestDimension('cpu_limit_s', *CPU_LIMIT_S))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('mt_dop', *MT_DOP_VALUES))
 
     cls.ImpalaTestMatrix.add_constraint(
         lambda v: v.get_value('query_type') != 'CTAS' or (\
@@ -145,6 +150,7 @@ class TestCancellation(ImpalaTestSuite):
     vector.get_value('exec_option')['buffer_pool_limit'] =\
         vector.get_value('buffer_pool_limit')
     vector.get_value('exec_option')['cpu_limit_s'] = vector.get_value('cpu_limit_s')
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
 
     # Execute the query multiple times, cancelling it each time.
     for i in xrange(NUM_CANCELATION_ITERATIONS):
diff --git a/tests/query_test/test_join_queries.py b/tests/query_test/test_join_queries.py
index a331901..b4ed2ef 100644
--- a/tests/query_test/test_join_queries.py
+++ b/tests/query_test/test_join_queries.py
@@ -32,6 +32,9 @@ from tests.common.test_vector import ImpalaTestDimension
 
 class TestJoinQueries(ImpalaTestSuite):
   BATCH_SIZES = [0, 1]
+  MT_DOP_VALUES = [0, 4]
+  # Additional values for exhaustive tests.
+  MT_DOP_VALUES_EXHAUSTIVE = [1]
 
   @classmethod
   def get_workload(cls):
@@ -42,6 +45,11 @@ class TestJoinQueries(ImpalaTestSuite):
     super(TestJoinQueries, cls).add_test_dimensions()
     cls.ImpalaTestMatrix.add_dimension(
         ImpalaTestDimension('batch_size', *TestJoinQueries.BATCH_SIZES))
+    mt_dop_values = cls.MT_DOP_VALUES
+    if cls.exploration_strategy() == 'exhaustive':
+      mt_dop_values += cls.MT_DOP_VALUES_EXHAUSTIVE
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('mt_dop', *mt_dop_values))
     # TODO: Look into splitting up join tests to accomodate hbase.
     # Joins with hbase tables produce drastically different results.
     cls.ImpalaTestMatrix.add_constraint(lambda v:\
@@ -54,12 +62,14 @@ class TestJoinQueries(ImpalaTestSuite):
   def test_basic_joins(self, vector):
     new_vector = deepcopy(vector)
     new_vector.get_value('exec_option')['batch_size'] = vector.get_value('batch_size')
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/joins', new_vector)
 
   def test_single_node_joins_with_limits_exhaustive(self, vector):
     if self.exploration_strategy() != 'exhaustive': pytest.skip()
     new_vector = deepcopy(vector)
     new_vector.get_value('exec_option')['num_nodes'] = 1
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     del new_vector.get_value('exec_option')['batch_size']  # .test file sets batch_size
     self.run_test_case('QueryTest/single-node-joins-with-limits-exhaustive', new_vector)
 
@@ -72,11 +82,13 @@ class TestJoinQueries(ImpalaTestSuite):
   def test_joins_against_hbase(self, vector):
     new_vector = deepcopy(vector)
     new_vector.get_value('exec_option')['batch_size'] = vector.get_value('batch_size')
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/joins-against-hbase', new_vector)
 
   def test_outer_joins(self, vector):
     new_vector = deepcopy(vector)
     new_vector.get_value('exec_option')['batch_size'] = vector.get_value('batch_size')
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/outer-joins', new_vector)
 
   def test_single_node_nested_loop_joins(self, vector):
@@ -90,11 +102,13 @@ class TestJoinQueries(ImpalaTestSuite):
     if self.exploration_strategy() != 'exhaustive': pytest.skip()
     new_vector = deepcopy(vector)
     new_vector.get_value('exec_option')['num_nodes'] = 1
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/single-node-nlj-exhaustive', new_vector)
 
   def test_empty_build_joins(self, vector):
     new_vector = deepcopy(vector)
     new_vector.get_value('exec_option')['batch_size'] = vector.get_value('batch_size')
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/empty-build-joins', new_vector)
 
 class TestTPCHJoinQueries(ImpalaTestSuite):
diff --git a/tests/query_test/test_mt_dop.py b/tests/query_test/test_mt_dop.py
index dd53f1e..5908a8d 100644
--- a/tests/query_test/test_mt_dop.py
+++ b/tests/query_test/test_mt_dop.py
@@ -20,12 +20,14 @@
 import pytest
 
 from copy import deepcopy
-from tests.common.environ import ImpalaTestClusterProperties
+from tests.common.environ import ImpalaTestClusterProperties, build_flavor_timeout
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.kudu_test_suite import KuduTestSuite
-from tests.common.skip import SkipIfEC, SkipIfNotHdfsMinicluster
+from tests.common.skip import SkipIfABFS, SkipIfEC, SkipIfNotHdfsMinicluster
 from tests.common.test_vector import ImpalaTestDimension
 
+WAIT_TIME_MS = build_flavor_timeout(60000, slow_build_timeout=100000)
+
 # COMPUTE STATS on Parquet tables automatically sets MT_DOP=4, so include
 # the value 0 to cover the non-MT path as well.
 MT_DOP_VALUES = [0, 1, 2, 8]
@@ -110,6 +112,27 @@ class TestMtDopParquet(ImpalaTestSuite):
     vector.get_value('exec_option')['parquet_read_statistics'] = '0'
     self.run_test_case('QueryTest/parquet-filtering', vector)
 
+  @pytest.mark.execute_serially
+  @SkipIfABFS.file_or_folder_name_ends_with_period
+  def test_mt_dop_insert(self, vector, unique_database):
+    """Basic tests for inserts with mt_dop > 0"""
+    mt_dop = vector.get_value('mt_dop')
+    if mt_dop == 0:
+      pytest.skip("Non-mt inserts tested elsewhere")
+    self.run_test_case('QueryTest/insert', vector)
+
+  def test_mt_dop_only_joins(self, vector, unique_database):
+    """MT_DOP specific tests for joins."""
+    mt_dop = vector.get_value('mt_dop')
+    if mt_dop == 0:
+      pytest.skip("Test requires mt_dop > 0")
+    vector = deepcopy(vector)
+    # Allow test to override num_nodes.
+    del vector.get_value('exec_option')['num_nodes']
+    self.run_test_case('QueryTest/joins_mt_dop', vector,
+       test_file_vars={'$RUNTIME_FILTER_WAIT_TIME_MS': str(WAIT_TIME_MS)})
+
+
 class TestMtDopKudu(KuduTestSuite):
   @classmethod
   def add_test_dimensions(cls):
diff --git a/tests/query_test/test_nested_types.py b/tests/query_test/test_nested_types.py
index d1e2719..f03dfbf 100644
--- a/tests/query_test/test_nested_types.py
+++ b/tests/query_test/test_nested_types.py
@@ -36,6 +36,8 @@ from tests.common.test_vector import ImpalaTestDimension
 from tests.util.filesystem_utils import WAREHOUSE, get_fs_path, IS_HDFS
 
 class TestNestedTypes(ImpalaTestSuite):
+  """Functional tests for nested types, run for all file formats that support nested
+  types."""
   @classmethod
   def get_workload(self):
     return 'functional-query'
@@ -45,40 +47,58 @@ class TestNestedTypes(ImpalaTestSuite):
     super(TestNestedTypes, cls).add_test_dimensions()
     cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format in ['parquet', 'orc'])
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('mt_dop', 0, 2))
 
   def test_scanner_basic(self, vector):
     """Queries that do not materialize arrays."""
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/nested-types-scanner-basic', vector)
 
   def test_scanner_array_materialization(self, vector):
     """Queries that materialize arrays."""
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/nested-types-scanner-array-materialization', vector)
 
   def test_scanner_multiple_materialization(self, vector):
     """Queries that materialize the same array multiple times."""
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/nested-types-scanner-multiple-materialization', vector)
 
   def test_scanner_position(self, vector):
     """Queries that materialize the artifical position element."""
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/nested-types-scanner-position', vector)
 
   def test_scanner_map(self, vector):
     """Queries that materialize maps. (Maps looks like arrays of key/value structs, so
     most map functionality is already tested by the array tests.)"""
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/nested-types-scanner-maps', vector)
 
   def test_runtime(self, vector):
     """Queries that send collections through the execution runtime."""
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/nested-types-runtime', vector)
 
   def test_subplan(self, vector):
     """Test subplans with various exec nodes inside it."""
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     db_suffix = vector.get_value('table_format').db_suffix()
     self.run_test_case('QueryTest/nested-types-subplan', vector,
                        use_db='tpch_nested' + db_suffix)
 
   def test_subplan_single_node(self, vector):
     """Test subplans with various exec nodes inside it and num_nodes=1."""
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     new_vector = deepcopy(vector)
     new_vector.get_value('exec_option')['num_nodes'] = 1
     self.run_test_case('QueryTest/nested-types-subplan-single-node', new_vector)
@@ -89,6 +109,19 @@ class TestNestedTypes(ImpalaTestSuite):
     self.run_test_case('QueryTest/nested-types-with-clause', vector,
                        use_db='tpch_nested' + db_suffix)
 
+
+class TestNestedTypesNoMtDop(ImpalaTestSuite):
+  """Functional tests for nested types that do not need to be run with mt_dop > 0."""
+  @classmethod
+  def get_workload(self):
+    return 'functional-query'
+
+  @classmethod
+  def add_test_dimensions(cls):
+    super(TestNestedTypesNoMtDop, cls).add_test_dimensions()
+    cls.ImpalaTestMatrix.add_constraint(lambda v:
+        v.get_value('table_format').file_format in ['parquet', 'orc'])
+
   def test_tpch(self, vector):
     """Queries over the larger nested TPCH dataset."""
     db_suffix = vector.get_value('table_format').db_suffix()
@@ -181,6 +214,7 @@ class TestNestedTypes(ImpalaTestSuite):
     self.run_test_case('QueryTest/nested-types-basic-partitioned', vector,
         unique_database)
 
+
 class TestParquetArrayEncodings(ImpalaTestSuite):
   TESTFILE_DIR = os.path.join(os.environ['IMPALA_HOME'],
                               "testdata/parquet_nested_types_encodings")
diff --git a/tests/query_test/test_runtime_filters.py b/tests/query_test/test_runtime_filters.py
index 6c7c41f..5e53c6e 100644
--- a/tests/query_test/test_runtime_filters.py
+++ b/tests/query_test/test_runtime_filters.py
@@ -16,6 +16,7 @@
 # under the License.
 #
 
+from copy import deepcopy
 import pytest
 import re
 import time
@@ -25,6 +26,7 @@ from tests.common.environ import build_flavor_timeout
 from tests.common.impala_cluster import ImpalaCluster
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfLocal, SkipIfIsilon
+from tests.common.test_vector import ImpalaTestDimension
 from tests.verifiers.metric_verifier import MetricVerifier
 
 # slow_build_timeout is set to 200000 to avoid failures like IMPALA-8064 where the
@@ -49,14 +51,26 @@ class TestRuntimeFilters(ImpalaTestSuite):
     # Runtime filters are disabled on HBase
     cls.ImpalaTestMatrix.add_constraint(
         lambda v: v.get_value('table_format').file_format not in ['hbase'])
+    # Exercise both mt and non-mt code paths. Some tests assume 3 finstances, so
+    # tests are not expected to work unmodified with higher mt_dop values.
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('mt_dop', 0, 1))
+    # Don't test all combinations of file format and mt_dop, only test a few
+    # representative formats.
+    cls.ImpalaTestMatrix.add_constraint(
+        lambda v: v.get_value('table_format').file_format in ['parquet', 'text', 'kudu']
+        or v.get_value('mt_dop') == 0)
 
   def test_basic_filters(self, vector):
+    new_vector = deepcopy(vector)
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/runtime_filters', vector,
         test_file_vars={'$RUNTIME_FILTER_WAIT_TIME_MS' : str(WAIT_TIME_MS)})
 
   def test_wait_time(self, vector):
     """Test that a query that has global filters does not wait for them if run in LOCAL
     mode"""
+    new_vector = deepcopy(vector)
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     now = time.time()
     self.run_test_case('QueryTest/runtime_filters_wait', vector)
     duration_s = time.time() - now
@@ -68,6 +82,8 @@ class TestRuntimeFilters(ImpalaTestSuite):
   def test_wait_time_cancellation(self, vector):
     """Regression test for IMPALA-9065 to ensure that threads waiting for filters
     get woken up and exit promptly when the query is cancelled."""
+    new_vector = deepcopy(vector)
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     # Make sure the cluster is quiesced before we start this test
     self._verify_no_fragments_running()
 
@@ -101,6 +117,8 @@ class TestRuntimeFilters(ImpalaTestSuite):
   def test_file_filtering(self, vector):
     if 'kudu' in str(vector.get_value('table_format')):
       return
+    new_vector = deepcopy(vector)
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.change_database(self.client, vector.get_value('table_format'))
     self.execute_query("SET RUNTIME_FILTER_MODE=GLOBAL")
     self.execute_query("SET RUNTIME_FILTER_WAIT_TIME_MS=10000")
@@ -216,7 +234,12 @@ class TestRuntimeRowFilters(ImpalaTestSuite):
     super(TestRuntimeRowFilters, cls).add_test_dimensions()
     cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format in ['parquet'])
+    # Exercise both mt and non-mt code paths. Some tests assume 3 finstances, so
+    # tests are not expected to work unmodified with higher mt_dop values.
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('mt_dop', 0, 4))
 
   def test_row_filters(self, vector):
+    new_vector = deepcopy(vector)
+    new_vector.get_value('exec_option')['mt_dop'] = vector.get_value('mt_dop')
     self.run_test_case('QueryTest/runtime_row_filters', vector,
                        test_file_vars={'$RUNTIME_FILTER_WAIT_TIME_MS' : str(WAIT_TIME_MS)})
diff --git a/tests/query_test/test_spilling.py b/tests/query_test/test_spilling.py
index 83f553b..48daf49 100644
--- a/tests/query_test/test_spilling.py
+++ b/tests/query_test/test_spilling.py
@@ -57,8 +57,19 @@ class TestSpillingDebugActionDimensions(ImpalaTestSuite):
       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}))
+        create_exec_option_dimension_from_dict({'default_spillable_buffer_size': ['256k'],
+          'debug_action': debug_action_dims, 'mt_dop': [0, 1]}))
+    # Pare down the combinations of mt_dop and debug_action that run to reduce test time.
+    # The MT code path for joins is more complex, so focus testing there.
+    if cls.exploration_strategy() == 'exhaustive':
+      debug_action_dims = CORE_DEBUG_ACTION_DIMS + EXHAUSTIVE_DEBUG_ACTION_DIMS
+      cls.ImpalaTestMatrix.add_constraint(lambda v:
+          v.get_value('exec_option')['mt_dop'] == 1 or
+          v.get_value('exec_option')['debug_action'] in CORE_DEBUG_ACTION_DIMS)
+    elif cls.exploration_strategy() == 'core':
+      cls.ImpalaTestMatrix.add_constraint(lambda v:
+          v.get_value('exec_option')['mt_dop'] == 1 or
+          v.get_value('exec_option')['debug_action'] is None)
 
   def test_spilling(self, vector):
     self.run_test_case('QueryTest/spilling', vector)
@@ -102,7 +113,8 @@ class TestSpillingNoDebugActionDimensions(ImpalaTestSuite):
     cls.ImpalaTestMatrix.add_dimension(create_parquet_dimension('tpch'))
     # 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']}))
+        create_exec_option_dimension_from_dict({'default_spillable_buffer_size': ['256k'],
+            'mt_dop': [0, 4]}))
 
   def test_spilling_naaj_no_deny_reservation(self, vector):
     """