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

[impala] 03/07: IMPALA-11604 (part 2): Compute Effective Parallelism of Query

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

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

commit dafc0fb7a86273e1d5f3550e5d6f8cb0f42b05fc
Author: Riza Suminto <ri...@cloudera.com>
AuthorDate: Mon Mar 6 12:21:32 2023 -0800

    IMPALA-11604 (part 2): Compute Effective Parallelism of Query
    
    Part 1 of IMPALA-11604 implements the ProcessingCost model for each
    PlanNode and DataSink. This second part builds on top of ProcessingCost
    model by adjusting the number of instances for each fragment after
    considering their production-consumption ratio, and then finally returns
    a number representing an ideal CPU core count required for a query to
    run efficiently. A more detailed explanation of the CPU costing
    algorithm can be found in the three steps below.
    
    I. Compute the total ProcessingCost of a fragment.
    
    The costing algorithm splits a query fragment into several segments
    divided by blocking PlanNode/DataSink boundary. Each fragment segment is
    a subtree of PlanNodes/DataSink in the fragment with a DataSink or
    blocking PlanNode as root and non-blocking leaves. All other nodes in
    the segment are non-blocking. PlanNodes or DataSink that belong to the
    same segment will have their ProcessingCost summed. A new CostingSegment
    class is added to represent this segment.
    
    A fragment that has a blocking PlanNode or blocking DataSink is called a
    blocking fragment. Currently, only JoinBuildSink is considered as
    blocking DataSink. A fragment without any blocking nodes is called a
    non-blocking fragment. Step III discuss further about blocking and
    non-blocking fragment.
    
    Take an example of the following fragment plant, which is blocking since
    it has 3 blocking PlanNode: 12:AGGREGATE, 06:SORT, and 08:TOP-N.
    
      F03:PLAN FRAGMENT [HASH(i_class)] hosts=3 instances=6 (adjusted from 12)
      fragment-costs=[34974657, 2159270, 23752870, 22]
      08:TOP-N [LIMIT=100]
      |  cost=900
      |
      07:ANALYTIC
      |  cost=23751970
      |
      06:SORT
      |  cost=2159270
      |
      12:AGGREGATE [FINALIZE]
      |  cost=34548320
      |
      11:EXCHANGE [HASH(i_class)]
         cost=426337
    
    In bottom-up direction, there exist four segments in F03:
      Blocking segment 1: (11:EXCHANGE, 12:AGGREGATE)
      Blocking segment 2: 06:SORT
      Blocking segment 3: (07:ANALYTIC, 08:TOP-N)
      Non-blocking segment 4: DataStreamSink of F03
    
    Therefore we have:
      PC(segment 1) = 426337+34548320
      PC(segment 2) = 2159270
      PC(segment 3) = 23751970+900
      PC(segment 4) = 22
    
    These per-segment costs stored in a CostingSegment tree rooted at
    PlanFragment.rootSegment_, and are [34974657, 2159270, 23752870, 22]
    respectively after the post-order traversal.
    
    This is implemented in PlanFragment.computeCostingSegment() and
    PlanFragment.collectCostingSegmentHelper().
    
    II. Compute the effective degree of parallelism (EDoP) of fragments.
    
    The costing algorithm walks PlanFragments of the query plan tree in
    post-order traversal. Upon visiting a PlanFragment, the costing
    algorithm attempts to adjust the number of instances (effective
    parallelism) of that fragment by comparing the last segment's
    ProcessingCost of its child and production-consumption rate between its
    adjacent segments from step I. To simplify this initial implementation,
    the parallelism of PlanFragment containing EmptySetNode, UnionNode, or
    ScanNode will remain unchanged (follow MT_DOP).
    
    This step is implemented at PlanFragment.traverseEffectiveParallelism().
    
    III. Compute the EDoP of the query.
    
    Effective parallelism of a query is the maximum upper bound of CPU core
    count that can parallelly work on a query when considering the
    overlapping between fragment execution and blocking operators. We
    compute this in a similar post-order traversal as step II and split the
    query tree into blocking fragment subtrees similar to step I. The
    following is an example of a query plan from TPCDS-Q12.
    
      F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
      PLAN-ROOT SINK
      |
      13:MERGING-EXCHANGE [UNPARTITIONED]
      |
      F03:PLAN FRAGMENT [HASH(i_class)] hosts=3 instances=3 (adjusted from 12)
      08:TOP-N [LIMIT=100]
      |
      07:ANALYTIC
      |
      06:SORT
      |
      12:AGGREGATE [FINALIZE]
      |
      11:EXCHANGE [HASH(i_class)]
      |
      F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
      05:AGGREGATE [STREAMING]
      |
      04:HASH JOIN [INNER JOIN, BROADCAST]
      |
      |--F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
      |  JOIN BUILD
      |  |
      |  10:EXCHANGE [BROADCAST]
      |  |
      |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
      |  02:SCAN HDFS [tpcds10_parquet.date_dim, RANDOM]
      |
      03:HASH JOIN [INNER JOIN, BROADCAST]
      |
      |--F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
      |  JOIN BUILD
      |  |
      |  09:EXCHANGE [BROADCAST]
      |  |
      |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
      |  01:SCAN HDFS [tpcds10_parquet.item, RANDOM]
      |
      00:SCAN HDFS [tpcds10_parquet.web_sales, RANDOM]
    
    A blocking fragment is a fragment that has a blocking PlanNode or
    blocking DataSink in it. The costing algorithm splits the query plan
    tree into blocking subtrees divided by blocking fragment boundary. Each
    blocking subtree has a blocking fragment as a root and non-blocking
    fragments as the intermediate or leaf nodes. From the TPCDS-Q12 example
    above, the query plan is divided into five blocking subtrees of
    [(F05, F02), (F06, F01), F00, F03, F04].
    
    A CoreCount is a container class that represents the CPU core
    requirement of a subtree of a query or the query itself. Each blocking
    subtree will have its fragment's adjusted instance count summed into a
    single CoreCount. This means that all fragments within a blocking
    subtree can run in parallel and should be assigned one core per fragment
    instance. The CoreCount for each blocking subtree in the TPCDS-Q12
    example is [4, 4, 12, 3, 1].
    
    Upon visiting a blocking fragment, the maximum between current
    CoreCount (rooted at that blocking fragment) vs previous blocking
    subtrees CoreCount is taken and the algorithm continues up to the next
    ancestor PlanFragment. The final CoreCount for the TPCDS-Q12 example is
    12.
    
    This step is implemented at Planner.computeBlockingAwareCores() and
    PlanFragment.traverseBlockingAwareCores().
    
    The resulting CoreCount at the root PlanFragment is then taken as the
    ideal CPU core count / EDoP of the query. This number will be compared
    against the total CPU count of an Impala executor group to determine if
    it fits to run in that set or not. A backend flag
    query_cpu_count_divisor is added to help scale down/up the EDoP of a
    query if needed.
    
    Two query options are added to control the entire computation of EDoP.
    1. COMPUTE_PROCESSING_COST
       Control whether to enable this CPU costing algorithm or not.
       Must also set MT_DOP > 0 for this query option to take effect.
    
    2. PROCESSING_COST_MIN_THREADS
       Control the minimum number of fragment instances (threads) that the
       costing algorithm is allowed to adjust. The costing algorithm is in
       charge of increasing the fragment's instance count beyond this
       minimum number through producer-consumer rate comparison. The maximum
       number of fragment is max between PROCESSING_COST_MIN_THREADS,
       MT_DOP, and number of cores per executor.
    
    This patch also adds three backend flags to tune the algorithm.
    1. query_cpu_count_divisor
       Divide the CPU requirement of a query to fit the total available CPU
       in the executor group. For example, setting value 2 will fit the
       query with CPU requirement 2X to an executor group with total
       available CPU X. Note that setting with a fractional value less than
       1 effectively multiplies the query CPU requirement. A valid value is
       > 0.0. The default value is 1.
    
    2. processing_cost_use_equal_expr_weight
       If true, all expression evaluations are weighted equally to 1 during
       the plan node's processing cost calculation. If false, expression
       cost from IMPALA-2805 will be used. Default to true.
    
    3. min_processing_per_thread
       Minimum processing load (in processing cost unit) that a fragment
       instance needs to work on before planner considers increasing
       instance count based on the processing cost rather than the MT_DOP
       setting. The decision is per fragment. Setting this to high number
       will reduce parallelism of a fragment (more workload per fragment),
       while setting to low number will increase parallelism (less workload
       per fragment). Actual parallelism might still be constrained by the
       total number of cores in selected executor group, MT_DOP, or
       PROCESSING_COST_MIN_THREAD query option. Must be a positive integer.
       Currently default to 10M.
    
    As an example, the following are additional ProcessingCost information
    printed to coordinator log for Q3, Q12, and Q15 ran on TPCDS 10GB scale,
    3 executors, MT_DOP=4, PROCESSING_COST_MAX_THREADS=4, and
    processing_cost_use_equal_expr_weight=false.
    
      Q3
      CoreCount={total=12 trace=F00:12}
    
      Q12
      CoreCount={total=12 trace=F00:12}
    
      Q15
      CoreCount={total=15 trace=N07:3+F00:12}
    
    There are a few TODOs which will be done in follow up tasks:
    1. Factor in row width in ProcessingCost calcuation (IMPALA-11972).
    2. Tune the individual expression cost from IMPALA-2805.
    3. Benchmark and tune min_processing_per_thread with an optimal value.
    4. Revisit cases where cardinality is not available (getCardinality() or
       getInputCardinality() return -1).
    5. Bound SCAN and UNION fragments by ProcessingCost as well (need to
       address IMPALA-8081).
    
    Testing:
    - Add TestTpcdsQueryWithProcessingCost, which is a similar run of
      TestTpcdsQuery, but with COMPUTE_PROCESSING_COST=1 and MT_DOP=4.
      Setting log level TRACE for PlanFragment and manually running
      TestTpcdsQueryWithProcessingCost in minicluster shows several fragment
      instance count reduction from 12 to either of 9, 6, or 3 in
      coordinator log.
    - Add PlannerTest#testProcessingCost
      Adjusted fragment count is indicated by "(adjusted from 12)" in the
      query profile.
    - Add TestExecutorGroups::test_query_cpu_count_divisor.
    
    Co-authored-by: Qifan Chen <qc...@cloudera.com>
    
    Change-Id: Ibb2a796fdf78336e95991955d89c671ec82be62e
    Reviewed-on: http://gerrit.cloudera.org:8080/19593
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Kurt Deschler <kd...@cloudera.com>
    Reviewed-by: Riza Suminto <ri...@cloudera.com>
---
 be/src/scheduling/scheduler.cc                     |   51 +-
 be/src/scheduling/scheduler.h                      |   28 +-
 be/src/service/query-options.cc                    |   16 +
 be/src/service/query-options.h                     |    8 +-
 be/src/util/backend-gflag-util.cc                  |   53 +
 common/thrift/BackendGflags.thrift                 |    6 +
 common/thrift/ImpalaService.thrift                 |   10 +-
 common/thrift/Planner.thrift                       |    7 +
 common/thrift/Query.thrift                         |   11 +
 .../java/org/apache/impala/planner/CoreCount.java  |  110 +
 .../org/apache/impala/planner/CostingSegment.java  |  235 +
 .../org/apache/impala/planner/ExchangeNode.java    |    5 +-
 .../org/apache/impala/planner/HdfsTableSink.java   |    6 +
 .../org/apache/impala/planner/JoinBuildSink.java   |    8 +
 .../org/apache/impala/planner/PlanFragment.java    |  440 +-
 .../java/org/apache/impala/planner/PlanNode.java   |    9 +-
 .../org/apache/impala/planner/PlanRootSink.java    |    6 +
 .../java/org/apache/impala/planner/Planner.java    |  104 +-
 .../org/apache/impala/planner/ProcessingCost.java  |    8 +-
 .../org/apache/impala/service/BackendConfig.java   |   10 +
 .../java/org/apache/impala/service/Frontend.java   |   54 +-
 .../main/java/org/apache/impala/util/ExprUtil.java |   16 +-
 .../org/apache/impala/planner/PlannerTest.java     |   19 +-
 ...er-2-groups.xml => fair-scheduler-3-groups.xml} |    2 +
 ...a-site-2-groups.xml => llama-site-3-groups.xml} |   22 +
 .../queries/PlannerTest/tpcds-processing-cost.test | 8739 ++++++++++++++++++++
 tests/custom_cluster/test_executor_groups.py       |   85 +-
 tests/query_test/test_tpcds_queries.py             |   17 +
 28 files changed, 10038 insertions(+), 47 deletions(-)

diff --git a/be/src/scheduling/scheduler.cc b/be/src/scheduling/scheduler.cc
index 947f234f2..09204ff4a 100644
--- a/be/src/scheduling/scheduler.cc
+++ b/be/src/scheduling/scheduler.cc
@@ -264,6 +264,25 @@ void Scheduler::ComputeFragmentExecParams(
   }
 }
 
+void Scheduler::CheckEffectiveInstanceCount(
+    const FragmentScheduleState* fragment_state, const ScheduleState* state) {
+  int effective_instance_count = fragment_state->fragment.effective_instance_count;
+  if (effective_instance_count > 0) {
+    if (ContainsUnionNode(fragment_state->fragment.plan)
+        || ContainsScanNode(fragment_state->fragment.plan)
+        || IsExceedMaxFsWriters(fragment_state, fragment_state, state)) {
+      // TODO: Fragment with UnionNode or ScanNode or one where IsExceedMaxFsWriters
+      // equals true is not checked for now since it require further modification of the
+      // scan-range based scheduling in CreateCollocatedAndScanInstances().
+      return;
+    }
+
+    DCHECK_GE(effective_instance_count, fragment_state->instance_states.size())
+        << fragment_state->fragment.display_name
+        << " scheduled higher than the effective count.";
+  }
+}
+
 void Scheduler::ComputeFragmentExecParams(const ExecutorConfig& executor_config,
     const TPlanExecInfo& plan_exec_info, FragmentScheduleState* fragment_state,
     ScheduleState* state) {
@@ -338,8 +357,7 @@ void Scheduler::ComputeFragmentExecParams(const ExecutorConfig& executor_config,
         instance_state.AddScanRanges(entry.first, entry.second);
       }
     }
-  } else if (ContainsNode(fragment.plan, TPlanNodeType::UNION_NODE)
-      || ContainsScanNode(fragment.plan)) {
+  } else if (ContainsUnionNode(fragment.plan) || ContainsScanNode(fragment.plan)) {
     VLOG(3) << "Computing exec params for scan and/or union fragment.";
     // case 2: leaf fragment (i.e. no input fragments) with a single scan node.
     // case 3: union fragment, which may have scan nodes and may have input fragments.
@@ -348,9 +366,11 @@ void Scheduler::ComputeFragmentExecParams(const ExecutorConfig& executor_config,
     VLOG(3) << "Computing exec params for interior fragment.";
     // case 4: interior (non-leaf) fragment without a scan or union.
     // We assign the same hosts as those of our leftmost input fragment (so that a
-    // merge aggregation fragment runs on the hosts that provide the input data).
+    // merge aggregation fragment runs on the hosts that provide the input data) OR
+    // follow the effective_instance_count specified by planner.
     CreateInputCollocatedInstances(fragment_state, state);
   }
+  CheckEffectiveInstanceCount(fragment_state, state);
 }
 
 // Maybe the easiest way to understand the objective of this algorithm is as a
@@ -381,7 +401,7 @@ void Scheduler::ComputeFragmentExecParams(const ExecutorConfig& executor_config,
 void Scheduler::CreateCollocatedAndScanInstances(const ExecutorConfig& executor_config,
     FragmentScheduleState* fragment_state, ScheduleState* state) {
   const TPlanFragment& fragment = fragment_state->fragment;
-  bool has_union = ContainsNode(fragment.plan, TPlanNodeType::UNION_NODE);
+  bool has_union = ContainsUnionNode(fragment.plan);
   DCHECK(has_union || ContainsScanNode(fragment.plan));
   // Build a map of hosts to the num instances this fragment should have, before we take
   // into account scan ranges. If this fragment has input fragments, we always run with
@@ -499,10 +519,7 @@ void Scheduler::CreateCollocatedAndScanInstances(const ExecutorConfig& executor_
       }
     }
   }
-  if (fragment.output_sink.__isset.table_sink
-      && fragment.output_sink.table_sink.__isset.hdfs_table_sink
-      && state->query_options().max_fs_writers > 0
-      && fragment_state->instance_states.size() > state->query_options().max_fs_writers) {
+  if (IsExceedMaxFsWriters(fragment_state, fragment_state, state)) {
     LOG(WARNING) << "Extra table sink instances scheduled, probably due to mismatch of "
                     "cluster state during planning vs scheduling. Expected: "
                  << state->query_options().max_fs_writers
@@ -536,11 +553,14 @@ void Scheduler::CreateInputCollocatedInstances(
       *state->GetFragmentScheduleState(fragment_state->exchange_input_fragments[0]);
   int per_fragment_instance_idx = 0;
 
-  if (fragment.output_sink.__isset.table_sink
-      && fragment.output_sink.table_sink.__isset.hdfs_table_sink
-      && state->query_options().max_fs_writers > 0
-      && input_fragment_state.instance_states.size()
-          > state->query_options().max_fs_writers) {
+  int max_instances = input_fragment_state.instance_states.size();
+  if (IsExceedMaxFsWriters(fragment_state, &input_fragment_state, state)) {
+    max_instances = state->query_options().max_fs_writers;
+  } else if (fragment.effective_instance_count > 0) {
+    max_instances = fragment.effective_instance_count;
+  }
+
+  if (max_instances != input_fragment_state.instance_states.size()) {
     std::unordered_set<std::pair<NetworkAddressPB, NetworkAddressPB>> all_hosts;
     for (const FInstanceScheduleState& input_instance_state :
         input_fragment_state.instance_states) {
@@ -550,7 +570,6 @@ void Scheduler::CreateInputCollocatedInstances(
     // across hosts and ensuring that instances on the same host get consecutive instance
     // indexes.
     int num_hosts = all_hosts.size();
-    int max_instances = state->query_options().max_fs_writers;
     int instances_per_host = max_instances / num_hosts;
     int remainder = max_instances % num_hosts;
     auto host_itr = all_hosts.begin();
@@ -787,6 +806,10 @@ bool Scheduler::ContainsScanNode(const TPlan& plan) {
   return ContainsNode(plan, SCAN_NODE_TYPES);
 }
 
+bool Scheduler::ContainsUnionNode(const TPlan& plan) {
+  return ContainsNode(plan, TPlanNodeType::UNION_NODE);
+}
+
 std::vector<TPlanNodeId> Scheduler::FindNodes(
     const TPlan& plan, const vector<TPlanNodeType::type>& types) {
   vector<TPlanNodeId> results;
diff --git a/be/src/scheduling/scheduler.h b/be/src/scheduling/scheduler.h
index f75bace41..084d9cb34 100644
--- a/be/src/scheduling/scheduler.h
+++ b/be/src/scheduling/scheduler.h
@@ -420,13 +420,17 @@ class Scheduler {
       std::vector<NetworkAddressPB>* scan_hosts);
 
   /// Return true if 'plan' contains a node of the given type.
-  bool ContainsNode(const TPlan& plan, TPlanNodeType::type type);
+  static bool ContainsNode(const TPlan& plan, TPlanNodeType::type type);
 
   /// Return true if 'plan' contains a node of one of the given types.
-  bool ContainsNode(const TPlan& plan, const std::vector<TPlanNodeType::type>& types);
+  static bool ContainsNode(
+      const TPlan& plan, const std::vector<TPlanNodeType::type>& types);
 
   /// Return true if 'plan' contains a scan node.
-  bool ContainsScanNode(const TPlan& plan);
+  static bool ContainsScanNode(const TPlan& plan);
+
+  /// Return true if 'plan' contains a union node.
+  static bool ContainsUnionNode(const TPlan& plan);
 
   /// Return all ids of nodes in 'plan' of any of the given types.
   std::vector<TPlanNodeId> FindNodes(
@@ -435,6 +439,24 @@ class Scheduler {
   /// Return all ids of all scan nodes in 'plan'.
   std::vector<TPlanNodeId> FindScanNodes(const TPlan& plan);
 
+  /// If TPlanFragment.effective_instance_count is positive, verify that resulting
+  /// instance_states size match with effective_instance_count. Fragment with UnionNode or
+  /// ScanNode or one where IsExceedMaxFsWriters equals true is not checked.
+  static void CheckEffectiveInstanceCount(
+      const FragmentScheduleState* fragment_state, const ScheduleState* state);
+
+  /// Check if sink_fragment_state has hdfs_table_sink AND ref_fragment_state scheduled
+  /// to exceed max_fs_writers query option.
+  static inline bool IsExceedMaxFsWriters(
+      const FragmentScheduleState* sink_fragment_state,
+      const FragmentScheduleState* ref_fragment_state, const ScheduleState* state) {
+    return (sink_fragment_state->fragment.output_sink.__isset.table_sink
+        && sink_fragment_state->fragment.output_sink.table_sink.__isset.hdfs_table_sink
+        && state->query_options().max_fs_writers > 0
+        && ref_fragment_state->instance_states.size()
+            > state->query_options().max_fs_writers);
+  }
+
   friend class impala::test::SchedulerWrapper;
   FRIEND_TEST(SimpleAssignmentTest, ComputeAssignmentDeterministicNonCached);
   FRIEND_TEST(SimpleAssignmentTest, ComputeAssignmentRandomNonCached);
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index e9c72c4ed..3e98dc619 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -1060,6 +1060,22 @@ Status impala::SetQueryOption(const string& key, const string& value,
         query_options->__set_stringify_map_keys(IsTrue(value));
         break;
       }
+      case TImpalaQueryOptions::COMPUTE_PROCESSING_COST: {
+        query_options->__set_compute_processing_cost(IsTrue(value));
+        break;
+      }
+      case TImpalaQueryOptions::PROCESSING_COST_MIN_THREADS: {
+        StringParser::ParseResult result;
+        const int32_t min_num =
+            StringParser::StringToInt<int32_t>(value.c_str(), value.length(), &result);
+        if (result != StringParser::PARSE_SUCCESS || min_num < 1 || min_num > 128) {
+          return Status(Substitute("$0 is not valid for processing_cost_min_threads. "
+                                   "Valid values are in [1, 128].",
+              value));
+        }
+        query_options->__set_processing_cost_min_threads(min_num);
+        break;
+      }
       default:
         if (IsRemovedQueryOption(key)) {
           LOG(WARNING) << "Ignoring attempt to set removed query option '" << key << "'";
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index 402582bf8..741888f84 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -50,7 +50,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
 // time we add or remove a query option to/from the enum TImpalaQueryOptions.
 #define QUERY_OPTS_TABLE                                                                 \
   DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),                                 \
-      TImpalaQueryOptions::ENABLE_TRIVIAL_QUERY_FOR_ADMISSION + 1);                      \
+      TImpalaQueryOptions::PROCESSING_COST_MIN_THREADS + 1);                             \
   REMOVED_QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED) \
   QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR, TQueryOptionLevel::REGULAR)               \
   REMOVED_QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)             \
@@ -283,7 +283,11 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
   QUERY_OPT_FN(codegen_cache_mode, CODEGEN_CACHE_MODE, TQueryOptionLevel::DEVELOPMENT)   \
   QUERY_OPT_FN(stringify_map_keys, STRINGIFY_MAP_KEYS, TQueryOptionLevel::ADVANCED)      \
   QUERY_OPT_FN(enable_trivial_query_for_admission, ENABLE_TRIVIAL_QUERY_FOR_ADMISSION,   \
-      TQueryOptionLevel::REGULAR);
+      TQueryOptionLevel::REGULAR)                                                        \
+  QUERY_OPT_FN(                                                                          \
+      compute_processing_cost, COMPUTE_PROCESSING_COST, TQueryOptionLevel::ADVANCED)     \
+  QUERY_OPT_FN(processing_cost_min_threads, PROCESSING_COST_MIN_THREADS,                 \
+      TQueryOptionLevel::ADVANCED);
 
 /// Enforce practical limits on some query options to avoid undesired query state.
 static const int64_t SPILLABLE_BUFFER_LIMIT = 1LL << 40; // 1 TB
diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc
index 1e8b9683f..d39c5c73b 100644
--- a/be/src/util/backend-gflag-util.cc
+++ b/be/src/util/backend-gflag-util.cc
@@ -197,8 +197,57 @@ DEFINE_string(ignored_dir_prefix_list, ".,_tmp.,_spark_metadata",
     "Comma separated list to specify the prefix for tmp/staging dirs that catalogd should"
     " skip in loading file metadata.");
 
+DEFINE_double_hidden(query_cpu_count_divisor, 1.0,
+    "(Advance) Divide the CPU requirement of a query to fit the total available CPU in "
+    "the executor group. For example, setting value 2 will fit the query with CPU "
+    "requirement 2X to an executor group with total available CPU X. Note that setting "
+    "with a fractional value less than 1 effectively multiplies the query CPU "
+    "requirement. A valid value is > 0.0. The default value is 1.");
+
+// TODO: Tune the individual expression cost from IMPALA-2805.
+DEFINE_bool_hidden(processing_cost_use_equal_expr_weight, true,
+    "(Advance) If true, all expression evaluations are weighted equally to 1 during the "
+    "plan node's processing cost calculation. If false, expression cost from IMPALA-2805 "
+    "will be used. Default to false.");
+
+// TODO: Benchmark and tune this config with an optimal value.
+DEFINE_int64_hidden(min_processing_per_thread, 10000000,
+    "(Advance) Minimum processing load (in processing cost unit) that a fragment "
+    "instance need to work on before planner consider increasing instance count. Used to "
+    "adjust fragment instance count based on estimated workload rather than the MT_DOP "
+    "setting. Setting this to high number will reduce parallelism of a fragment (more "
+    "workload per fragment), while setting to low number will increase parallelism (less "
+    "workload per fragment). Actual parallelism might still be constrained by the total "
+    "number of cores in selected executor group, MT_DOP, or PROCESSING_COST_MIN_THREAD "
+    "query option. Must be a positive integer. Default to 10M.");
+
+using strings::Substitute;
+
 namespace impala {
 
+// Flag validation
+// ------------------------------------------------------------
+static bool ValidateCpuCountDivisor(const char* flagname, double value) {
+  if (0.0 < value) {
+    return true;
+  }
+  LOG(ERROR) << Substitute(
+      "$0 must be greater than 0.0, value $1 is invalid", flagname, value);
+  return false;
+}
+
+static bool ValidateMinProcessingPerThread(const char* flagname, int64_t value) {
+  if (0 < value) {
+    return true;
+  }
+  LOG(ERROR) << Substitute(
+      "$0 must be a positive integer, value $1 is invalid", flagname, value);
+  return false;
+}
+
+DEFINE_validator(query_cpu_count_divisor, &ValidateCpuCountDivisor);
+DEFINE_validator(min_processing_per_thread, &ValidateMinProcessingPerThread);
+
 Status GetConfigFromCommand(const string& flag_cmd, string& result) {
   result.clear();
   if (flag_cmd.empty()) return Status::OK();
@@ -344,6 +393,10 @@ Status PopulateThriftBackendGflags(TBackendGflags& cfg) {
     DCHECK_EQ(FLAGS_geospatial_library, to_string(TGeospatialLibrary::HIVE_ESRI));
     cfg.__set_geospatial_library(TGeospatialLibrary::HIVE_ESRI);
   }
+  cfg.__set_query_cpu_count_divisor(FLAGS_query_cpu_count_divisor);
+  cfg.__set_processing_cost_use_equal_expr_weight(
+      FLAGS_processing_cost_use_equal_expr_weight);
+  cfg.__set_min_processing_per_thread(FLAGS_min_processing_per_thread);
   return Status::OK();
 }
 
diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift
index 809c4f88a..dec1f44ad 100644
--- a/common/thrift/BackendGflags.thrift
+++ b/common/thrift/BackendGflags.thrift
@@ -240,4 +240,10 @@ struct TBackendGflags {
   104: required bool enable_reload_events
 
   105: required TGeospatialLibrary geospatial_library
+
+  106: required double query_cpu_count_divisor
+
+  107: required bool processing_cost_use_equal_expr_weight
+
+  108: required i64 min_processing_per_thread
 }
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index add37998b..208bb9926 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -738,7 +738,7 @@ enum TImpalaQueryOptions {
   ORC_SCHEMA_RESOLUTION = 146;
 
   // Expands complex types in star queries
-  EXPAND_COMPLEX_TYPES = 147
+  EXPAND_COMPLEX_TYPES = 147;
 
   // Specify the database name which stores global udf
   FALLBACK_DB_FOR_FUNCTIONS = 148;
@@ -760,6 +760,14 @@ enum TImpalaQueryOptions {
 
   // Enable immediate admission for trivial queries.
   ENABLE_TRIVIAL_QUERY_FOR_ADMISSION = 152
+
+  // Control whether to consider CPU processing cost during query planning.
+  COMPUTE_PROCESSING_COST = 153;
+
+  // Minimum number of threads of a query fragment per host in processing
+  // cost algorithm. It is recommend to not set it with value more than number of
+  // physical cores in executor node. Valid values are in [1, 128]. Default to 1.
+  PROCESSING_COST_MIN_THREADS = 154;
 }
 
 // The summary of a DML statement.
diff --git a/common/thrift/Planner.thrift b/common/thrift/Planner.thrift
index ba206cb39..5eb11c452 100644
--- a/common/thrift/Planner.thrift
+++ b/common/thrift/Planner.thrift
@@ -83,6 +83,13 @@ struct TPlanFragment {
   // Maximum number of required threads that will be executing concurrently for this plan
   // fragment, i.e. the number of threads that this query needs to execute successfully.
   10: optional i64 thread_reservation
+
+  // The effective number of parallelism for this fragment that dictated by the frontend
+  // planner. If the frontend planner set this to a positive number, the backend scheduler
+  // must make sure that it schedules no more than this many instance fragments. Currently
+  // not enforced on fragments having scan nodes or union nodes or when fragment need to
+  // exceed max_fs_writers query option (see IMPALA-8125).
+  14: optional i32 effective_instance_count
 }
 
 // location information for a single scan range
diff --git a/common/thrift/Query.thrift b/common/thrift/Query.thrift
index 49d34b40d..b190f2f11 100644
--- a/common/thrift/Query.thrift
+++ b/common/thrift/Query.thrift
@@ -618,6 +618,12 @@ struct TQueryOptions {
 
   // See comment in ImpalaService.thrift
   153: optional bool enable_trivial_query_for_admission = true;
+
+  // See comment in ImpalaService.thrift
+  154: optional bool compute_processing_cost = false;
+
+  // See comment in ImpalaService.thrift
+  155: optional i32 processing_cost_min_threads = 1;
 }
 
 // Impala currently has three types of sessions: Beeswax, HiveServer2 and external
@@ -898,5 +904,10 @@ struct TQueryExecRequest {
 
   // Indicate whether the request is a trivial query. Used by admission control.
   13: optional bool is_trivial_query
+
+  // CPU core count required to run the query. Used by admission control to decide which
+  // executor group to run the query. Non-positive value means no specific CPU core count
+  // is required.
+  14: optional i32 cores_required;
 }
 
diff --git a/fe/src/main/java/org/apache/impala/planner/CoreCount.java b/fe/src/main/java/org/apache/impala/planner/CoreCount.java
new file mode 100644
index 000000000..e8fc87101
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/CoreCount.java
@@ -0,0 +1,110 @@
+// 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.
+
+package org.apache.impala.planner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.impala.common.Id;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * A container class that represent CPU cores, computed from the CPU cost, of certain
+ * subtree of a query or the query itself.
+ */
+public class CoreCount {
+  // List of Id (either PlanFragmentId or PlanNodeId) that describe the origin of counts_
+  // element.
+  private final ImmutableList<Id> ids_;
+
+  // List of CPU core count contributing to this CoreCount.
+  private final ImmutableList<Integer> counts_;
+
+  // Sum of all elements in count_.
+  // Cached after the first call of total().
+  private int total_ = -1;
+
+  public CoreCount(Id id, int count) {
+    Preconditions.checkArgument(count >= 0, "Core count must be a non-negative number");
+    ids_ = ImmutableList.of(id);
+    counts_ = ImmutableList.of(count);
+  }
+
+  private CoreCount(ImmutableList<Id> ids, ImmutableList<Integer> counts) {
+    Preconditions.checkArgument(
+        ids.size() == counts.size(), "ids and counts must have same size!");
+    ids_ = ids;
+    counts_ = counts;
+  }
+
+  public int total() {
+    if (total_ < 0) {
+      total_ = counts_.stream().mapToInt(v -> v).sum();
+    }
+    return total_;
+  }
+
+  @Override
+  public String toString() {
+    if (ids_.isEmpty()) {
+      return "<empty>";
+    } else {
+      StringBuilder sb = new StringBuilder();
+      sb.append("{total=");
+      sb.append(total());
+      sb.append(" trace=");
+      sb.append(IntStream.range(0, ids_.size())
+                    .mapToObj(i
+                        -> ((ids_.get(i) instanceof PlanNodeId) ? "N" : "") + ids_.get(i)
+                            + ":" + counts_.get(i))
+                    .collect(Collectors.joining("+")));
+      sb.append("}");
+      return sb.toString();
+    }
+  }
+
+  protected static CoreCount sum(List<CoreCount> cores) {
+    ImmutableList.Builder<Id> idBuilder = new ImmutableList.Builder<Id>();
+    ImmutableList.Builder<Integer> countBuilder = new ImmutableList.Builder<Integer>();
+    for (CoreCount coreRequirement : cores) {
+      idBuilder.addAll(coreRequirement.ids_);
+      countBuilder.addAll(coreRequirement.counts_);
+    }
+    return new CoreCount(idBuilder.build(), countBuilder.build());
+  }
+
+  protected static CoreCount sum(CoreCount core1, CoreCount core2) {
+    ImmutableList.Builder<Id> idBuilder = new ImmutableList.Builder<Id>();
+    ImmutableList.Builder<Integer> countBuilder = new ImmutableList.Builder<Integer>();
+
+    idBuilder.addAll(core1.ids_);
+    idBuilder.addAll(core2.ids_);
+    countBuilder.addAll(core1.counts_);
+    countBuilder.addAll(core2.counts_);
+
+    return new CoreCount(idBuilder.build(), countBuilder.build());
+  }
+
+  protected static CoreCount max(CoreCount core1, CoreCount core2) {
+    return (core1.total() < core2.total()) ? core2 : core1;
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/CostingSegment.java b/fe/src/main/java/org/apache/impala/planner/CostingSegment.java
new file mode 100644
index 000000000..a3c5ed775
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/CostingSegment.java
@@ -0,0 +1,235 @@
+// 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.
+
+package org.apache.impala.planner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.apache.impala.common.Id;
+import org.apache.impala.common.Pair;
+import org.apache.impala.common.TreeNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * A grouping of adjacent PlanNode and DataSink of a fragment for CPU costing purpose.
+ * <p>
+ * Each fragment segment is a subtree of PlanNodes/DataSink in the fragment with a
+ * DataSink or blocking PlanNode as root. All other nodes in the segment are non-blocking.
+ * In most cases, every segment is a blocking segment, that is the root is either
+ * a blocking operator or blocking DataSink (ie., JoinBuildSink). A segment is not a
+ * blocking segment if its root is a non-blocking DataSink (ie., DataStreamSink).
+ * <p>
+ * PlanNodes or DataSink that belong to the same segment will have their ProcessingCost
+ * summed. Analyses done through this class might recurse around the CostingSegment tree,
+ * but should not traverse into different fragment.
+ */
+public class CostingSegment extends TreeNode<CostingSegment> {
+  private final static Logger LOG = LoggerFactory.getLogger(CostingSegment.class);
+
+  // List of PlanNode belong to this segment.
+  private List<PlanNode> nodes_ = Lists.newArrayList();
+
+  // The ProcessingCost of this fragment segment, which is the sum of the processing cost
+  // of all nodes in nodes_ and sink_ (if set).
+  private ProcessingCost cost_ = ProcessingCost.zero();
+
+  // DataSink associated with this segment.
+  // Must not be null for output segment.
+  private DataSink sink_ = null;
+
+  public CostingSegment(DataSink sink) { setSink(sink); }
+
+  public CostingSegment(PlanNode node) { appendNode(node); }
+
+  private CostingSegment() {}
+
+  public ProcessingCost getProcessingCost() { return cost_; }
+  public boolean isOutputSegment() { return sink_ != null; }
+
+  private Id getRootId() {
+    if (isOutputSegment()) {
+      return sink_.getFragment().getId();
+    } else {
+      Preconditions.checkState(!nodes_.isEmpty());
+      return nodes_.get(nodes_.size() - 1).getId();
+    }
+  }
+
+  private void appendCost(ProcessingCost additionalCost) {
+    Preconditions.checkArgument(additionalCost.isValid());
+    ProcessingCost newTotalCost = ProcessingCost.sumCost(additionalCost, cost_);
+    newTotalCost.setNumRowToConsume(cost_.getNumRowToConsume());
+    newTotalCost.setNumRowToProduce(additionalCost.getNumRowToConsume());
+    cost_ = newTotalCost;
+  }
+
+  protected void setSink(DataSink sink) {
+    appendCost(sink.getProcessingCost());
+    sink_ = sink;
+  }
+
+  protected void appendNode(PlanNode node) {
+    appendCost(node.getProcessingCost());
+    nodes_.add(node);
+  }
+
+  /**
+   * Gather {@link CoreCount} rooted from this segment and populate
+   * subtreeCoreBuilder with {@link CoreCount} of child-blocking-subtree.
+   * @param fragmentCoreState A map holding per-fragment core state.
+   * @param subtreeCoreBuilder An ImmutableList builder to populate.
+   * @return A {@link CoreCount} value of segment tree
+   * rooted at this segment.
+   */
+  protected CoreCount traverseBlockingAwareCores(
+      Map<PlanFragmentId, Pair<CoreCount, List<CoreCount>>> fragmentCoreState,
+      ImmutableList.Builder<CoreCount> subtreeCoreBuilder) {
+    CoreCount segmentCore = new CoreCount(getRootId(), cost_.getNumInstancesExpected());
+    // If not in input segment, gather cost of children first.
+    for (CostingSegment childSegment : getChildren()) {
+      CoreCount childSegmentCores =
+          childSegment.traverseBlockingAwareCores(fragmentCoreState, subtreeCoreBuilder);
+      if (childSegmentCores.total() > 0) {
+        segmentCore = CoreCount.max(segmentCore, childSegmentCores);
+      }
+    }
+
+    // Look up child fragment that is connected through this segment.
+    for (PlanNode node : nodes_) {
+      for (int i = 0; i < node.getChildCount(); i++) {
+        PlanFragment childFragment = node.getChild(i).getFragment();
+        if (childFragment == node.getFragment()) continue;
+
+        Pair<CoreCount, List<CoreCount>> childCores =
+            fragmentCoreState.get(childFragment.getId());
+        Preconditions.checkNotNull(childCores);
+
+        if (childFragment.hasBlockingNode()) {
+          CoreCount childCoreCount =
+              childFragment.maxCore(childCores.first, CoreCount.sum(childCores.second));
+          subtreeCoreBuilder.add(childCoreCount);
+        } else {
+          Preconditions.checkState(node instanceof ExchangeNode);
+          Preconditions.checkState(i == 0);
+          segmentCore = CoreCount.sum(segmentCore, childCores.first);
+          subtreeCoreBuilder.addAll(childCores.second);
+        }
+      }
+    }
+
+    return segmentCore;
+  }
+
+  /**
+   * Try to come up with lower parallelism for this segment by comparing the output
+   * ProcessingCost of child segment or input fragment.
+   * <p>
+   * This segment cost is the Consumer cost, while the ProcessingCost of child segment
+   * or input fragment is the Producer cost. It compares between per-row production cost
+   * of Producer versus the per-row consumption cost of Consumer. The expected parallelism
+   * of Consumer (this segment) then is adjusted to get closer to the produce-consume
+   * ratio when compared to the Producer.
+   *
+   * @param nodeStepCount The step count used to increase this fragment's parallelism.
+   *                      Usually equal to number of nodes or just 1.
+   * @param minParallelism The minimum parallelism of this segment.
+   * @param maxParallelism The maximum parallelism this segment is allowed to adjust to.
+   */
+  protected int tryAdjustParallelism(
+      int nodeStepCount, int minParallelism, int maxParallelism) {
+    // TODO: The ratio based adjustment can be further improved by considering the plan
+    //   nodes too. For example, one does not need to make DoP of the parent of a scale
+    //   aggregate fragment the same as the aggregate fragment, say the aggregate fragment
+    //   produces one row and uses 10 instances.
+    int newParallelism = minParallelism;
+    ProcessingCost producerCost = ProcessingCost.zero();
+
+    if (getChildCount() > 0) {
+      for (CostingSegment childSegment : getChildren()) {
+        newParallelism = Math.max(newParallelism,
+            childSegment.tryAdjustParallelism(
+                nodeStepCount, minParallelism, maxParallelism));
+      }
+      producerCost = mergeCostingSegment(getChildren()).getProcessingCost();
+    }
+
+    // If this segment has UnionNode, it may have ExchangeNode belonging to this segment.
+    List<ProcessingCost> childOutputCosts =
+        nodes_.stream()
+            .filter(Predicates.instanceOf(ExchangeNode.class))
+            .map(p -> p.getChild(0).getFragment().getLastCostingSegment())
+            .collect(Collectors.toList());
+
+    if (!childOutputCosts.isEmpty()) {
+      if (producerCost.getTotalCost() > 0) childOutputCosts.add(producerCost);
+      producerCost = ProcessingCost.fullMergeCosts(childOutputCosts);
+    }
+
+    ProcessingCost.tryAdjustConsumerParallelism(
+        nodeStepCount, minParallelism, maxParallelism, producerCost, cost_);
+    newParallelism = cost_.getNumInstancesExpected();
+    Preconditions.checkState(newParallelism >= minParallelism);
+    Preconditions.checkState(newParallelism <= maxParallelism);
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Adjust ProcessingCost on {}. minParallelism={} maxParallelism={} "
+              + "newParallelism={} consumerCost={} consumerInstCount={} producerCost={} "
+              + "producerInstCount={}",
+          getRootId(), minParallelism, maxParallelism, newParallelism,
+          cost_.getTotalCost(), cost_.getNumInstancesExpected(),
+          producerCost.getTotalCost(), producerCost.getNumInstancesExpected());
+    }
+
+    return newParallelism;
+  }
+
+  /**
+   * Merge given list of CostingSegment into a new combined CostingSegment.
+   * <p>
+   * The resulting CostingSegment will contain all nodes, children segments, and sum of
+   * ProcessingCost from all given CostingSegment.
+   *
+   * @param costingSegments List of CostingSegment to merge. Must not be empty and must
+   *                        not contain output segment (segment with DataSink set).
+   * @return A combined CostingSegment.
+   */
+  protected static CostingSegment mergeCostingSegment(
+      List<CostingSegment> costingSegments) {
+    Preconditions.checkNotNull(costingSegments);
+    Preconditions.checkArgument(!costingSegments.isEmpty());
+
+    if (costingSegments.size() == 1) return costingSegments.get(0);
+    CostingSegment mergedCost = new CostingSegment();
+    List<ProcessingCost> allCosts = Lists.newArrayList();
+    for (CostingSegment costingSegment : costingSegments) {
+      Preconditions.checkArgument(!costingSegment.isOutputSegment());
+      mergedCost.nodes_.addAll(costingSegment.nodes_);
+      mergedCost.addChildren(costingSegment.getChildren());
+      allCosts.add(costingSegment.getProcessingCost());
+    }
+    mergedCost.cost_ = ProcessingCost.fullMergeCosts(allCosts);
+    return mergedCost;
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
index 44232b53d..62f2a50ca 100644
--- a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
@@ -208,7 +208,10 @@ public class ExchangeNode extends PlanNode {
 
     if (isBroadcastExchange()) {
       processingCost_ = ProcessingCost.broadcastCost(processingCost_,
-          () -> getNumReceivers());
+          ()
+              -> fragment_.hasAdjustedInstanceCount() ?
+              fragment_.getAdjustedInstanceCount() :
+              getNumReceivers());
     }
   }
 
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
index 399e45829..8c6df6174 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
@@ -378,4 +378,10 @@ public class HdfsTableSink extends TableSink {
     }
     return num_instances;
   }
+
+  @Override
+  public void computeRowConsumptionAndProductionToCost() {
+    super.computeRowConsumptionAndProductionToCost();
+    fragment_.setFixedInstanceCount(fragment_.getNumInstances());
+  }
 }
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 feed83381..ab268c0c4 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
@@ -163,4 +163,12 @@ public class JoinBuildSink extends DataSink {
   public void collectExprs(List<Expr> exprs) {
     exprs.addAll(buildExprs_);
   }
+
+  @Override
+  public void computeRowConsumptionAndProductionToCost() {
+    super.computeRowConsumptionAndProductionToCost();
+    if (isShared()) {
+      fragment_.setFixedInstanceCount(getNumInstances());
+    }
+  }
 }
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 1093014bf..c50877cbf 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -17,28 +17,39 @@
 
 package org.apache.impala.planner;
 
+import java.math.RoundingMode;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
+
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.InternalException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.common.PrintUtils;
 import org.apache.impala.common.TreeNode;
 import org.apache.impala.planner.JoinNode.DistributionMode;
 import org.apache.impala.planner.PlanNode.ExecPhaseResourceProfiles;
 import org.apache.impala.planner.RuntimeFilterGenerator.RuntimeFilter;
+import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPartitionType;
 import org.apache.impala.thrift.TPlanFragment;
 import org.apache.impala.thrift.TPlanFragmentTree;
 import org.apache.impala.thrift.TQueryOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.math.IntMath;
+import com.google.common.math.LongMath;
 
 /**
  * PlanFragments form a tree structure via their ExchangeNodes. A tree of fragments
@@ -77,6 +88,7 @@ import com.google.common.base.Predicates;
  * - toThrift()
  */
 public class PlanFragment extends TreeNode<PlanFragment> {
+  private final static Logger LOG = LoggerFactory.getLogger(PlanFragment.class);
   private final PlanFragmentId fragmentId_;
   private PlanId planId_;
   private CohortId cohortId_;
@@ -129,6 +141,32 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   // on a backend.
   private long consumedGlobalRuntimeFiltersMemReservationBytes_ = 0;
 
+  // The root of segment costs tree of this fragment.
+  // Individual element of the tree describe processing cost of subset of plan nodes
+  // that is divided by blocking PlanNode/DataSink boundary. Together, they describe total
+  // processing cost of this fragment. Set in computeCostingSegment().
+  private CostingSegment rootSegment_;
+
+  // Maximum allowed parallelism based on minimum processing load per fragment.
+  // Set in getCostBasedMaxParallelism().
+  private int costBasedMaxParallelism_ = -1;
+
+  // An adjusted number of instance based on ProcessingCost calculation.
+  // A positive value implies that the instance count has been adjusted, either through
+  // traverseEffectiveParallelism() or by fragment member (PlanNode or DataSink) calling
+  // setFixedInstanceCount(). Internally, this must be set through
+  // setAdjustedInstanceCount().
+  private int adjustedInstanceCount_ = -1;
+
+  // Mark if this fragment has a fixed instance count dictated by any of its PlanNode or
+  // DataSink member.
+  private boolean isFixedParallelism_ = false;
+
+  // The original instance count before ProcessingCost based adjustment.
+  // Set in setEffectiveNumInstance() and only set if instance count differ between
+  // the original plan vs the ProcessingCost based plan.
+  private int originalInstanceCount_ = -1;
+
   public long getProducedRuntimeFiltersMemReservationBytes() {
     return producedRuntimeFiltersMemReservationBytes_;
   }
@@ -187,6 +225,89 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     }
   }
 
+  /**
+   * Compute processing cost of PlanNodes and DataSink of this fragment, and aggregate
+   * them into {@link CostingSegment} rooted at {@link #rootSegment_}.
+   * <p>For example, given the following fragment plan:
+   * <pre>
+   * F03:PLAN FRAGMENT [HASH(i_class)] hosts=3 instances=3
+   * fragment-costs=[34550429, 2159270, 23752870, 1]
+   * 08:TOP-N [LIMIT=100]
+   * |  cost=900
+   * |
+   * 07:ANALYTIC
+   * |  cost=23751970
+   * |
+   * 06:SORT
+   * |  cost=2159270
+   * |
+   * 12:AGGREGATE [FINALIZE]
+   * |  cost=34548320
+   * |
+   * 11:EXCHANGE [HASH(i_class)]
+   *    cost=2109
+   * </pre>
+   * The post-order traversal of {@link #rootSegment_} tree show processing cost detail of
+   * {@code [(2109+34548320), 2159270, (23751970+900), 1]}.
+   * The DataSink with cost 1 is a separate segment since the last PlanNode (TOP-N) is a
+   * blocking node.
+   *
+   * @param queryOptions A query options for this query.
+   */
+  public void computeCostingSegment(TQueryOptions queryOptions) {
+    for (PlanNode node : collectPlanNodes()) {
+      node.computeProcessingCost(queryOptions);
+      node.computeRowConsumptionAndProductionToCost();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("ProcessingCost Node " + node.getProcessingCost().debugString());
+      }
+    }
+    sink_.computeProcessingCost(queryOptions);
+    sink_.computeRowConsumptionAndProductionToCost();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("ProcessingCost Sink " + sink_.getProcessingCost().debugString());
+    }
+
+    CostingSegment topSegment = collectCostingSegmentHelper(planRoot_);
+
+    if (isBlockingNode(planRoot_)) {
+      rootSegment_ = new CostingSegment(sink_);
+      rootSegment_.addChild(topSegment);
+    } else {
+      topSegment.setSink(sink_);
+      rootSegment_ = topSegment;
+    }
+  }
+
+  private CostingSegment collectCostingSegmentHelper(PlanNode root) {
+    Preconditions.checkNotNull(root);
+
+    List<CostingSegment> blockingChildSegments = Lists.newArrayList();
+    List<CostingSegment> nonBlockingChildSegments = Lists.newArrayList();
+    for (PlanNode child : root.getChildren()) {
+      if (child.getFragment() != this) continue;
+      CostingSegment childCostingSegment = collectCostingSegmentHelper(child);
+
+      if (isBlockingNode(child)) {
+        blockingChildSegments.add(childCostingSegment);
+      } else {
+        nonBlockingChildSegments.add(childCostingSegment);
+      }
+    }
+
+    CostingSegment thisSegment;
+    if (nonBlockingChildSegments.isEmpty()) {
+      // No child or all children are blocking nodes.
+      thisSegment = new CostingSegment(root);
+    } else {
+      thisSegment = CostingSegment.mergeCostingSegment(nonBlockingChildSegments);
+      thisSegment.appendNode(root);
+    }
+
+    if (!blockingChildSegments.isEmpty()) thisSegment.addChildren(blockingChildSegments);
+    return thisSegment;
+  }
+
   /**
    * Do any final work to set up the ExchangeNodes and DataStreamSinks for this fragment.
    * If this fragment has partitioned joins, ensures that the corresponding partition
@@ -462,6 +583,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     result.setBackend_min_mem_reservation_bytes(
         perBackendResourceProfile_.getMinMemReservationBytes());
     result.setThread_reservation(perInstanceResourceProfile_.getThreadReservation());
+    result.setEffective_instance_count(getAdjustedInstanceCount());
     return result;
   }
 
@@ -496,7 +618,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
       prefix = "  ";
       rootPrefix = "  ";
       detailPrefix = prefix + "|  ";
-      str.append(getFragmentHeaderString("", "", queryOptions.getMt_dop()));
+      str.append(getFragmentHeaderString("", "", queryOptions, detailLevel));
       if (sink_ != null && sink_ instanceof DataStreamSink) {
         str.append(
             sink_.getExplainString(rootPrefix, detailPrefix, queryOptions, detailLevel));
@@ -504,7 +626,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     } else if (detailLevel == TExplainLevel.EXTENDED) {
       // Print a fragment prefix displaying the # nodes and # instances
       str.append(
-          getFragmentHeaderString(rootPrefix, detailPrefix, queryOptions.getMt_dop()));
+          getFragmentHeaderString(rootPrefix, detailPrefix, queryOptions, detailLevel));
       rootPrefix = prefix;
     }
 
@@ -530,12 +652,17 @@ public class PlanFragment extends TreeNode<PlanFragment> {
    * Get a header string for a fragment in an explain plan.
    */
   public String getFragmentHeaderString(String firstLinePrefix, String detailPrefix,
-      int mt_dop) {
+      TQueryOptions queryOptions, TExplainLevel explainLevel) {
+    int mt_dop = queryOptions.getMt_dop();
     StringBuilder builder = new StringBuilder();
     builder.append(String.format("%s%s:PLAN FRAGMENT [%s]", firstLinePrefix,
         fragmentId_.toString(), dataPartition_.getExplainString()));
     builder.append(PrintUtils.printNumHosts(" ", getNumNodes()));
     builder.append(PrintUtils.printNumInstances(" ", getNumInstances()));
+    if (ProcessingCost.isComputeCost(queryOptions)
+        && originalInstanceCount_ != getNumInstances()) {
+      builder.append(" (adjusted from " + originalInstanceCount_ + ")");
+    }
     builder.append("\n");
     String perHostPrefix = mt_dop == 0 ?
         "Per-Host Resources: " : "Per-Host Shared Resources: ";
@@ -589,6 +716,25 @@ public class PlanFragment extends TreeNode<PlanFragment> {
       builder.append(perInstanceExplainString);
       builder.append("\n");
     }
+    if (ProcessingCost.isComputeCost(queryOptions) && rootSegment_ != null
+        && explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+      // Print processing cost.
+      builder.append(detailPrefix);
+      builder.append("max-parallelism=");
+      if (isFixedParallelism_
+          || ((sink_ instanceof JoinBuildSink) && !((JoinBuildSink) sink_).isShared())) {
+        builder.append(getAdjustedInstanceCount());
+      } else {
+        builder.append(getCostBasedMaxParallelism());
+      }
+      builder.append(" fragment-costs=");
+      builder.append(costingSegmentSummary());
+      builder.append("\n");
+      if (explainLevel.ordinal() >= TExplainLevel.VERBOSE.ordinal()) {
+        builder.append(explainProcessingCosts(detailPrefix, false));
+        builder.append("\n");
+      }
+    }
     return builder.toString();
   }
 
@@ -709,4 +855,292 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     // We add one to prevent having a hash seed of 0.
     return planRoot_.getId().asInt() + 1;
   }
+
+  /**
+   * Get maximum allowed parallelism based on minimum processing load per fragment.
+   * <p>This is controlled by {@code min_processing_per_thread} flag. Only valid after
+   * {@link #computeCostingSegment(TQueryOptions)} has been called.
+   *
+   * @return maximum allowed parallelism based on minimum processing load per fragment.
+   */
+  protected int getCostBasedMaxParallelism() {
+    if (costBasedMaxParallelism_ >= 0) return costBasedMaxParallelism_;
+
+    ProcessingCost maxCostingSegment = ProcessingCost.zero();
+    List<CostingSegment> allSegments = rootSegment_.getNodesPostOrder();
+    for (CostingSegment costingSegment : allSegments) {
+      maxCostingSegment =
+          ProcessingCost.maxCost(maxCostingSegment, costingSegment.getProcessingCost());
+    }
+
+    long maxParallelism = LongMath.divide(maxCostingSegment.getTotalCost(),
+        BackendConfig.INSTANCE.getMinProcessingPerThread(), RoundingMode.CEILING);
+    // Round up to the nearest multiple of numNodes.
+    // Little over-parallelize is better than under-parallelize.
+    int numNodes = getNumNodes();
+    maxParallelism =
+        LongMath.divide(maxParallelism, numNodes, RoundingMode.CEILING) * numNodes;
+
+    if (maxParallelism <= 0) {
+      costBasedMaxParallelism_ = 1;
+    } else if (maxParallelism <= Integer.MAX_VALUE) {
+      costBasedMaxParallelism_ = (int) maxParallelism;
+    } else {
+      // Floor Integer.MAX_VALUE to the nearest multiple of numNodes.
+      costBasedMaxParallelism_ = Integer.MAX_VALUE - (Integer.MAX_VALUE % numNodes);
+    }
+    return costBasedMaxParallelism_;
+  }
+
+  protected boolean hasBlockingNode() {
+    if (sink_ instanceof JoinBuildSink) return true;
+    for (PlanNode p : collectPlanNodes()) {
+      if (isBlockingNode(p)) return true;
+    }
+    return false;
+  }
+
+  protected boolean hasAdjustedInstanceCount() { return adjustedInstanceCount_ > 0; }
+
+  protected void setFixedInstanceCount(int count) {
+    isFixedParallelism_ = true;
+    setAdjustedInstanceCount(count);
+  }
+
+  private void setAdjustedInstanceCount(int count) {
+    Preconditions.checkState(count > 0,
+        getId() + " adjusted instance count (" + count + ") is not positive number.");
+    boolean isFirstAdjustment = adjustedInstanceCount_ <= 0;
+    adjustedInstanceCount_ = count;
+    if (rootSegment_ != null) {
+      List<CostingSegment> costingSegments = rootSegment_.getNodesPostOrder();
+      for (CostingSegment costingSegment : costingSegments) {
+        // Reset for each segment cost since it might be overriden during
+        // tryLowerParallelism().
+        costingSegment.getProcessingCost().setNumInstanceExpected(
+            this::getAdjustedInstanceCount);
+      }
+    }
+
+    if (isFirstAdjustment) {
+      // Set num instance expected for ProcessingCost attached to PlanNodes and DataSink.
+      for (PlanNode node : collectPlanNodes()) {
+        node.getProcessingCost().setNumInstanceExpected(this::getAdjustedInstanceCount);
+      }
+      sink_.getProcessingCost().setNumInstanceExpected(this::getAdjustedInstanceCount);
+    }
+  }
+
+  protected int getAdjustedInstanceCount() { return adjustedInstanceCount_; }
+
+  protected ProcessingCost getLastCostingSegment() {
+    return rootSegment_.getProcessingCost();
+  }
+
+  private List<Long> costingSegmentSummary() {
+    return rootSegment_.getNodesPostOrder()
+        .stream()
+        .map(s -> ((CostingSegment) s).getProcessingCost().getTotalCost())
+        .collect(Collectors.toList());
+  }
+
+  private String explainProcessingCosts(String linePrefix, boolean fullExplain) {
+    return rootSegment_.getNodesPreOrder()
+        .stream()
+        .map(s
+            -> ((CostingSegment) s)
+                   .getProcessingCost()
+                   .getExplainString(linePrefix, fullExplain))
+        .collect(Collectors.joining("\n"));
+  }
+
+  private String debugProcessingCosts() { return explainProcessingCosts("", true); }
+
+  /**
+   * Validates that properties related to processing cost of this fragment are complete
+   * and valid.
+   */
+  private void validateProcessingCosts() {
+    Preconditions.checkState(hasAdjustedInstanceCount());
+    Preconditions.checkNotNull(rootSegment_);
+    List<CostingSegment> costingSegments = rootSegment_.getNodesPreOrder();
+    for (CostingSegment costingSegment : costingSegments) {
+      ProcessingCost cost = costingSegment.getProcessingCost();
+      Preconditions.checkState(cost.isValid());
+      Preconditions.checkState(
+          cost.getNumInstancesExpected() == getAdjustedInstanceCount());
+    }
+  }
+
+  /**
+   * Traverse down the query tree starting from this fragment and calculate the effective
+   * parallelism of each PlanFragments.
+   *
+   * @param minThreadPerNode Minimum thread per fragment per node based on
+   *                         {@code processing_cost_min_threads} flag.
+   * @param maxThreadPerNode Maximum thread per fragment per node based on
+   *                         TExecutorGroupSet.num_cores_per_executor flag.
+   * @param parentParallelism Number of instance of parent fragment.
+   */
+  protected void traverseEffectiveParallelism(
+      int minThreadPerNode, int maxThreadPerNode, int parentParallelism) {
+    Preconditions.checkNotNull(
+        rootSegment_, "ProcessingCost Fragment %s has not been computed!", getId());
+    int nodeStepCount = getNumInstances() % getNumNodes() == 0 ? getNumNodes() : 1;
+
+    // step 1: Set initial parallelism to the maximum possible.
+    //   Subsequent steps after this will not exceed maximum parallelism sets here.
+    boolean canTryLower =
+        adjustToMaxParallelism(maxThreadPerNode, parentParallelism, nodeStepCount);
+
+    if (canTryLower) {
+      // step 2: Try lower parallelism by comparing output ProcessingCost of the input
+      //   child fragment against this fragment's segment costs.
+      Preconditions.checkState(getChildCount() > 0);
+      Preconditions.checkState(getChild(0).getSink() instanceof DataStreamSink);
+
+      // Check if this fragment parallelism can be lowered.
+      int maxParallelism = getAdjustedInstanceCount();
+      int effectiveParallelism = rootSegment_.tryAdjustParallelism(
+          nodeStepCount, minThreadPerNode, maxParallelism);
+      setAdjustedInstanceCount(effectiveParallelism);
+      if (LOG.isTraceEnabled() && effectiveParallelism != maxParallelism) {
+        logCountAdjustmentTrace(maxParallelism, effectiveParallelism,
+            "Lower parallelism based on load and produce-consume rate ratio.");
+      }
+    }
+    validateProcessingCosts();
+
+    // step 3: Compute the parallelism of join build fragment.
+    //   Child parallelism may be enforced to follow this fragment's parallelism.
+    // TODO: This code assume that probe side of the join always have higher per-instance
+    //   cost than the build side. If this assumption is false and the child is a
+    //   non-shared join build fragment, then this fragment should increase its
+    //   parallelism to match the child fragment parallelism.
+    for (PlanFragment child : getChildren()) {
+      if (child.getSink() instanceof JoinBuildSink) {
+        child.traverseEffectiveParallelism(
+            minThreadPerNode, maxThreadPerNode, getAdjustedInstanceCount());
+      }
+    }
+  }
+
+  /**
+   * Adjust parallelism of this fragment to the maximum allowed.
+   *
+   * @param maxThreadPerNode Maximum thread per fragment per node based on
+   *                         TExecutorGroupSet.num_cores_per_executor flag.
+   * @param parentParallelism Parallelism of parent fragment.
+   * @param nodeStepCount The step count used to increase this fragment's parallelism.
+   *                      Usually equal to number of nodes or just 1.
+   * @return True if it is possible to lower this fragment's parallelism through
+   * ProcessingCost comparison. False if the parallelism should not be changed anymore.
+   */
+  private boolean adjustToMaxParallelism(
+      int maxThreadPerNode, int parentParallelism, int nodeStepCount) {
+    boolean canTryLower = true;
+    // Compute maximum allowed parallelism.
+    int maxParallelism = getNumInstances();
+    if (isFixedParallelism_) {
+      maxParallelism = getAdjustedInstanceCount();
+      canTryLower = false;
+    } else if ((sink_ instanceof JoinBuildSink) && !((JoinBuildSink) sink_).isShared()) {
+      // This is a non-shared (PARTITIONED) join build fragment.
+      // Parallelism of this fragment is equal to its parent parallelism.
+      Preconditions.checkState(parentParallelism > 0);
+      if (LOG.isTraceEnabled() && maxParallelism != parentParallelism) {
+        logCountAdjustmentTrace(maxParallelism, parentParallelism,
+            "Partitioned join build fragment follow parent's parallelism.");
+      }
+      maxParallelism = parentParallelism;
+      canTryLower = false; // no need to compute effective parallelism anymore.
+    } else {
+      // TODO: Fragment with UnionNode but without ScanNode should have its parallelism
+      //   bounded by the maximum parallelism between its exchanging child.
+      //   For now, it wont get here since fragment with UnionNode has fixed parallelism
+      //   (equal to MT_DOP, and previouslyAdjusted == true).
+      maxParallelism = IntMath.saturatedMultiply(maxThreadPerNode, getNumNodes());
+      int costBasedMaxParallelism = Math.max(nodeStepCount, getCostBasedMaxParallelism());
+      if (costBasedMaxParallelism < maxParallelism) {
+        maxParallelism = costBasedMaxParallelism;
+      }
+
+      if (LOG.isTraceEnabled() && maxParallelism != getNumInstances()) {
+        if (maxParallelism == maxThreadPerNode) {
+          logCountAdjustmentTrace(
+              getNumInstances(), maxParallelism, "Follow maxThreadPerNode.");
+        } else {
+          logCountAdjustmentTrace(
+              getNumInstances(), maxParallelism, "Follow minimum work per thread.");
+        }
+      }
+    }
+
+    // Initialize this fragment's parallelism to the maxParallelism.
+    setAdjustedInstanceCount(maxParallelism);
+    return canTryLower;
+  }
+
+  /**
+   * Compute {@link CoreCount} of this fragment and populate it into 'fragmentCoreState'.
+   * @param fragmentCoreState A map holding per-fragment core state.
+   * All successor of this fragment must already have its CoreCount registered into this
+   * map.
+   */
+  protected void computeBlockingAwareCores(
+      Map<PlanFragmentId, Pair<CoreCount, List<CoreCount>>> fragmentCoreState) {
+    Preconditions.checkNotNull(
+        rootSegment_, "ProcessingCost Fragment %s has not been computed!", getId());
+    ImmutableList.Builder<CoreCount> subtreeCoreBuilder =
+        new ImmutableList.Builder<CoreCount>();
+    CoreCount coreReq =
+        rootSegment_.traverseBlockingAwareCores(fragmentCoreState, subtreeCoreBuilder);
+    fragmentCoreState.put(getId(), Pair.create(coreReq, subtreeCoreBuilder.build()));
+  }
+
+  protected CoreCount maxCore(CoreCount core1, CoreCount core2) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("At {}, compare {} ({}) vs {} ({})", getId(), core1, core1.total(), core2,
+          core2.total());
+    }
+    return CoreCount.max(core1, core2);
+  }
+
+  /**
+   * Override parallelism of this fragment with adjusted parallelism from CPU costing
+   * algorithm.
+   * <p>Only valid after {@link #traverseEffectiveParallelism(int, int, int)}
+   * called.
+   */
+  protected void setEffectiveNumInstance() {
+    validateProcessingCosts();
+    if (originalInstanceCount_ <= 0) {
+      originalInstanceCount_ = getNumInstances();
+    }
+
+    if (LOG.isTraceEnabled() && originalInstanceCount_ != getAdjustedInstanceCount()) {
+      logCountAdjustmentTrace(originalInstanceCount_, getAdjustedInstanceCount(),
+          "Finalize effective parallelism.");
+    }
+
+    for (PlanNode node : collectPlanNodes()) {
+      node.numInstances_ = getAdjustedInstanceCount();
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("ProcessingCost Fragment {}:\n{}", getId(), debugProcessingCosts());
+    }
+  }
+
+  private void logCountAdjustmentTrace(int oldCount, int newCount, String reason) {
+    LOG.trace("{} adjust instance count from {} to {}. {}", getId(), oldCount, newCount,
+        reason);
+  }
+
+  private static boolean isBlockingNode(PlanNode node) {
+    // Preaggregation node can behave like final aggregation node when it does not
+    // passedthrough any row. From CPU costing perspective, treat both final aggregation
+    // and preaggregation as a blocking node. Otherwise, follow PlanNode.isBlockingNode().
+    return node.isBlockingNode() || node instanceof AggregationNode;
+  }
 }
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 6dc9709c5..2bf5ea579 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -424,8 +424,8 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
       PlanFragment childFragment = children_.get(0).fragment_;
       if (fragment_ != childFragment && detailLevel == TExplainLevel.EXTENDED) {
         // we're crossing a fragment boundary - print the fragment header.
-        expBuilder.append(childFragment.getFragmentHeaderString(prefix, prefix,
-            queryOptions.getMt_dop()));
+        expBuilder.append(childFragment.getFragmentHeaderString(
+            prefix, prefix, queryOptions, detailLevel));
       }
       expBuilder.append(
           children_.get(0).getExplainString(prefix, prefix, queryOptions, detailLevel));
@@ -934,6 +934,11 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
         "Processing cost of PlanNode " + getDisplayLabel() + " is invalid!");
     processingCost_.setNumRowToConsume(getInputCardinality());
     processingCost_.setNumRowToProduce(getCardinality());
+    if (isLeafNode()
+        && (!fragment_.hasAdjustedInstanceCount()
+            || fragment_.getAdjustedInstanceCount() < getNumInstances())) {
+      fragment_.setFixedInstanceCount(getNumInstances());
+    }
   }
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
index 133aec186..8da761a32 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
@@ -226,4 +226,10 @@ public class PlanRootSink extends DataSink {
   public void collectExprs(List<Expr> exprs) {
     exprs.addAll(outputExprs_);
   }
+
+  @Override
+  public void computeRowConsumptionAndProductionToCost() {
+    super.computeRowConsumptionAndProductionToCost();
+    fragment_.setFixedInstanceCount(fragment_.getNumInstances());
+  }
 }
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 2e077fe31..a12b08824 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -20,6 +20,7 @@ package org.apache.impala.planner;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 import org.apache.impala.analysis.AnalysisContext;
@@ -38,7 +39,7 @@ import org.apache.impala.catalog.FeHBaseTable;
 import org.apache.impala.catalog.FeKuduTable;
 import org.apache.impala.catalog.FeTable;
 import org.apache.impala.common.ImpalaException;
-import org.apache.impala.common.NotImplementedException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.common.PrintUtils;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.service.BackendConfig;
@@ -59,6 +60,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import static org.apache.impala.analysis.ToSqlOptions.SHOW_IMPLICIT_CASTS;
 
@@ -212,7 +214,6 @@ public class Planner {
             List<String> mentionedColumns = insertStmt.getMentionedColumns();
             Preconditions.checkState(!mentionedColumns.isEmpty());
             List<ColumnLabel> targetColLabels = new ArrayList<>();
-            String tblFullName = targetTable.getFullName();
             for (String column: mentionedColumns) {
               targetColLabels.add(new ColumnLabel(column, targetTable.getTableName()));
             }
@@ -307,6 +308,15 @@ public class Planner {
         str.append(String.format("Dedicated Coordinator Resource Estimate: Memory=%s\n",
             PrintUtils.printBytesRoundedToMb(request.getDedicated_coord_mem_estimate())));
       }
+
+      TQueryOptions queryOptions =
+          request.getQuery_ctx().getClient_request().getQuery_options();
+      if (ProcessingCost.isComputeCost(queryOptions)
+          && explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+        str.append("Effective parallelism: ");
+        str.append(request.getCores_required());
+        str.append("\n");
+      }
       hasHeader = true;
     }
     // Warn if the planner is running in DEBUG mode.
@@ -398,6 +408,96 @@ public class Planner {
     return str.toString();
   }
 
+  /**
+   * Adjust effective parallelism of each plan fragment of query after considering
+   * processing cost rate and blocking operator.
+   * <p>
+   * Only valid after {@link PlanFragment#computeCostingSegment(TQueryOptions)} has
+   * been called for all plan fragments in the list.
+   */
+  private static void computeEffectiveParallelism(
+      List<PlanFragment> postOrderFragments, int minThreadPerNode, int maxThreadPerNode) {
+    for (PlanFragment fragment : postOrderFragments) {
+      if (!(fragment.getSink() instanceof JoinBuildSink)) {
+        // Only adjust parallelism of non-join build fragment.
+        // Join build fragment will be adjusted later by fragment hosting the join node.
+        fragment.traverseEffectiveParallelism(minThreadPerNode, maxThreadPerNode, -1);
+      }
+    }
+
+    for (PlanFragment fragment : postOrderFragments) {
+      fragment.setEffectiveNumInstance();
+    }
+  }
+
+  /**
+   * This method returns the effective CPU requirement of a query when considering
+   * processing cost rate and blocking operator.
+   * <p>
+   * Only valid after {@link #computeEffectiveParallelism(List, int, int)} has
+   * been called over the plan fragment list.
+   */
+  private static CoreCount computeBlockingAwareCores(
+      List<PlanFragment> postOrderFragments) {
+    // fragmentCoreState is a mapping between a fragment (through its PlanFragmentId) and
+    // its CoreCount. The first element of the pair is the CoreCount of subtree rooted at
+    // that fragment. The second element of the pair is the CoreCount of blocking-child
+    // subtrees under that fragment. The effective CoreCount of a fragment is derived from
+    // the pair through the following formula:
+    //   max(Pair.first, sum(Pair.second))
+    Map<PlanFragmentId, Pair<CoreCount, List<CoreCount>>> fragmentCoreState =
+        Maps.newHashMap();
+
+    for (PlanFragment fragment : postOrderFragments) {
+      fragment.computeBlockingAwareCores(fragmentCoreState);
+    }
+
+    PlanFragment root = postOrderFragments.get(postOrderFragments.size() - 1);
+    Pair<CoreCount, List<CoreCount>> rootCores = fragmentCoreState.get(root.getId());
+
+    return root.maxCore(rootCores.first, CoreCount.sum(rootCores.second));
+  }
+
+  /**
+   * Compute processing cost of each plan fragment in the query plan and adjust each
+   * fragment parallelism according to producer-consumer rate between them.
+   */
+  public static void computeProcessingCost(List<PlanFragment> planRoots,
+      TQueryExecRequest request, PlannerContext planCtx, int numCoresPerExecutor) {
+    TQueryOptions queryOptions = planCtx.getRootAnalyzer().getQueryOptions();
+
+    if (!ProcessingCost.isComputeCost(queryOptions)) {
+      request.setCores_required(-1);
+      return;
+    }
+
+    // TODO: remove dependence on MT_DOP in the future.
+    //   We still depend on MT_DOP here since many aspect of query planning is still
+    //   controlled through MT_DOP.
+    int mtDop = queryOptions.getMt_dop();
+    int numNode = planCtx.getRootAnalyzer().numExecutorsForPlanning();
+    int minThreads = queryOptions.getProcessing_cost_min_threads();
+    int maxThreads = Math.max(minThreads, Math.max(mtDop, numCoresPerExecutor));
+
+    PlanFragment rootFragment = planRoots.get(0);
+    List<PlanFragment> postOrderFragments = rootFragment.getNodesPostOrder();
+    for (PlanFragment fragment : postOrderFragments) {
+      fragment.computeCostingSegment(queryOptions);
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Computing effective parallelism. numNode=" + numNode + " mtDop=" + mtDop
+          + " numCoresPerExecutor=" + numCoresPerExecutor + " minThreads=" + minThreads
+          + " maxThreads=" + maxThreads);
+    }
+
+    computeEffectiveParallelism(postOrderFragments, minThreads, maxThreads);
+    CoreCount effectiveCores = computeBlockingAwareCores(postOrderFragments);
+    request.setCores_required(effectiveCores.total());
+
+    LOG.info("CoreCount=" + effectiveCores);
+  }
+
   /**
    * Computes the per-host resource profile for the given plans, i.e. the peak resources
    * consumed by all fragment instances belonging to the query per host. Sets the
diff --git a/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java b/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
index df695bddc..6093a28b3 100644
--- a/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
+++ b/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
@@ -20,6 +20,7 @@ package org.apache.impala.planner;
 import com.google.common.base.Preconditions;
 import com.google.common.math.LongMath;
 
+import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TQueryOptions;
 
 import java.math.RoundingMode;
@@ -96,8 +97,7 @@ public abstract class ProcessingCost implements Cloneable {
   }
 
   public static boolean isComputeCost(TQueryOptions queryOptions) {
-    // TODO: Replace with proper check in IMPALA-11604 part 2.
-    return false;
+    return queryOptions.getMt_dop() > 0 && queryOptions.isCompute_processing_cost();
   }
 
   /**
@@ -207,10 +207,8 @@ public abstract class ProcessingCost implements Cloneable {
   }
 
   private int getNumInstanceMax() {
-    // TODO: replace minProcessingCostPerThread with backend flag.
-    long minProcessingCostPerThread = 10000000L;
     long maxInstance = LongMath.divide(getTotalCost(),
-        minProcessingCostPerThread, RoundingMode.CEILING);
+        BackendConfig.INSTANCE.getMinProcessingPerThread(), RoundingMode.CEILING);
     if (maxInstance > 0) {
       return maxInstance < Integer.MAX_VALUE ? (int) maxInstance : Integer.MAX_VALUE;
     } else {
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index d3d06b880..f5daefa7e 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -380,4 +380,14 @@ public class BackendConfig {
   public TGeospatialLibrary getGeospatialLibrary() {
     return backendCfg_.geospatial_library;
   }
+
+  public double getQueryCpuCountDivisor() { return backendCfg_.query_cpu_count_divisor; }
+
+  public boolean isProcessingCostUseEqualExprWeight() {
+    return backendCfg_.processing_cost_use_equal_expr_weight;
+  }
+
+  public long getMinProcessingPerThread() {
+    return backendCfg_.min_processing_per_thread;
+  }
 }
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 40469880e..c9da2bdea 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -25,6 +25,7 @@ import com.google.common.base.Predicates;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import com.google.common.math.IntMath;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.Uninterruptibles;
 import java.io.IOException;
@@ -139,6 +140,7 @@ import org.apache.impala.hooks.QueryEventHookManager;
 import org.apache.impala.planner.HdfsScanNode;
 import org.apache.impala.planner.PlanFragment;
 import org.apache.impala.planner.Planner;
+import org.apache.impala.planner.ProcessingCost;
 import org.apache.impala.planner.ScanNode;
 import org.apache.impala.thrift.TAlterDbParams;
 import org.apache.impala.thrift.TBackendGflags;
@@ -267,6 +269,12 @@ public class Frontend {
       // TExecRequest.query_exec_request field.
       protected long estimated_memory_per_host_ = -1;
 
+      // The processing cores required to execute the query.
+      // Certain queries such as EXPLAIN that do not populate
+      // TExecRequest.query_exec_request. Therefore, cores requirement will be set here
+      // through setCoresRequired().
+      protected int cores_required_ = -1;
+
       // The initial length of content in explain buffer to help return the buffer
       // to the initial position prior to another auto-scaling compilation.
       protected int initialExplainBufLen_ = -1;
@@ -293,6 +301,9 @@ public class Frontend {
       public long getEstimatedMemoryPerHost() { return estimated_memory_per_host_; }
       public void setEstimatedMemoryPerHost(long x) { estimated_memory_per_host_ = x; }
 
+      public int getCoresRequired() { return cores_required_; }
+      public void setCoresRequired(int x) { cores_required_ = x; }
+
       // Capture the current state and initialize before iterative compilations begin.
       public void captureState() {
         disableAuthorization_ = false;
@@ -1738,6 +1749,8 @@ public class Frontend {
 
     // Compute resource requirements of the final plans.
     TQueryExecRequest result = new TQueryExecRequest();
+    Planner.computeProcessingCost(planRoots, result, planner.getPlannerCtx(),
+        planCtx.compilationState_.getGroupSet().getNum_cores_per_executor());
     Planner.computeResourceReqs(planRoots, queryCtx, result,
         planner.getPlannerCtx(), planner.getAnalysisResult().isQueryStmt());
 
@@ -1774,6 +1787,8 @@ public class Frontend {
     planCtx.compilationState_.setEstimatedMemoryPerHost(
         result.getPer_host_mem_estimate());
 
+    planCtx.compilationState_.setCoresRequired(result.getCores_required());
+
     return result;
   }
 
@@ -1900,14 +1915,18 @@ public class Frontend {
           + " does not map to any known executor group set.");
     }
 
-    // Sort 'executorGroupSets' by max_mem_limit field in ascending order. Use
-    // exec_group_name_prefix to break the tie.
+    // Sort 'executorGroupSets' by
+    //   <max_mem_limit, expected_num_executors * num_cores_per_executor>
+    // in ascending order. Use exec_group_name_prefix to break the tie.
     Collections.sort(result, new Comparator<TExecutorGroupSet>() {
       @Override
       public int compare(TExecutorGroupSet e1, TExecutorGroupSet e2) {
         int i = Long.compare(e1.getMax_mem_limit(), e2.getMax_mem_limit());
         if (i == 0) {
-          i = e1.getExec_group_name_prefix().compareTo(e2.getExec_group_name_prefix());
+          i = Long.compare(expectedTotalCores(e1), expectedTotalCores(e2));
+          if (i == 0) {
+            i = e1.getExec_group_name_prefix().compareTo(e2.getExec_group_name_prefix());
+          }
         }
         return i;
       }
@@ -1922,6 +1941,14 @@ public class Frontend {
     return type == TStmtType.EXPLAIN || type == TStmtType.QUERY || type == TStmtType.DML;
   }
 
+  private static int expectedTotalCores(TExecutorGroupSet execGroupSet) {
+    int numExecutors = execGroupSet.getCurr_num_executors() > 0 ?
+        execGroupSet.getCurr_num_executors() :
+        execGroupSet.getExpected_num_executors();
+    return IntMath.saturatedMultiply(
+        numExecutors, execGroupSet.getNum_cores_per_executor());
+  }
+
   private TExecRequest getTExecRequest(PlanCtx planCtx, EventSequence timeline)
       throws ImpalaException {
     TQueryCtx queryCtx = planCtx.getQueryContext();
@@ -1963,7 +1990,6 @@ public class Frontend {
     planCtx.compilationState_.captureState();
 
     TExecutorGroupSet group_set = null;
-    long per_host_mem_estimate = -1;
     String reason = "Unknown";
     int attempt = 0;
     for (int i = 0; i < num_executor_group_sets; i++) {
@@ -2008,20 +2034,34 @@ public class Frontend {
       }
 
       // Find out the per host memory estimated from two possible sources.
-      per_host_mem_estimate = -1;
+      long per_host_mem_estimate = -1;
+      int cores_requirement = -1;
       if (req.query_exec_request != null) {
         // For non-explain queries
         per_host_mem_estimate = req.query_exec_request.per_host_mem_estimate;
+        cores_requirement = req.query_exec_request.getCores_required();
       } else {
         // For explain queries
         per_host_mem_estimate = planCtx.compilationState_.getEstimatedMemoryPerHost();
+        cores_requirement = planCtx.compilationState_.getCoresRequired();
       }
 
       Preconditions.checkState(per_host_mem_estimate >= 0);
+      boolean cpuReqSatisfied = true;
+      int scaled_cores_requirement = -1;
+      if (ProcessingCost.isComputeCost(queryOptions)) {
+        Preconditions.checkState(cores_requirement > 0);
+        scaled_cores_requirement = (int) Math.min(Integer.MAX_VALUE,
+            Math.ceil(
+                cores_requirement / BackendConfig.INSTANCE.getQueryCpuCountDivisor()));
+        cpuReqSatisfied = scaled_cores_requirement <= expectedTotalCores(group_set);
+      }
 
-      if (per_host_mem_estimate <= group_set.getMax_mem_limit()) {
+      if (per_host_mem_estimate <= group_set.getMax_mem_limit() && cpuReqSatisfied) {
         reason = "suitable group found (estimated per-host memory="
-            + PrintUtils.printBytes(per_host_mem_estimate) + ")";
+            + PrintUtils.printBytes(per_host_mem_estimate)
+            + ", estimated cpu cores required=" + cores_requirement
+            + ", scaled cpu cores=" + scaled_cores_requirement + ")";
 
         // Set the group name prefix in both the returned query options and
         // the query context for non default group setup.
diff --git a/fe/src/main/java/org/apache/impala/util/ExprUtil.java b/fe/src/main/java/org/apache/impala/util/ExprUtil.java
index 2b34835da..6a7e2cedc 100644
--- a/fe/src/main/java/org/apache/impala/util/ExprUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/ExprUtil.java
@@ -111,12 +111,24 @@ public class ExprUtil {
     // TODO: Implement the cost for conjunts once the implemetation for
     // 'Expr' is in place.
     if (exprs == null) return 0;
-    return exprs.size();
+    if (BackendConfig.INSTANCE.isProcessingCostUseEqualExprWeight()) {
+      return exprs.size();
+    } else {
+      float totalCost = 0;
+      for (Expr e : exprs) {
+        totalCost += e.hasCost() ? e.getCost() : 1;
+      }
+      return totalCost;
+    }
   }
 
   public static float computeExprCost(Expr e) {
     if (e == null) return 0;
-    return 1;
+    if (BackendConfig.INSTANCE.isProcessingCostUseEqualExprWeight()) {
+      return 1;
+    } else {
+      return e.hasCost() ? e.getCost() : 1;
+    }
     // TODO Implement a function that can take into consideration of data types,
     // expressions and potentially LLVM translation in BE. The function must also
     // run fast.
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index bf48910ab..e86579a24 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -37,7 +37,6 @@ import org.apache.impala.testutil.TestUtils.IgnoreValueFilter;
 import org.apache.impala.thrift.TRuntimeFilterType;
 import org.apache.impala.thrift.TExecRequest;
 import org.apache.impala.thrift.TExplainLevel;
-import org.apache.impala.thrift.TExplainResult;
 import org.apache.impala.thrift.TJoinDistributionMode;
 import org.apache.impala.thrift.TKuduReplicaSelection;
 import org.apache.impala.thrift.TQueryCtx;
@@ -1358,4 +1357,22 @@ public class PlannerTest extends PlannerTestBase {
     runPlannerTestFile("explain-verbose-mt_dop", "tpcds_parquet",
         ImmutableSet.of(PlannerTestOption.INCLUDE_RESOURCE_HEADER));
   }
+
+  /**
+   * Test that processing cost can adjust effective instance count of fragment.
+   */
+  @Test
+  public void testProcessingCost() {
+    TQueryOptions options = new TQueryOptions();
+    options.setMt_dop(4);
+    options.setCompute_processing_cost(true);
+    options.setMinmax_filter_threshold(0.5);
+    options.setMinmax_filter_sorted_columns(false);
+    options.setMinmax_filter_partition_columns(false);
+    runPlannerTestFile("tpcds-processing-cost", "tpcds_parquet", options,
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN,
+            PlannerTestOption.INCLUDE_RESOURCE_HEADER,
+            PlannerTestOption.VALIDATE_RESOURCES,
+            PlannerTestOption.VALIDATE_CARDINALITY));
+  }
 }
diff --git a/fe/src/test/resources/fair-scheduler-2-groups.xml b/fe/src/test/resources/fair-scheduler-3-groups.xml
similarity index 88%
rename from fe/src/test/resources/fair-scheduler-2-groups.xml
rename to fe/src/test/resources/fair-scheduler-3-groups.xml
index 0472a31e9..81b71e33f 100644
--- a/fe/src/test/resources/fair-scheduler-2-groups.xml
+++ b/fe/src/test/resources/fair-scheduler-3-groups.xml
@@ -1,6 +1,8 @@
 <?xml version="1.0"?>
 <allocations>
   <queue name="root">
+    <queue name="tiny">
+    </queue>
     <queue name="small">
     </queue>
     <queue name="large">
diff --git a/fe/src/test/resources/llama-site-2-groups.xml b/fe/src/test/resources/llama-site-3-groups.xml
similarity index 70%
rename from fe/src/test/resources/llama-site-2-groups.xml
rename to fe/src/test/resources/llama-site-3-groups.xml
index f06b35a28..6fdd67697 100644
--- a/fe/src/test/resources/llama-site-2-groups.xml
+++ b/fe/src/test/resources/llama-site-3-groups.xml
@@ -1,6 +1,28 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <configuration>
   <!-- Default values -->
+  <property>
+    <name>impala.admission-control.pool-queue-timeout-ms.root.tiny</name>
+    <value>100000</value>
+  </property>
+  <property>
+    <name>impala.admission-control.max-query-mem-limit.root.tiny</name>
+   <!-- 64 MB -->
+    <value>67108864</value>
+  </property>
+  <property>
+    <name>impala.admission-control.min-query-mem-limit.root.tiny</name>
+   <!-- 0MB -->
+    <value>0</value>
+  </property>
+  <property>
+    <name>impala.admission-control.max-query-cpu-core-per-node-limit.root.tiny</name>
+    <value>2</value>
+  </property>
+  <property>
+    <name>impala.admission-control.max-query-cpu-core-coordinator-limit.root.tiny</name>
+    <value>2</value>
+  </property>
   <property>
     <name>llama.am.throttling.maximum.placed.reservations.root.small</name>
     <value>10</value>
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
new file mode 100644
index 000000000..d930c697f
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
@@ -0,0 +1,8739 @@
+# TPCDS-Q3
+select
+  dt.d_year,
+  item.i_brand_id brand_id,
+  item.i_brand brand,
+  sum(ss_ext_sales_price) sum_agg
+from
+  date_dim dt,
+  store_sales,
+  item
+where
+  dt.d_date_sk = store_sales.ss_sold_date_sk
+  and store_sales.ss_item_sk = item.i_item_sk
+  and item.i_manufact_id = 436
+  and dt.d_moy = 12
+group by
+  dt.d_year,
+  item.i_brand,
+  item.i_brand_id
+order by
+  dt.d_year,
+  sum_agg desc,
+  brand_id
+limit 100
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=26.38MB Threads=11
+Per-Host Resource Estimates: Memory=139MB
+F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[406]
+PLAN-ROOT SINK
+|  output exprs: dt.d_year, item.i_brand_id, item.i_brand, sum(ss_ext_sales_price)
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=400
+|
+12:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
+|  limit: 100
+|  mem-estimate=18.16KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=4 row-size=52B cardinality=100 cost=6
+|  in pipelines: 06(GETNEXT)
+|
+F04:PLAN FRAGMENT [HASH(dt.d_year,item.i_brand,item.i_brand_id)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=10.22MB mem-reservation=1.94MB thread-reservation=1
+max-parallelism=3 fragment-costs=[12331, 300, 6]
+06:TOP-N [LIMIT=100]
+|  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
+|  mem-estimate=5.10KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=4 row-size=52B cardinality=100 cost=300
+|  in pipelines: 06(GETNEXT), 11(OPEN)
+|
+11:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_ext_sales_price)
+|  group by: dt.d_year, item.i_brand, item.i_brand_id
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3 row-size=52B cardinality=3.04K cost=12164
+|  in pipelines: 11(GETNEXT), 01(OPEN)
+|
+10:EXCHANGE [HASH(dt.d_year,item.i_brand,item.i_brand_id)]
+|  mem-estimate=220.26KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=52B cardinality=3.04K cost=167
+|  in pipelines: 01(GETNEXT)
+|
+F03:PLAN FRAGMENT [HASH(store_sales.ss_sold_date_sk)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=11.52MB mem-reservation=2.00MB thread-reservation=1
+max-parallelism=3 fragment-costs=[15408, 167]
+05:AGGREGATE [STREAMING]
+|  output: sum(ss_ext_sales_price)
+|  group by: dt.d_year, item.i_brand, item.i_brand_id
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3 row-size=52B cardinality=3.04K cost=12164
+|  in pipelines: 01(GETNEXT)
+|
+04:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: store_sales.ss_sold_date_sk = dt.d_date_sk
+|  fk/pk conjuncts: store_sales.ss_sold_date_sk = dt.d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=1,2,0 row-size=72B cardinality=3.04K cost=3041
+|  in pipelines: 01(GETNEXT), 00(OPEN)
+|
+|--F06:PLAN FRAGMENT [HASH(store_sales.ss_sold_date_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=3.02MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[6183]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: dt.d_date_sk
+|  |  runtime filters: RF000[bloom] <- dt.d_date_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=6087
+|  |
+|  09:EXCHANGE [HASH(dt.d_date_sk)]
+|  |  mem-estimate=87.33KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0 row-size=12B cardinality=6.09K cost=96
+|  |  in pipelines: 00(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.19MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[74002]
+|  00:SCAN HDFS [tpcds_parquet.date_dim dt, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: dt.d_moy = CAST(12 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: dt.d_moy = CAST(12 AS INT)
+|     parquet dictionary predicates: dt.d_moy = CAST(12 AS INT)
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=0 row-size=12B cardinality=6.09K cost=73906
+|     in pipelines: 00(GETNEXT)
+|
+08:EXCHANGE [HASH(store_sales.ss_sold_date_sk)]
+|  mem-estimate=877.96KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=1,2 row-size=60B cardinality=3.04K cost=203
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=16.80MB mem-reservation=1.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[2925614]
+03:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=01
+|  hash predicates: store_sales.ss_item_sk = item.i_item_sk
+|  fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=1,2 row-size=60B cardinality=3.04K cost=2880404
+|  in pipelines: 01(GETNEXT), 02(OPEN)
+|
+|--F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[22]
+|  JOIN BUILD
+|  |  join-table-id=01 plan-id=02 cohort-id=01
+|  |  build expressions: item.i_item_sk
+|  |  runtime filters: RF002[bloom] <- item.i_item_sk, RF003[min_max] <- item.i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=19
+|  |
+|  07:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=44B cardinality=19 cost=3
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[18778]
+|  02:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     predicates: item.i_manufact_id = CAST(436 AS INT)
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     parquet statistics predicates: item.i_manufact_id = CAST(436 AS INT)
+|     parquet dictionary predicates: item.i_manufact_id = CAST(436 AS INT)
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=2 row-size=44B cardinality=19 cost=18777
+|     in pipelines: 02(GETNEXT)
+|
+01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF003[min_max] -> store_sales.ss_item_sk, RF000[bloom] -> store_sales.ss_sold_date_sk, RF002[bloom] -> store_sales.ss_item_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=1 row-size=16B cardinality=2.88M cost=45007
+   in pipelines: 01(GETNEXT)
+====
+# TPCDS-Q14a first of two
+with  cross_items as
+ (select i_item_sk ss_item_sk
+ from item,
+ (select iss.i_brand_id brand_id
+     ,iss.i_class_id class_id
+     ,iss.i_category_id category_id
+ from store_sales
+     ,item iss
+     ,date_dim d1
+ where ss_item_sk = iss.i_item_sk
+   and ss_sold_date_sk = d1.d_date_sk
+   and d1.d_year between 1999 AND 1999 + 2
+ intersect
+ select ics.i_brand_id
+     ,ics.i_class_id
+     ,ics.i_category_id
+ from catalog_sales
+     ,item ics
+     ,date_dim d2
+ where cs_item_sk = ics.i_item_sk
+   and cs_sold_date_sk = d2.d_date_sk
+   and d2.d_year between 1999 AND 1999 + 2
+ intersect
+ select iws.i_brand_id
+     ,iws.i_class_id
+     ,iws.i_category_id
+ from web_sales
+     ,item iws
+     ,date_dim d3
+ where ws_item_sk = iws.i_item_sk
+   and ws_sold_date_sk = d3.d_date_sk
+   and d3.d_year between 1999 AND 1999 + 2) t1
+ where i_brand_id = brand_id
+      and i_class_id = class_id
+      and i_category_id = category_id
+),
+ avg_sales as
+ (select avg(quantity*list_price) average_sales
+  from (select ss_quantity quantity
+             ,ss_list_price list_price
+       from store_sales
+           ,date_dim
+       where ss_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2
+       union all
+       select cs_quantity quantity
+             ,cs_list_price list_price
+       from catalog_sales
+           ,date_dim
+       where cs_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2
+       union all
+       select ws_quantity quantity
+             ,ws_list_price list_price
+       from web_sales
+           ,date_dim
+       where ws_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2) x)
+ select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales)
+ from(
+       select 'store' channel, i_brand_id,i_class_id
+             ,i_category_id,sum(ss_quantity*ss_list_price) sales
+             , count(*) number_sales
+       from store_sales
+           ,item
+           ,date_dim
+       where ss_item_sk in (select ss_item_sk from cross_items)
+         and ss_item_sk = i_item_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year = 1999+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales
+       from catalog_sales
+           ,item
+           ,date_dim
+       where cs_item_sk in (select ss_item_sk from cross_items)
+         and cs_item_sk = i_item_sk
+         and cs_sold_date_sk = d_date_sk
+         and d_year = 1999+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales
+       from web_sales
+           ,item
+           ,date_dim
+       where ws_item_sk in (select ss_item_sk from cross_items)
+         and ws_item_sk = i_item_sk
+         and ws_sold_date_sk = d_date_sk
+         and d_year = 1999+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales)
+ ) y
+ group by rollup (channel, i_brand_id,i_class_id,i_category_id)
+ order by channel,i_brand_id,i_class_id,i_category_id
+LIMIT 100
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=780.38MB Threads=141
+Per-Host Resource Estimates: Memory=2.81GB
+F80:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[606]
+PLAN-ROOT SINK
+|  output exprs: CASE valid_tid(104,105,106,107,108) WHEN 104 THEN channel WHEN 105 THEN channel WHEN 106 THEN channel WHEN 107 THEN channel WHEN 108 THEN NULL END, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_brand_id WHEN 105 THEN i_brand_id WHEN 106 THEN i_brand_id WHEN 107 THEN NULL WHEN 108 THEN NULL END, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_class_id WHEN 105 THEN i_class_id WHEN 106 THEN NULL WHEN 107 THEN NULL WHEN 108 THEN NULL END, CASE valid_tid(104,105,10 [...]
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=600
+|
+216:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: CASE valid_tid(104,105,106,107,108) WHEN 104 THEN channel WHEN 105 THEN channel WHEN 106 THEN channel WHEN 107 THEN channel WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_brand_id WHEN 105 THEN i_brand_id WHEN 106 THEN i_brand_id WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_class_id WHEN 105 THEN i_class_id WHEN 106 THEN NULL WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(10 [...]
+|  limit: 100
+|  mem-estimate=16.80KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=110 row-size=48B cardinality=100 cost=6
+|  in pipelines: 129(GETNEXT)
+|
+F79:PLAN FRAGMENT [HASH(CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(channel) WHEN 105 THEN murmur_hash(channel) WHEN 106 THEN murmur_hash(channel) WHEN 107 THEN murmur_hash(channel) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(i_brand_id) WHEN 105 THEN murmur_hash(i_brand_id) WHEN 106 THEN murmur_hash(i_brand_id) WHEN 107 THEN murmur_hash(NULL) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WH [...]
+Per-Instance Resources: mem-estimate=137.09MB mem-reservation=93.62MB thread-reservation=1
+max-parallelism=3 fragment-costs=[17011772, 3936100, 400, 6]
+129:TOP-N [LIMIT=100]
+|  order by: CASE valid_tid(104,105,106,107,108) WHEN 104 THEN channel WHEN 105 THEN channel WHEN 106 THEN channel WHEN 107 THEN channel WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_brand_id WHEN 105 THEN i_brand_id WHEN 106 THEN i_brand_id WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_class_id WHEN 105 THEN i_class_id WHEN 106 THEN NULL WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(10 [...]
+|  mem-estimate=4.69KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=110 row-size=48B cardinality=100 cost=400
+|  in pipelines: 129(GETNEXT), 128(OPEN)
+|
+128:AGGREGATE [FINALIZE]
+|  output: aggif(valid_tid(104,105,106,107,108) IN (CAST(104 AS INT), CAST(105 AS INT), CAST(106 AS INT), CAST(107 AS INT), CAST(108 AS INT)), CASE valid_tid(104,105,106,107,108) WHEN CAST(104 AS INT) THEN sum(sales) WHEN CAST(105 AS INT) THEN sum(sales) WHEN CAST(106 AS INT) THEN sum(sales) WHEN CAST(107 AS INT) THEN sum(sales) WHEN CAST(108 AS INT) THEN sum(sales) END), aggif(valid_tid(104,105,106,107,108) IN (CAST(104 AS INT), CAST(105 AS INT), CAST(106 AS INT), CAST(107 AS INT), CAST [...]
+|  group by: CASE valid_tid(104,105,106,107,108) WHEN CAST(104 AS INT) THEN channel WHEN CAST(105 AS INT) THEN channel WHEN CAST(106 AS INT) THEN channel WHEN CAST(107 AS INT) THEN channel WHEN CAST(108 AS INT) THEN NULL END, CASE valid_tid(104,105,106,107,108) WHEN CAST(104 AS INT) THEN i_brand_id WHEN CAST(105 AS INT) THEN i_brand_id WHEN CAST(106 AS INT) THEN i_brand_id WHEN CAST(107 AS INT) THEN NULL WHEN CAST(108 AS INT) THEN NULL END, CASE valid_tid(104,105,106,107,108) WHEN CAST(1 [...]
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=109 row-size=52B cardinality=562.30K cost=3936100
+|  in pipelines: 128(GETNEXT), 215(OPEN)
+|
+215:AGGREGATE [FINALIZE]
+|  Class 0
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: channel, i_brand_id, i_class_id, i_category_id
+|  Class 1
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: channel, i_brand_id, i_class_id, NULL
+|  Class 2
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: channel, i_brand_id, NULL, NULL
+|  Class 3
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: channel, NULL, NULL, NULL
+|  Class 4
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: NULL, NULL, NULL, NULL
+|  mem-estimate=120.09MB mem-reservation=76.62MB thread-reservation=0
+|  tuple-ids=104N,105N,106N,107N,108N row-size=240B cardinality=562.30K cost=16869000
+|  in pipelines: 215(GETNEXT), 151(OPEN), 179(OPEN), 207(OPEN)
+|
+214:EXCHANGE [HASH(CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(channel) WHEN 105 THEN murmur_hash(channel) WHEN 106 THEN murmur_hash(channel) WHEN 107 THEN murmur_hash(channel) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(i_brand_id) WHEN 105 THEN murmur_hash(i_brand_id) WHEN 106 THEN murmur_hash(i_brand_id) WHEN 107 THEN murmur_hash(NULL) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 10 [...]
+|  mem-estimate=13.05MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=104N,105N,106N,107N,108N row-size=240B cardinality=562.30K cost=142772
+|  in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT)
+|
+F78:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Instance Resources: mem-estimate=65.22MB mem-reservation=28.94MB thread-reservation=1
+max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
+127:AGGREGATE [STREAMING]
+|  Class 0
+|    output: sum(sales), sum(number_sales)
+|    group by: channel, i_brand_id, i_class_id, i_category_id
+|  Class 1
+|    output: sum(sales), sum(number_sales)
+|    group by: channel, i_brand_id, i_class_id, NULL
+|  Class 2
+|    output: sum(sales), sum(number_sales)
+|    group by: channel, i_brand_id, NULL, NULL
+|  Class 3
+|    output: sum(sales), sum(number_sales)
+|    group by: channel, NULL, NULL, NULL
+|  Class 4
+|    output: sum(sales), sum(number_sales)
+|    group by: NULL, NULL, NULL, NULL
+|  mem-estimate=50.00MB mem-reservation=27.00MB thread-reservation=0
+|  tuple-ids=104N,105N,106N,107N,108N row-size=240B cardinality=562.30K cost=8308890
+|  in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT)
+|
+00:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=102 row-size=48B cardinality=276.96K cost=4985334
+|  in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT)
+|
+|--126:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  |  join table id: 08
+|  |  predicates: sum(ws_quantity * ws_list_price) > avg(quantity * list_price)
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=88,99 row-size=52B cardinality=42.85K cost=0
+|  |  in pipelines: 207(GETNEXT), 212(OPEN)
+|  |
+|  |--F89:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  JOIN BUILD
+|  |  |  join-table-id=08 plan-id=09 cohort-id=01
+|  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |
+|  |  213:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=99 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 212(GETNEXT)
+|  |  |
+|  |  F77:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  212:AGGREGATE [FINALIZE]
+|  |  |  output: avg:merge(quantity * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=99 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 212(GETNEXT), 125(OPEN)
+|  |  |
+|  |  211:EXCHANGE [UNPARTITIONED]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=98 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 125(GETNEXT)
+|  |  |
+|  |  F76:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[40389768, 1]
+|  |  125:AGGREGATE
+|  |  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=98 row-size=16B cardinality=1 cost=5041336
+|  |  |  in pipelines: 125(GETNEXT), 116(OPEN), 119(OPEN), 122(OPEN)
+|  |  |
+|  |  115:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=96 row-size=8B cardinality=5.04M cost=30248016
+|  |  |  in pipelines: 116(GETNEXT), 119(GETNEXT), 122(GETNEXT)
+|  |  |
+|  |  |--124:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=11
+|  |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=94,95 row-size=20B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 122(GETNEXT), 123(OPEN)
+|  |  |  |
+|  |  |  |--F92:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=11 plan-id=12 cohort-id=04
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF107[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  210:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=95 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 123(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F75:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  123:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=95 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 123(GETNEXT)
+|  |  |  |
+|  |  |  122:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF107[min_max] -> ws_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=94 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 122(GETNEXT)
+|  |  |
+|  |  |--121:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=10
+|  |  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=92,93 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 119(GETNEXT), 120(OPEN)
+|  |  |  |
+|  |  |  |--F91:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=04
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF105[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  209:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=93 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 120(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F73:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  120:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=93 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 120(GETNEXT)
+|  |  |  |
+|  |  |  119:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF105[min_max] -> cs_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=92 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 119(GETNEXT)
+|  |  |
+|  |  118:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=09
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=90,91 row-size=20B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 116(GETNEXT), 117(OPEN)
+|  |  |
+|  |  |--F90:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=09 plan-id=10 cohort-id=04
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  208:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=91 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 117(GETNEXT)
+|  |  |  |
+|  |  |  F71:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  117:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=91 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 117(GETNEXT)
+|  |  |
+|  |  116:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=90 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 116(GETNEXT)
+|  |
+|  207:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ws_quantity * ws_list_price), count:merge(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=88 row-size=36B cardinality=42.85K cost=214270
+|  |  in pipelines: 207(GETNEXT), 85(OPEN)
+|  |
+|  206:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  mem-estimate=833.29KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=88 row-size=36B cardinality=42.85K cost=1674
+|  |  in pipelines: 85(GETNEXT)
+|  |
+|  F52:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=43.88MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=2 fragment-costs=[1033413, 1674]
+|  114:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price), count(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=88 row-size=36B cardinality=42.85K cost=214270
+|  |  in pipelines: 85(GETNEXT)
+|  |
+|  113:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=36
+|  |  hash predicates: ws_item_sk = tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=68,70,69 row-size=52B cardinality=42.85K cost=42854
+|  |  in pipelines: 85(GETNEXT), 204(OPEN)
+|  |
+|  |--F117:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[18397]
+|  |  JOIN BUILD
+|  |  |  join-table-id=36 plan-id=37 cohort-id=01
+|  |  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  |  runtime filters: RF073[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |
+|  |  205:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=170 row-size=8B cardinality=17.98K cost=422
+|  |  |  in pipelines: 204(GETNEXT)
+|  |  |
+|  |  F68:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  204:AGGREGATE [FINALIZE]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=170 row-size=8B cardinality=17.98K cost=17975
+|  |  |  in pipelines: 204(GETNEXT), 191(OPEN)
+|  |  |
+|  |  203:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=170 row-size=8B cardinality=17.98K cost=211
+|  |  |  in pipelines: 191(GETNEXT)
+|  |  |
+|  |  F58:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  |  112:AGGREGATE [STREAMING]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=170 row-size=8B cardinality=17.98K cost=148800
+|  |  |  in pipelines: 191(GETNEXT)
+|  |  |
+|  |  109:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=37
+|  |  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=75,71 row-size=32B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 191(GETNEXT), 88(OPEN)
+|  |  |
+|  |  |--F118:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=2.30MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=37 plan-id=38 cohort-id=11
+|  |  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  |  runtime filters: RF081[min_max] <- i_brand_id, RF082[min_max] <- i_category_id, RF083[min_max] <- i_class_id
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |  |
+|  |  |  202:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=71 row-size=20B cardinality=18.00K cost=422
+|  |  |  |  in pipelines: 88(GETNEXT)
+|  |  |  |
+|  |  |  F67:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  88:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=71 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 88(GETNEXT)
+|  |  |
+|  |  108:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=38
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 191(GETNEXT), 200(OPEN)
+|  |  |
+|  |  |--F119:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=38 plan-id=39 cohort-id=11
+|  |  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  201:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=169 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 200(GETNEXT)
+|  |  |  |
+|  |  |  F66:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  |  200:AGGREGATE [FINALIZE]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=169 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 200(GETNEXT), 100(OPEN)
+|  |  |  |
+|  |  |  199:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=169 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 100(GETNEXT)
+|  |  |  |
+|  |  |  F63:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  |  107:AGGREGATE [STREAMING]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=169 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  |  in pipelines: 100(GETNEXT)
+|  |  |  |
+|  |  |  104:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=39
+|  |  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=81,82,83 row-size=40B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 100(GETNEXT), 102(OPEN)
+|  |  |  |
+|  |  |  |--F120:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=39 plan-id=40 cohort-id=12
+|  |  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  |  runtime filters: RF099[min_max] <- d3.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  198:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=83 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  |  in pipelines: 102(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F65:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  102:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=83 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 102(GETNEXT)
+|  |  |  |
+|  |  |  103:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=40
+|  |  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=81,82 row-size=32B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 100(GETNEXT), 101(OPEN)
+|  |  |  |
+|  |  |  |--F121:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=40 plan-id=41 cohort-id=12
+|  |  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  |  runtime filters: RF101[min_max] <- iws.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  197:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=82 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  |  in pipelines: 101(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F64:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  101:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=82 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 101(GETNEXT)
+|  |  |  |
+|  |  |  100:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF099[min_max] -> ws_sold_date_sk, RF101[min_max] -> ws_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=81 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 100(GETNEXT)
+|  |  |
+|  |  106:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=41
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 191(GETNEXT), 195(OPEN)
+|  |  |
+|  |  |--F122:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=41 plan-id=42 cohort-id=11
+|  |  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  196:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=168 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 195(GETNEXT)
+|  |  |  |
+|  |  |  F62:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  |  195:AGGREGATE [FINALIZE]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=168 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 195(GETNEXT), 95(OPEN)
+|  |  |  |
+|  |  |  194:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=168 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 95(GETNEXT)
+|  |  |  |
+|  |  |  F59:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  |  105:AGGREGATE [STREAMING]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=168 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  |  in pipelines: 95(GETNEXT)
+|  |  |  |
+|  |  |  99:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=42
+|  |  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=77,78,79 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 95(GETNEXT), 97(OPEN)
+|  |  |  |
+|  |  |  |--F123:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=42 plan-id=43 cohort-id=13
+|  |  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  |  runtime filters: RF095[min_max] <- d2.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  193:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=79 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 97(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F61:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  97:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=79 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 97(GETNEXT)
+|  |  |  |
+|  |  |  98:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=43
+|  |  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=77,78 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 95(GETNEXT), 96(OPEN)
+|  |  |  |
+|  |  |  |--F124:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=43 plan-id=44 cohort-id=13
+|  |  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  |  runtime filters: RF097[min_max] <- ics.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  192:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=78 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  |  in pipelines: 96(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F60:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  96:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=78 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 96(GETNEXT)
+|  |  |  |
+|  |  |  95:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF095[min_max] -> cs_sold_date_sk, RF097[min_max] -> cs_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=77 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 95(GETNEXT)
+|  |  |
+|  |  191:AGGREGATE [FINALIZE]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 191(GETNEXT), 89(OPEN)
+|  |  |
+|  |  190:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 89(GETNEXT)
+|  |  |
+|  |  F55:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  |  94:AGGREGATE [STREAMING]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=8641212
+|  |  |  in pipelines: 89(GETNEXT)
+|  |  |
+|  |  93:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=44
+|  |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=72,73,74 row-size=40B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 89(GETNEXT), 91(OPEN)
+|  |  |
+|  |  |--F125:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=44 plan-id=45 cohort-id=11
+|  |  |  |  build expressions: d1.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  189:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=74 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 91(GETNEXT)
+|  |  |  |
+|  |  |  F57:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  91:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=74 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 91(GETNEXT)
+|  |  |
+|  |  92:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=45
+|  |  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=72,73 row-size=32B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 89(GETNEXT), 90(OPEN)
+|  |  |
+|  |  |--F126:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=45 plan-id=46 cohort-id=11
+|  |  |  |  build expressions: iss.i_item_sk
+|  |  |  |  runtime filters: RF093[min_max] <- iss.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  188:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=73 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 90(GETNEXT)
+|  |  |  |
+|  |  |  F56:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  90:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     runtime filters: RF081[min_max] -> iss.i_brand_id, RF082[min_max] -> iss.i_category_id, RF083[min_max] -> iss.i_class_id
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=73 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 90(GETNEXT)
+|  |  |
+|  |  89:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF093[min_max] -> ss_item_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=72 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 89(GETNEXT)
+|  |
+|  111:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=46
+|  |  hash predicates: ws_item_sk = i_item_sk
+|  |  fk/pk conjuncts: ws_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=68,70,69 row-size=52B cardinality=42.85K cost=42854
+|  |  in pipelines: 85(GETNEXT), 86(OPEN)
+|  |
+|  |--F127:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  JOIN BUILD
+|  |  |  join-table-id=46 plan-id=47 cohort-id=01
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF075[min_max] <- i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  187:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=69 row-size=20B cardinality=18.00K cost=844
+|  |  |  in pipelines: 86(GETNEXT)
+|  |  |
+|  |  F54:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  86:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF073[min_max] -> tpcds_parquet.item.i_item_sk
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=69 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 86(GETNEXT)
+|  |
+|  110:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=47
+|  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=68,70 row-size=32B cardinality=42.85K cost=719384
+|  |  in pipelines: 85(GETNEXT), 87(OPEN)
+|  |
+|  |--F128:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=2 fragment-costs=[112]
+|  |  JOIN BUILD
+|  |  |  join-table-id=47 plan-id=48 cohort-id=01
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF076[bloom] <- d_date_sk, RF077[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  186:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=70 row-size=12B cardinality=108 cost=4
+|  |  |  in pipelines: 87(GETNEXT)
+|  |  |
+|  |  F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  87:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=70 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 87(GETNEXT)
+|  |
+|  85:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|     HDFS partitions=1/1 files=2 size=45.09MB
+|     runtime filters: RF077[min_max] -> ws_sold_date_sk, RF073[min_max] -> ws_item_sk, RF075[min_max] -> ws_item_sk, RF076[bloom] -> ws_sold_date_sk
+|     stored statistics:
+|       table: rows=719.38K size=45.09MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|     tuple-ids=68 row-size=20B cardinality=719.38K cost=14051
+|     in pipelines: 85(GETNEXT)
+|
+|--84:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  |  join table id: 04
+|  |  predicates: sum(cs_quantity * cs_list_price) > avg(quantity * list_price)
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=54,65 row-size=52B cardinality=85.31K cost=0
+|  |  in pipelines: 179(GETNEXT), 184(OPEN)
+|  |
+|  |--F85:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=01
+|  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |
+|  |  185:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=65 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 184(GETNEXT)
+|  |  |
+|  |  F51:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  184:AGGREGATE [FINALIZE]
+|  |  |  output: avg:merge(quantity * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=65 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 184(GETNEXT), 83(OPEN)
+|  |  |
+|  |  183:EXCHANGE [UNPARTITIONED]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=64 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 83(GETNEXT)
+|  |  |
+|  |  F50:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[40389768, 1]
+|  |  83:AGGREGATE
+|  |  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=64 row-size=16B cardinality=1 cost=5041336
+|  |  |  in pipelines: 83(GETNEXT), 74(OPEN), 77(OPEN), 80(OPEN)
+|  |  |
+|  |  73:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=62 row-size=8B cardinality=5.04M cost=30248016
+|  |  |  in pipelines: 74(GETNEXT), 77(GETNEXT), 80(GETNEXT)
+|  |  |
+|  |  |--82:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=07
+|  |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=60,61 row-size=20B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 80(GETNEXT), 81(OPEN)
+|  |  |  |
+|  |  |  |--F88:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=07 plan-id=08 cohort-id=03
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF071[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  182:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=61 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 81(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F49:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  81:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=61 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 81(GETNEXT)
+|  |  |  |
+|  |  |  80:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF071[min_max] -> ws_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=60 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 80(GETNEXT)
+|  |  |
+|  |  |--79:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=06
+|  |  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=58,59 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 77(GETNEXT), 78(OPEN)
+|  |  |  |
+|  |  |  |--F87:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=06 plan-id=07 cohort-id=03
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF069[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  181:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=59 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 78(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F47:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  78:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=59 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 78(GETNEXT)
+|  |  |  |
+|  |  |  77:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF069[min_max] -> cs_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=58 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 77(GETNEXT)
+|  |  |
+|  |  76:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=05
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=56,57 row-size=20B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 74(GETNEXT), 75(OPEN)
+|  |  |
+|  |  |--F86:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=05 plan-id=06 cohort-id=03
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  180:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=57 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 75(GETNEXT)
+|  |  |  |
+|  |  |  F45:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  75:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=57 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 75(GETNEXT)
+|  |  |
+|  |  74:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=56 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 74(GETNEXT)
+|  |
+|  179:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(cs_quantity * cs_list_price), count:merge(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=54 row-size=36B cardinality=85.31K cost=426545
+|  |  in pipelines: 179(GETNEXT), 43(OPEN)
+|  |
+|  178:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  mem-estimate=1.09MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=54 row-size=36B cardinality=85.31K cost=3333
+|  |  in pipelines: 43(GETNEXT)
+|  |
+|  F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=59.88MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[2066867, 3333]
+|  72:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price), count(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=54 row-size=36B cardinality=85.31K cost=426545
+|  |  in pipelines: 43(GETNEXT)
+|  |
+|  71:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=24
+|  |  hash predicates: cs_item_sk = tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=34,36,35 row-size=52B cardinality=85.31K cost=85309
+|  |  in pipelines: 43(GETNEXT), 176(OPEN)
+|  |
+|  |--F105:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[18608]
+|  |  JOIN BUILD
+|  |  |  join-table-id=24 plan-id=25 cohort-id=01
+|  |  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  |  runtime filters: RF037[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |
+|  |  177:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=148 row-size=8B cardinality=17.98K cost=633
+|  |  |  in pipelines: 176(GETNEXT)
+|  |  |
+|  |  F42:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  176:AGGREGATE [FINALIZE]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=148 row-size=8B cardinality=17.98K cost=17975
+|  |  |  in pipelines: 176(GETNEXT), 163(OPEN)
+|  |  |
+|  |  175:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=148 row-size=8B cardinality=17.98K cost=211
+|  |  |  in pipelines: 163(GETNEXT)
+|  |  |
+|  |  F32:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  |  70:AGGREGATE [STREAMING]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=148 row-size=8B cardinality=17.98K cost=148800
+|  |  |  in pipelines: 163(GETNEXT)
+|  |  |
+|  |  67:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=25
+|  |  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=41,37 row-size=32B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 163(GETNEXT), 46(OPEN)
+|  |  |
+|  |  |--F106:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=2.30MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=25 plan-id=26 cohort-id=08
+|  |  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  |  runtime filters: RF045[min_max] <- i_brand_id, RF046[min_max] <- i_category_id, RF047[min_max] <- i_class_id
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |  |
+|  |  |  174:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=37 row-size=20B cardinality=18.00K cost=422
+|  |  |  |  in pipelines: 46(GETNEXT)
+|  |  |  |
+|  |  |  F41:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  46:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=37 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 46(GETNEXT)
+|  |  |
+|  |  66:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=26
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 163(GETNEXT), 172(OPEN)
+|  |  |
+|  |  |--F107:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=26 plan-id=27 cohort-id=08
+|  |  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  173:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=147 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 172(GETNEXT)
+|  |  |  |
+|  |  |  F40:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  |  172:AGGREGATE [FINALIZE]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=147 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 172(GETNEXT), 58(OPEN)
+|  |  |  |
+|  |  |  171:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=147 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 58(GETNEXT)
+|  |  |  |
+|  |  |  F37:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  |  65:AGGREGATE [STREAMING]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=147 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  |  in pipelines: 58(GETNEXT)
+|  |  |  |
+|  |  |  62:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=27
+|  |  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=47,48,49 row-size=40B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 58(GETNEXT), 60(OPEN)
+|  |  |  |
+|  |  |  |--F108:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=27 plan-id=28 cohort-id=09
+|  |  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  |  runtime filters: RF063[min_max] <- d3.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  170:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=49 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  |  in pipelines: 60(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F39:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  60:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=49 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 60(GETNEXT)
+|  |  |  |
+|  |  |  61:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=28
+|  |  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=47,48 row-size=32B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 58(GETNEXT), 59(OPEN)
+|  |  |  |
+|  |  |  |--F109:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=28 plan-id=29 cohort-id=09
+|  |  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  |  runtime filters: RF065[min_max] <- iws.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  169:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=48 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  |  in pipelines: 59(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F38:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  59:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=48 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 59(GETNEXT)
+|  |  |  |
+|  |  |  58:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF063[min_max] -> ws_sold_date_sk, RF065[min_max] -> ws_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=47 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 58(GETNEXT)
+|  |  |
+|  |  64:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=29
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 163(GETNEXT), 167(OPEN)
+|  |  |
+|  |  |--F110:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=29 plan-id=30 cohort-id=08
+|  |  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  168:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=146 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 167(GETNEXT)
+|  |  |  |
+|  |  |  F36:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  |  167:AGGREGATE [FINALIZE]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=146 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 167(GETNEXT), 53(OPEN)
+|  |  |  |
+|  |  |  166:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=146 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 53(GETNEXT)
+|  |  |  |
+|  |  |  F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  |  63:AGGREGATE [STREAMING]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=146 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  |  in pipelines: 53(GETNEXT)
+|  |  |  |
+|  |  |  57:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=30
+|  |  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=43,44,45 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 53(GETNEXT), 55(OPEN)
+|  |  |  |
+|  |  |  |--F111:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=30 plan-id=31 cohort-id=10
+|  |  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  |  runtime filters: RF059[min_max] <- d2.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  165:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=45 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 55(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F35:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  55:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=45 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 55(GETNEXT)
+|  |  |  |
+|  |  |  56:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=31
+|  |  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=43,44 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 53(GETNEXT), 54(OPEN)
+|  |  |  |
+|  |  |  |--F112:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=31 plan-id=32 cohort-id=10
+|  |  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  |  runtime filters: RF061[min_max] <- ics.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  164:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=44 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  |  in pipelines: 54(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F34:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  54:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=44 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 54(GETNEXT)
+|  |  |  |
+|  |  |  53:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF059[min_max] -> cs_sold_date_sk, RF061[min_max] -> cs_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=43 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 53(GETNEXT)
+|  |  |
+|  |  163:AGGREGATE [FINALIZE]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 163(GETNEXT), 47(OPEN)
+|  |  |
+|  |  162:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 47(GETNEXT)
+|  |  |
+|  |  F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  |  52:AGGREGATE [STREAMING]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=8641212
+|  |  |  in pipelines: 47(GETNEXT)
+|  |  |
+|  |  51:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=32
+|  |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=38,39,40 row-size=40B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 47(GETNEXT), 49(OPEN)
+|  |  |
+|  |  |--F113:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=32 plan-id=33 cohort-id=08
+|  |  |  |  build expressions: d1.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  161:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=40 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 49(GETNEXT)
+|  |  |  |
+|  |  |  F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  49:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=40 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 49(GETNEXT)
+|  |  |
+|  |  50:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=33
+|  |  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=38,39 row-size=32B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 47(GETNEXT), 48(OPEN)
+|  |  |
+|  |  |--F114:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=33 plan-id=34 cohort-id=08
+|  |  |  |  build expressions: iss.i_item_sk
+|  |  |  |  runtime filters: RF057[min_max] <- iss.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  160:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 48(GETNEXT)
+|  |  |  |
+|  |  |  F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  48:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     runtime filters: RF045[min_max] -> iss.i_brand_id, RF046[min_max] -> iss.i_category_id, RF047[min_max] -> iss.i_class_id
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=39 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 48(GETNEXT)
+|  |  |
+|  |  47:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF057[min_max] -> ss_item_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=38 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 47(GETNEXT)
+|  |
+|  69:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=34
+|  |  hash predicates: cs_item_sk = i_item_sk
+|  |  fk/pk conjuncts: cs_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=34,36,35 row-size=52B cardinality=85.31K cost=85309
+|  |  in pipelines: 43(GETNEXT), 44(OPEN)
+|  |
+|  |--F115:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  JOIN BUILD
+|  |  |  join-table-id=34 plan-id=35 cohort-id=01
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF039[min_max] <- i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  159:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=35 row-size=20B cardinality=18.00K cost=1266
+|  |  |  in pipelines: 44(GETNEXT)
+|  |  |
+|  |  F28:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  44:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF037[min_max] -> tpcds_parquet.item.i_item_sk
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=35 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 44(GETNEXT)
+|  |
+|  68:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=35
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=34,36 row-size=32B cardinality=85.31K cost=1441548
+|  |  in pipelines: 43(GETNEXT), 45(OPEN)
+|  |
+|  |--F116:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=35 plan-id=36 cohort-id=01
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF040[bloom] <- d_date_sk, RF041[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  158:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=36 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 45(GETNEXT)
+|  |  |
+|  |  F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  45:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=36 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 45(GETNEXT)
+|  |
+|  43:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     runtime filters: RF041[min_max] -> cs_sold_date_sk, RF037[min_max] -> cs_item_sk, RF039[min_max] -> cs_item_sk, RF040[bloom] -> cs_sold_date_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|     tuple-ids=34 row-size=20B cardinality=1.44M cost=28156
+|     in pipelines: 43(GETNEXT)
+|
+42:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  join table id: 00
+|  predicates: sum(ss_quantity * ss_list_price) > avg(quantity * list_price)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=20,31 row-size=52B cardinality=148.80K cost=0
+|  in pipelines: 151(GETNEXT), 156(OPEN)
+|
+|--F81:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[3]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |
+|  157:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=31 row-size=16B cardinality=1 cost=3
+|  |  in pipelines: 156(GETNEXT)
+|  |
+|  F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[2, 1]
+|  156:AGGREGATE [FINALIZE]
+|  |  output: avg:merge(quantity * list_price)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=31 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 156(GETNEXT), 41(OPEN)
+|  |
+|  155:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=30 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 41(GETNEXT)
+|  |
+|  F24:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[40389768, 1]
+|  41:AGGREGATE
+|  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=30 row-size=16B cardinality=1 cost=5041336
+|  |  in pipelines: 41(GETNEXT), 32(OPEN), 35(OPEN), 38(OPEN)
+|  |
+|  31:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=28 row-size=8B cardinality=5.04M cost=30248016
+|  |  in pipelines: 32(GETNEXT), 35(GETNEXT), 38(GETNEXT)
+|  |
+|  |--40:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=03
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=26,27 row-size=20B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 38(GETNEXT), 39(OPEN)
+|  |  |
+|  |  |--F84:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=03 plan-id=04 cohort-id=02
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF035[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  154:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 39(GETNEXT)
+|  |  |  |
+|  |  |  F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  39:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=27 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 39(GETNEXT)
+|  |  |
+|  |  38:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF035[min_max] -> ws_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=26 row-size=12B cardinality=719.38K cost=8431
+|  |     in pipelines: 38(GETNEXT)
+|  |
+|  |--37:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=24,25 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 35(GETNEXT), 36(OPEN)
+|  |  |
+|  |  |--F83:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=02
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF033[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  153:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=25 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 36(GETNEXT)
+|  |  |  |
+|  |  |  F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  36:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=25 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 36(GETNEXT)
+|  |  |
+|  |  35:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF033[min_max] -> cs_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=24 row-size=12B cardinality=1.44M cost=16894
+|  |     in pipelines: 35(GETNEXT)
+|  |
+|  34:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=01
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=22,23 row-size=20B cardinality=2.88M cost=2880404
+|  |  in pipelines: 32(GETNEXT), 33(OPEN)
+|  |
+|  |--F82:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |
+|  |  152:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=23 row-size=8B cardinality=7.30K cost=258
+|  |  |  in pipelines: 33(GETNEXT)
+|  |  |
+|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146755]
+|  |  33:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=23 row-size=8B cardinality=7.30K cost=146669
+|  |     in pipelines: 33(GETNEXT)
+|  |
+|  32:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=22 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 32(GETNEXT)
+|
+151:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_quantity * ss_list_price), count:merge(*)
+|  group by: i_brand_id, i_class_id, i_category_id
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=20 row-size=36B cardinality=148.80K cost=744000
+|  in pipelines: 151(GETNEXT), 01(OPEN)
+|
+150:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  mem-estimate=2.17MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=20 row-size=36B cardinality=148.80K cost=5813
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+Per-Instance Resources: mem-estimate=27.88MB mem-reservation=3.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[4130519, 5813]
+30:AGGREGATE [STREAMING]
+|  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*)
+|  group by: i_brand_id, i_class_id, i_category_id
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=20 row-size=36B cardinality=148.80K cost=852755
+|  in pipelines: 01(GETNEXT)
+|
+29:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash-table-id=12
+|  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,2,1 row-size=52B cardinality=170.55K cost=170551
+|  in pipelines: 01(GETNEXT), 148(OPEN)
+|
+|--F93:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.92MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[18608]
+|  JOIN BUILD
+|  |  join-table-id=12 plan-id=13 cohort-id=01
+|  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  runtime filters: RF000[bloom] <- tpcds_parquet.item.i_item_sk, RF001[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |
+|  149:EXCHANGE [BROADCAST]
+|  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=126 row-size=8B cardinality=17.98K cost=633
+|  |  in pipelines: 148(GETNEXT)
+|  |
+|  F16:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[18186, 211]
+|  148:AGGREGATE [FINALIZE]
+|  |  group by: tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=126 row-size=8B cardinality=17.98K cost=17975
+|  |  in pipelines: 148(GETNEXT), 135(OPEN)
+|  |
+|  147:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=126 row-size=8B cardinality=17.98K cost=211
+|  |  in pipelines: 135(GETNEXT)
+|  |
+|  F06:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  28:AGGREGATE [STREAMING]
+|  |  group by: tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=126 row-size=8B cardinality=17.98K cost=148800
+|  |  in pipelines: 135(GETNEXT)
+|  |
+|  25:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=13
+|  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=7,3 row-size=32B cardinality=148.80K cost=446400
+|  |  in pipelines: 135(GETNEXT), 04(OPEN)
+|  |
+|  |--F94:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=5.30MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=3.00MB
+|  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  JOIN BUILD
+|  |  |  join-table-id=13 plan-id=14 cohort-id=05
+|  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  runtime filters: RF006[bloom] <- i_brand_id, RF007[bloom] <- i_category_id, RF008[bloom] <- i_class_id, RF009[min_max] <- i_brand_id, RF010[min_max] <- i_category_id, RF011[min_max] <- i_class_id
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |
+|  |  146:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=20B cardinality=18.00K cost=422
+|  |  |  in pipelines: 04(GETNEXT)
+|  |  |
+|  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  04:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=3 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 04(GETNEXT)
+|  |
+|  24:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=14
+|  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 135(GETNEXT), 144(OPEN)
+|  |
+|  |--F95:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=5.76MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=2.00MB
+|  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  JOIN BUILD
+|  |  |  join-table-id=14 plan-id=15 cohort-id=05
+|  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  runtime filters: RF012[bloom] <- iws.i_brand_id, RF013[bloom] <- iws.i_category_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |
+|  |  145:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=125 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 144(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  144:AGGREGATE [FINALIZE]
+|  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=125 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 144(GETNEXT), 16(OPEN)
+|  |  |
+|  |  143:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=125 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 16(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  23:AGGREGATE [STREAMING]
+|  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=125 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  in pipelines: 16(GETNEXT)
+|  |  |
+|  |  20:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=15
+|  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=13,14,15 row-size=40B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 16(GETNEXT), 18(OPEN)
+|  |  |
+|  |  |--F96:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=15 plan-id=16 cohort-id=06
+|  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  runtime filters: RF027[min_max] <- d3.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  142:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=15 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  in pipelines: 18(GETNEXT)
+|  |  |  |
+|  |  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  18:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=15 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 18(GETNEXT)
+|  |  |
+|  |  19:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=16
+|  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=13,14 row-size=32B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |  |
+|  |  |--F97:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=16 plan-id=17 cohort-id=06
+|  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  runtime filters: RF029[min_max] <- iws.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  141:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  in pipelines: 17(GETNEXT)
+|  |  |  |
+|  |  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  17:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=14 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 17(GETNEXT)
+|  |  |
+|  |  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF027[min_max] -> ws_sold_date_sk, RF029[min_max] -> ws_item_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=13 row-size=12B cardinality=719.38K cost=8431
+|  |     in pipelines: 16(GETNEXT)
+|  |
+|  22:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=17
+|  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 135(GETNEXT), 139(OPEN)
+|  |
+|  |--F98:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  JOIN BUILD
+|  |  |  join-table-id=17 plan-id=18 cohort-id=05
+|  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |
+|  |  140:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=124 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 139(GETNEXT)
+|  |  |
+|  |  F10:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  139:AGGREGATE [FINALIZE]
+|  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=124 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 139(GETNEXT), 11(OPEN)
+|  |  |
+|  |  138:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=124 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 11(GETNEXT)
+|  |  |
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  21:AGGREGATE [STREAMING]
+|  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=124 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  in pipelines: 11(GETNEXT)
+|  |  |
+|  |  15:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=18
+|  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10,11 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 11(GETNEXT), 13(OPEN)
+|  |  |
+|  |  |--F99:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=18 plan-id=19 cohort-id=07
+|  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  runtime filters: RF023[min_max] <- d2.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  137:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=11 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 13(GETNEXT)
+|  |  |  |
+|  |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  13:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=11 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 13(GETNEXT)
+|  |  |
+|  |  14:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=19
+|  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 11(GETNEXT), 12(OPEN)
+|  |  |
+|  |  |--F100:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=19 plan-id=20 cohort-id=07
+|  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  runtime filters: RF025[min_max] <- ics.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  136:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=10 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 12(GETNEXT)
+|  |  |  |
+|  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  12:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=10 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 12(GETNEXT)
+|  |  |
+|  |  11:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF023[min_max] -> cs_sold_date_sk, RF025[min_max] -> cs_item_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=9 row-size=12B cardinality=1.44M cost=16894
+|  |     in pipelines: 11(GETNEXT)
+|  |
+|  135:AGGREGATE [FINALIZE]
+|  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 135(GETNEXT), 05(OPEN)
+|  |
+|  134:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=2325
+|  |  in pipelines: 05(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  10:AGGREGATE [STREAMING]
+|  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=8641212
+|  |  in pipelines: 05(GETNEXT)
+|  |
+|  09:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=20
+|  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5,6 row-size=40B cardinality=2.88M cost=2880404
+|  |  in pipelines: 05(GETNEXT), 07(OPEN)
+|  |
+|  |--F101:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  JOIN BUILD
+|  |  |  join-table-id=20 plan-id=21 cohort-id=05
+|  |  |  build expressions: d1.d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |
+|  |  133:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=6 row-size=8B cardinality=7.30K cost=258
+|  |  |  in pipelines: 07(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146755]
+|  |  07:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=6 row-size=8B cardinality=7.30K cost=146669
+|  |     in pipelines: 07(GETNEXT)
+|  |
+|  08:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=21
+|  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5 row-size=32B cardinality=2.88M cost=2880404
+|  |  in pipelines: 05(GETNEXT), 06(OPEN)
+|  |
+|  |--F102:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  JOIN BUILD
+|  |  |  join-table-id=21 plan-id=22 cohort-id=05
+|  |  |  build expressions: iss.i_item_sk
+|  |  |  runtime filters: RF021[min_max] <- iss.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  132:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=20B cardinality=18.00K cost=1266
+|  |  |  in pipelines: 06(GETNEXT)
+|  |  |
+|  |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  06:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF009[min_max] -> iss.i_brand_id, RF010[min_max] -> iss.i_category_id, RF011[min_max] -> iss.i_class_id, RF006[bloom] -> iss.i_brand_id, RF007[bloom] -> iss.i_category_id, RF008[bloom] -> iss.i_class_id, RF012[bloom] -> iss.i_brand_id, RF013[bloom] -> iss.i_category_id
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=5 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 06(GETNEXT)
+|  |
+|  05:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF021[min_max] -> ss_item_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=4 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 05(GETNEXT)
+|
+27:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=22
+|  hash predicates: ss_item_sk = i_item_sk
+|  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,2,1 row-size=52B cardinality=170.55K cost=170551
+|  in pipelines: 01(GETNEXT), 02(OPEN)
+|
+|--F103:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=9.12MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[19266]
+|  JOIN BUILD
+|  |  join-table-id=22 plan-id=23 cohort-id=01
+|  |  build expressions: i_item_sk
+|  |  runtime filters: RF002[bloom] <- i_item_sk, RF003[min_max] <- i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |
+|  131:EXCHANGE [BROADCAST]
+|  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1 row-size=20B cardinality=18.00K cost=1266
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[774]
+|  02:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=1 row-size=20B cardinality=18.00K cost=352
+|     in pipelines: 02(GETNEXT)
+|
+26:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=23
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,2 row-size=32B cardinality=170.55K cost=2880404
+|  in pipelines: 01(GETNEXT), 03(OPEN)
+|
+|--F104:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[114]
+|  JOIN BUILD
+|  |  join-table-id=23 plan-id=24 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF004[bloom] <- d_date_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |
+|  130:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=12B cardinality=108 cost=6
+|  |  in pipelines: 03(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[146957]
+|  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=2 row-size=12B cardinality=108 cost=146955
+|     in pipelines: 03(GETNEXT)
+|
+01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF001[min_max] -> ss_item_sk, RF003[min_max] -> ss_item_sk, RF004[bloom] -> ss_sold_date_sk, RF000[bloom] -> ss_item_sk, RF002[bloom] -> ss_item_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=0 row-size=20B cardinality=2.88M cost=56258
+   in pipelines: 01(GETNEXT)
+====
+# TPCDS-Q14b second of two queries
+WITH cross_items AS
+
+  (SELECT i_item_sk ss_item_sk
+   FROM item,
+     (SELECT iss.i_brand_id brand_id,
+             iss.i_class_id class_id,
+             iss.i_category_id category_id
+      FROM store_sales,
+           item iss,
+           date_dim d1
+      WHERE ss_item_sk = iss.i_item_sk
+        AND ss_sold_date_sk = d1.d_date_sk
+        AND d1.d_year BETWEEN 1999 AND 1999 + 2 INTERSECT
+        SELECT ics.i_brand_id,
+               ics.i_class_id,
+               ics.i_category_id
+        FROM catalog_sales,
+             item ics,
+             date_dim d2 WHERE cs_item_sk = ics.i_item_sk
+        AND cs_sold_date_sk = d2.d_date_sk
+        AND d2.d_year BETWEEN 1999 AND 1999 + 2 INTERSECT
+        SELECT iws.i_brand_id,
+               iws.i_class_id,
+               iws.i_category_id
+        FROM web_sales,
+             item iws,
+             date_dim d3 WHERE ws_item_sk = iws.i_item_sk
+        AND ws_sold_date_sk = d3.d_date_sk
+        AND d3.d_year BETWEEN 1999 AND 1999 + 2) x
+   WHERE i_brand_id = brand_id
+     AND i_class_id = class_id
+     AND i_category_id = category_id ),
+     avg_sales AS
+
+  (SELECT avg(quantity*list_price) average_sales
+   FROM
+     (SELECT ss_quantity quantity,
+             ss_list_price list_price
+      FROM store_sales,
+           date_dim
+      WHERE ss_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2
+      UNION ALL SELECT cs_quantity quantity,
+                       cs_list_price list_price
+      FROM catalog_sales,
+           date_dim
+      WHERE cs_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2
+      UNION ALL SELECT ws_quantity quantity,
+                       ws_list_price list_price
+      FROM web_sales,
+           date_dim
+      WHERE ws_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2) x)
+SELECT this_year.channel ty_channel,
+       this_year.i_brand_id ty_brand,
+       this_year.i_class_id ty_class,
+       this_year.i_category_id ty_category,
+       this_year.sales ty_sales,
+       this_year.number_sales ty_number_sales,
+       last_year.channel ly_channel,
+       last_year.i_brand_id ly_brand,
+       last_year.i_class_id ly_class,
+       last_year.i_category_id ly_category,
+       last_year.sales ly_sales,
+       last_year.number_sales ly_number_sales
+FROM
+  (SELECT 'store' channel,
+                  i_brand_id,
+                  i_class_id,
+                  i_category_id,
+                  sum(ss_quantity*ss_list_price) sales,
+                  count(*) number_sales
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_item_sk IN
+
+       (SELECT ss_item_sk
+        FROM cross_items)
+     AND ss_item_sk = i_item_sk
+     AND ss_sold_date_sk = d_date_sk
+     AND d_week_seq =
+       (SELECT d_week_seq
+        FROM date_dim
+        WHERE d_year = 1999 + 1
+          AND d_moy = 12
+          AND d_dom = 11)
+   GROUP BY i_brand_id,
+            i_class_id,
+            i_category_id
+   HAVING sum(ss_quantity*ss_list_price) >
+     (SELECT average_sales
+      FROM avg_sales)) this_year,
+  (SELECT 'store' channel,
+                  i_brand_id,
+                  i_class_id,
+                  i_category_id,
+                  sum(ss_quantity*ss_list_price) sales,
+                  count(*) number_sales
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_item_sk IN
+
+       (SELECT ss_item_sk
+        FROM cross_items)
+     AND ss_item_sk = i_item_sk
+     AND ss_sold_date_sk = d_date_sk
+     AND d_week_seq =
+       (SELECT d_week_seq
+        FROM date_dim
+        WHERE d_year = 1999
+          AND d_moy = 12
+          AND d_dom = 11)
+   GROUP BY i_brand_id,
+            i_class_id,
+            i_category_id
+   HAVING sum(ss_quantity*ss_list_price) >
+     (SELECT average_sales
+      FROM avg_sales)) last_year
+WHERE this_year.i_brand_id= last_year.i_brand_id
+  AND this_year.i_class_id = last_year.i_class_id
+  AND this_year.i_category_id = last_year.i_category_id
+ORDER BY this_year.channel,
+         this_year.i_brand_id,
+         this_year.i_class_id,
+         this_year.i_category_id
+LIMIT 100
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=441.81MB Threads=103
+Per-Host Resource Estimates: Memory=1.78GB
+F56:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[61]
+PLAN-ROOT SINK
+|  output exprs: channel, i_brand_id, i_class_id, i_category_id, sales, number_sales, channel, i_brand_id, i_class_id, i_category_id, sales, number_sales
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=60
+|
+152:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: channel ASC, i_brand_id ASC, i_class_id ASC, i_category_id ASC
+|  limit: 100
+|  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=74 row-size=96B cardinality=5 cost=1
+|  in pipelines: 91(GETNEXT)
+|
+F19:PLAN FRAGMENT [HASH(i_brand_id,i_class_id,i_category_id)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=10.13MB mem-reservation=1.94MB thread-reservation=1
+max-parallelism=3 fragment-costs=[1366, 833, 1]
+91:TOP-N [LIMIT=100]
+|  order by: channel ASC, i_brand_id ASC, i_class_id ASC, i_category_id ASC
+|  mem-estimate=480B mem-reservation=0B thread-reservation=0
+|  tuple-ids=74 row-size=96B cardinality=5 cost=20
+|  in pipelines: 91(GETNEXT), 115(OPEN)
+|
+90:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: i_brand_id = i_brand_id, i_category_id = i_category_id, i_class_id = i_class_id
+|  fk/pk conjuncts: i_brand_id = i_brand_id, i_category_id = i_category_id, i_class_id = i_class_id
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=22,33,59,70 row-size=104B cardinality=5 cost=813
+|  in pipelines: 115(GETNEXT), 145(OPEN)
+|
+|--F57:PLAN FRAGMENT [HASH(i_brand_id,i_class_id,i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=14.94MB mem-reservation=6.88MB thread-reservation=1 runtime-filters-memory=3.00MB
+|  |  max-parallelism=3 fragment-costs=[1366, 813]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  runtime filters: RF000[bloom] <- i_brand_id, RF001[bloom] <- i_category_id, RF002[bloom] <- i_class_id, RF003[min_max] <- i_brand_id, RF004[min_max] <- i_category_id, RF005[min_max] <- i_class_id
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=813
+|  |
+|  89:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  |  join table id: 01
+|  |  predicates: sum(ss_quantity * ss_list_price) > avg(quantity * list_price)
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=59,70 row-size=52B cardinality=271 cost=0
+|  |  in pipelines: 145(GETNEXT), 150(OPEN)
+|  |
+|  |--F58:PLAN FRAGMENT [HASH(i_brand_id,i_class_id,i_category_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |
+|  |  151:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=70 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 150(GETNEXT)
+|  |  |
+|  |  F55:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  150:AGGREGATE [FINALIZE]
+|  |  |  output: avg:merge(quantity * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=70 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 150(GETNEXT), 88(OPEN)
+|  |  |
+|  |  149:EXCHANGE [UNPARTITIONED]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=69 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 88(GETNEXT)
+|  |  |
+|  |  F54:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[40389768, 1]
+|  |  88:AGGREGATE
+|  |  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=69 row-size=16B cardinality=1 cost=5041336
+|  |  |  in pipelines: 88(GETNEXT), 79(OPEN), 82(OPEN), 85(OPEN)
+|  |  |
+|  |  78:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=67 row-size=8B cardinality=5.04M cost=30248016
+|  |  |  in pipelines: 79(GETNEXT), 82(GETNEXT), 85(GETNEXT)
+|  |  |
+|  |  |--87:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=04
+|  |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=65,66 row-size=20B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 85(GETNEXT), 86(OPEN)
+|  |  |  |
+|  |  |  |--F61:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=04 plan-id=05 cohort-id=03
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF081[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  148:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=66 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 86(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  86:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=66 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 86(GETNEXT)
+|  |  |  |
+|  |  |  85:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF081[min_max] -> ws_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=65 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 85(GETNEXT)
+|  |  |
+|  |  |--84:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=03
+|  |  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=63,64 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 82(GETNEXT), 83(OPEN)
+|  |  |  |
+|  |  |  |--F60:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF079[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  147:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=64 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 83(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F51:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  83:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=64 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 83(GETNEXT)
+|  |  |  |
+|  |  |  82:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF079[min_max] -> cs_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=63 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 82(GETNEXT)
+|  |  |
+|  |  81:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=61,62 row-size=20B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 79(GETNEXT), 80(OPEN)
+|  |  |
+|  |  |--F59:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  146:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=62 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 80(GETNEXT)
+|  |  |  |
+|  |  |  F49:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  80:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=62 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 80(GETNEXT)
+|  |  |
+|  |  79:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=61 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 79(GETNEXT)
+|  |
+|  145:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ss_quantity * ss_list_price), count:merge(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=59 row-size=36B cardinality=271 cost=1355
+|  |  in pipelines: 145(GETNEXT), 45(OPEN)
+|  |
+|  144:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  mem-estimate=130.21KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=59 row-size=36B cardinality=271 cost=11
+|  |  in pipelines: 45(GETNEXT)
+|  |
+|  F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=26.47MB mem-reservation=3.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[11579229, 11]
+|  77:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=59 row-size=36B cardinality=271 cost=1355
+|  |  in pipelines: 45(GETNEXT)
+|  |
+|  76:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=05
+|  |  hash predicates: d_week_seq = d_week_seq
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=37,38,39 row-size=48B cardinality=271 cost=2880404
+|  |  in pipelines: 45(GETNEXT), 71(OPEN)
+|  |
+|  |--F62:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[4]
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=02
+|  |  |  build expressions: d_week_seq
+|  |  |  runtime filters: RF044[bloom] <- d_week_seq, RF045[min_max] <- d_week_seq
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1
+|  |  |
+|  |  143:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 71(GETNEXT)
+|  |  |
+|  |  F46:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=80.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1, 1]
+|  |  71:CARDINALITY CHECK
+|  |  |  limit: 1
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=16B cardinality=1 cost=0
+|  |  |  in pipelines: 71(GETNEXT), 70(OPEN)
+|  |  |
+|  |  142:EXCHANGE [UNPARTITIONED]
+|  |  |  limit: 2
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=16B cardinality=2 cost=1
+|  |  |  in pipelines: 70(GETNEXT)
+|  |  |
+|  |  F45:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[220290]
+|  |  70:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(1999 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(1999 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(1999 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|  |     limit: 2
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=57 row-size=16B cardinality=2 cost=220289
+|  |     in pipelines: 70(GETNEXT)
+|  |
+|  75:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=06
+|  |  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=37,38,39 row-size=48B cardinality=2.88M cost=2880404
+|  |  in pipelines: 45(GETNEXT), 140(OPEN)
+|  |
+|  |--F63:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[18608]
+|  |  JOIN BUILD
+|  |  |  join-table-id=06 plan-id=07 cohort-id=02
+|  |  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  |  runtime filters: RF047[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |
+|  |  141:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=114 row-size=8B cardinality=17.98K cost=633
+|  |  |  in pipelines: 140(GETNEXT)
+|  |  |
+|  |  F44:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  140:AGGREGATE [FINALIZE]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=114 row-size=8B cardinality=17.98K cost=17975
+|  |  |  in pipelines: 140(GETNEXT), 127(OPEN)
+|  |  |
+|  |  139:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=114 row-size=8B cardinality=17.98K cost=211
+|  |  |  in pipelines: 127(GETNEXT)
+|  |  |
+|  |  F34:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  |  74:AGGREGATE [STREAMING]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=114 row-size=8B cardinality=17.98K cost=148800
+|  |  |  in pipelines: 127(GETNEXT)
+|  |  |
+|  |  69:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=07
+|  |  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=44,40 row-size=32B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 127(GETNEXT), 48(OPEN)
+|  |  |
+|  |  |--F64:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=2.30MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=07 plan-id=08 cohort-id=04
+|  |  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  |  runtime filters: RF055[min_max] <- i_brand_id, RF056[min_max] <- i_category_id, RF057[min_max] <- i_class_id
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |  |
+|  |  |  138:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=40 row-size=20B cardinality=18.00K cost=422
+|  |  |  |  in pipelines: 48(GETNEXT)
+|  |  |  |
+|  |  |  F43:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  48:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=40 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 48(GETNEXT)
+|  |  |
+|  |  68:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=08
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 127(GETNEXT), 136(OPEN)
+|  |  |
+|  |  |--F65:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=08 plan-id=09 cohort-id=04
+|  |  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  137:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=112 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 136(GETNEXT)
+|  |  |  |
+|  |  |  F42:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  |  136:AGGREGATE [FINALIZE]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=112 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 136(GETNEXT), 60(OPEN)
+|  |  |  |
+|  |  |  135:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=112 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 60(GETNEXT)
+|  |  |  |
+|  |  |  F39:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  |  67:AGGREGATE [STREAMING]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=112 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  |  in pipelines: 60(GETNEXT)
+|  |  |  |
+|  |  |  64:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=09
+|  |  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=50,51,52 row-size=40B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 60(GETNEXT), 62(OPEN)
+|  |  |  |
+|  |  |  |--F66:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=09 plan-id=10 cohort-id=05
+|  |  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  |  runtime filters: RF073[min_max] <- d3.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  134:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=52 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  |  in pipelines: 62(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F41:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  62:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=52 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 62(GETNEXT)
+|  |  |  |
+|  |  |  63:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=10
+|  |  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=50,51 row-size=32B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 60(GETNEXT), 61(OPEN)
+|  |  |  |
+|  |  |  |--F67:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=05
+|  |  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  |  runtime filters: RF075[min_max] <- iws.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  133:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=51 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  |  in pipelines: 61(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F40:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  61:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=51 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 61(GETNEXT)
+|  |  |  |
+|  |  |  60:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF073[min_max] -> ws_sold_date_sk, RF075[min_max] -> ws_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=50 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 60(GETNEXT)
+|  |  |
+|  |  66:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=11
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 127(GETNEXT), 131(OPEN)
+|  |  |
+|  |  |--F68:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=11 plan-id=12 cohort-id=04
+|  |  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  132:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=111 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 131(GETNEXT)
+|  |  |  |
+|  |  |  F38:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  |  131:AGGREGATE [FINALIZE]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=111 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 131(GETNEXT), 55(OPEN)
+|  |  |  |
+|  |  |  130:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=111 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 55(GETNEXT)
+|  |  |  |
+|  |  |  F35:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  |  65:AGGREGATE [STREAMING]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=111 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  |  in pipelines: 55(GETNEXT)
+|  |  |  |
+|  |  |  59:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=12
+|  |  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=46,47,48 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 55(GETNEXT), 57(OPEN)
+|  |  |  |
+|  |  |  |--F69:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=12 plan-id=13 cohort-id=06
+|  |  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  |  runtime filters: RF069[min_max] <- d2.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  129:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=48 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 57(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F37:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  57:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=48 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 57(GETNEXT)
+|  |  |  |
+|  |  |  58:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=13
+|  |  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=46,47 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 55(GETNEXT), 56(OPEN)
+|  |  |  |
+|  |  |  |--F70:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=13 plan-id=14 cohort-id=06
+|  |  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  |  runtime filters: RF071[min_max] <- ics.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  128:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=47 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  |  in pipelines: 56(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F36:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  56:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=47 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 56(GETNEXT)
+|  |  |  |
+|  |  |  55:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF069[min_max] -> cs_sold_date_sk, RF071[min_max] -> cs_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=46 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 55(GETNEXT)
+|  |  |
+|  |  127:AGGREGATE [FINALIZE]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 127(GETNEXT), 49(OPEN)
+|  |  |
+|  |  126:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 49(GETNEXT)
+|  |  |
+|  |  F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  |  54:AGGREGATE [STREAMING]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=8641212
+|  |  |  in pipelines: 49(GETNEXT)
+|  |  |
+|  |  53:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=14
+|  |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=41,42,43 row-size=40B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 49(GETNEXT), 51(OPEN)
+|  |  |
+|  |  |--F71:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=14 plan-id=15 cohort-id=04
+|  |  |  |  build expressions: d1.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  125:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=43 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 51(GETNEXT)
+|  |  |  |
+|  |  |  F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  51:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=43 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 51(GETNEXT)
+|  |  |
+|  |  52:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=15
+|  |  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=41,42 row-size=32B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 49(GETNEXT), 50(OPEN)
+|  |  |
+|  |  |--F72:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=15 plan-id=16 cohort-id=04
+|  |  |  |  build expressions: iss.i_item_sk
+|  |  |  |  runtime filters: RF067[min_max] <- iss.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  124:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=42 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 50(GETNEXT)
+|  |  |  |
+|  |  |  F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  50:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     runtime filters: RF055[min_max] -> iss.i_brand_id, RF056[min_max] -> iss.i_category_id, RF057[min_max] -> iss.i_class_id
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=42 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 50(GETNEXT)
+|  |  |
+|  |  49:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF067[min_max] -> ss_item_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=41 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 49(GETNEXT)
+|  |
+|  73:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=16
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=37,38,39 row-size=48B cardinality=2.88M cost=2880404
+|  |  in pipelines: 45(GETNEXT), 47(OPEN)
+|  |
+|  |--F73:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=19.57MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[75620]
+|  |  JOIN BUILD
+|  |  |  join-table-id=16 plan-id=17 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=73049
+|  |  |
+|  |  123:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=582.70KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=39 row-size=8B cardinality=73.05K cost=2571
+|  |  |  in pipelines: 47(GETNEXT)
+|  |  |
+|  |  F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1428]
+|  |  47:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     runtime filters: RF045[min_max] -> d_week_seq, RF044[bloom] -> d_week_seq
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=39 row-size=8B cardinality=73.05K cost=571
+|  |     in pipelines: 47(GETNEXT)
+|  |
+|  72:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=17
+|  |  hash predicates: ss_item_sk = i_item_sk
+|  |  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=37,38 row-size=40B cardinality=2.88M cost=2880404
+|  |  in pipelines: 45(GETNEXT), 46(OPEN)
+|  |
+|  |--F74:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  JOIN BUILD
+|  |  |  join-table-id=17 plan-id=18 cohort-id=02
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF051[min_max] <- i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  122:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=38 row-size=20B cardinality=18.00K cost=1266
+|  |  |  in pipelines: 46(GETNEXT)
+|  |  |
+|  |  F29:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  46:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF047[min_max] -> tpcds_parquet.item.i_item_sk
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=38 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 46(GETNEXT)
+|  |
+|  45:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF047[min_max] -> ss_item_sk, RF051[min_max] -> ss_item_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|     tuple-ids=37 row-size=20B cardinality=2.88M cost=56258
+|     in pipelines: 45(GETNEXT)
+|
+44:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  join table id: 18
+|  predicates: sum(ss_quantity * ss_list_price) > avg(quantity * list_price)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=22,33 row-size=52B cardinality=271 cost=0
+|  in pipelines: 115(GETNEXT), 120(OPEN)
+|
+|--F75:PLAN FRAGMENT [HASH(i_brand_id,i_class_id,i_category_id)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[3]
+|  JOIN BUILD
+|  |  join-table-id=18 plan-id=19 cohort-id=01
+|  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |
+|  121:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=33 row-size=16B cardinality=1 cost=3
+|  |  in pipelines: 120(GETNEXT)
+|  |
+|  F27:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[2, 1]
+|  120:AGGREGATE [FINALIZE]
+|  |  output: avg:merge(quantity * list_price)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=33 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 120(GETNEXT), 43(OPEN)
+|  |
+|  119:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=32 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 43(GETNEXT)
+|  |
+|  F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[40389768, 1]
+|  43:AGGREGATE
+|  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=32 row-size=16B cardinality=1 cost=5041336
+|  |  in pipelines: 43(GETNEXT), 34(OPEN), 37(OPEN), 40(OPEN)
+|  |
+|  33:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=30 row-size=8B cardinality=5.04M cost=30248016
+|  |  in pipelines: 34(GETNEXT), 37(GETNEXT), 40(GETNEXT)
+|  |
+|  |--42:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=21
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=28,29 row-size=20B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 40(GETNEXT), 41(OPEN)
+|  |  |
+|  |  |--F78:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=21 plan-id=22 cohort-id=07
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF043[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  118:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=29 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 41(GETNEXT)
+|  |  |  |
+|  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  41:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=29 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 41(GETNEXT)
+|  |  |
+|  |  40:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF043[min_max] -> ws_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=28 row-size=12B cardinality=719.38K cost=8431
+|  |     in pipelines: 40(GETNEXT)
+|  |
+|  |--39:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=20
+|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=26,27 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 37(GETNEXT), 38(OPEN)
+|  |  |
+|  |  |--F77:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=20 plan-id=21 cohort-id=07
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF041[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  117:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 38(GETNEXT)
+|  |  |  |
+|  |  |  F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  38:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=27 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 38(GETNEXT)
+|  |  |
+|  |  37:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF041[min_max] -> cs_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=26 row-size=12B cardinality=1.44M cost=16894
+|  |     in pipelines: 37(GETNEXT)
+|  |
+|  36:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=19
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=24,25 row-size=20B cardinality=2.88M cost=2880404
+|  |  in pipelines: 34(GETNEXT), 35(OPEN)
+|  |
+|  |--F76:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  JOIN BUILD
+|  |  |  join-table-id=19 plan-id=20 cohort-id=07
+|  |  |  build expressions: d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |
+|  |  116:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=25 row-size=8B cardinality=7.30K cost=258
+|  |  |  in pipelines: 35(GETNEXT)
+|  |  |
+|  |  F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146755]
+|  |  35:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=25 row-size=8B cardinality=7.30K cost=146669
+|  |     in pipelines: 35(GETNEXT)
+|  |
+|  34:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=24 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 34(GETNEXT)
+|
+115:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_quantity * ss_list_price), count:merge(*)
+|  group by: i_brand_id, i_class_id, i_category_id
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=22 row-size=36B cardinality=271 cost=1355
+|  in pipelines: 115(GETNEXT), 00(OPEN)
+|
+114:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  mem-estimate=130.21KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=22 row-size=36B cardinality=271 cost=11
+|  in pipelines: 00(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+Per-Instance Resources: mem-estimate=26.47MB mem-reservation=3.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[11579229, 11]
+32:AGGREGATE [STREAMING]
+|  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*)
+|  group by: i_brand_id, i_class_id, i_category_id
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=22 row-size=36B cardinality=271 cost=1355
+|  in pipelines: 00(GETNEXT)
+|
+31:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash-table-id=22
+|  hash predicates: d_week_seq = d_week_seq
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1,2 row-size=48B cardinality=271 cost=2880404
+|  in pipelines: 00(GETNEXT), 26(OPEN)
+|
+|--F79:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[4]
+|  JOIN BUILD
+|  |  join-table-id=22 plan-id=23 cohort-id=01
+|  |  build expressions: d_week_seq
+|  |  runtime filters: RF006[bloom] <- d_week_seq, RF007[min_max] <- d_week_seq
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1
+|  |
+|  113:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=20 row-size=16B cardinality=1 cost=3
+|  |  in pipelines: 26(GETNEXT)
+|  |
+|  F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=80.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1, 1]
+|  26:CARDINALITY CHECK
+|  |  limit: 1
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=20 row-size=16B cardinality=1 cost=0
+|  |  in pipelines: 26(GETNEXT), 25(OPEN)
+|  |
+|  112:EXCHANGE [UNPARTITIONED]
+|  |  limit: 2
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=20 row-size=16B cardinality=2 cost=1
+|  |  in pipelines: 25(GETNEXT)
+|  |
+|  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[220290]
+|  25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2000 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2000 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|     limit: 2
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=20 row-size=16B cardinality=2 cost=220289
+|     in pipelines: 25(GETNEXT)
+|
+30:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash-table-id=23
+|  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1,2 row-size=48B cardinality=2.88M cost=2880404
+|  in pipelines: 00(GETNEXT), 110(OPEN)
+|
+|--F80:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.92MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[18608]
+|  JOIN BUILD
+|  |  join-table-id=23 plan-id=24 cohort-id=01
+|  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  runtime filters: RF008[bloom] <- tpcds_parquet.item.i_item_sk, RF009[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |
+|  111:EXCHANGE [BROADCAST]
+|  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=91 row-size=8B cardinality=17.98K cost=633
+|  |  in pipelines: 110(GETNEXT)
+|  |
+|  F16:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[18186, 211]
+|  110:AGGREGATE [FINALIZE]
+|  |  group by: tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=91 row-size=8B cardinality=17.98K cost=17975
+|  |  in pipelines: 110(GETNEXT), 97(OPEN)
+|  |
+|  109:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=91 row-size=8B cardinality=17.98K cost=211
+|  |  in pipelines: 97(GETNEXT)
+|  |
+|  F06:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  29:AGGREGATE [STREAMING]
+|  |  group by: tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=91 row-size=8B cardinality=17.98K cost=148800
+|  |  in pipelines: 97(GETNEXT)
+|  |
+|  24:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=24
+|  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=7,3 row-size=32B cardinality=148.80K cost=446400
+|  |  in pipelines: 97(GETNEXT), 03(OPEN)
+|  |
+|  |--F81:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=4.30MB mem-reservation=3.94MB thread-reservation=1 runtime-filters-memory=2.00MB
+|  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  JOIN BUILD
+|  |  |  join-table-id=24 plan-id=25 cohort-id=08
+|  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  runtime filters: RF014[bloom] <- i_brand_id, RF015[bloom] <- i_category_id, RF017[min_max] <- i_brand_id, RF018[min_max] <- i_category_id, RF019[min_max] <- i_class_id
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |
+|  |  108:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=20B cardinality=18.00K cost=422
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  03:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=3 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 03(GETNEXT)
+|  |
+|  23:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=25
+|  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 97(GETNEXT), 106(OPEN)
+|  |
+|  |--F82:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  JOIN BUILD
+|  |  |  join-table-id=25 plan-id=26 cohort-id=08
+|  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |
+|  |  107:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=89 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 106(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  106:AGGREGATE [FINALIZE]
+|  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=89 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 106(GETNEXT), 15(OPEN)
+|  |  |
+|  |  105:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=89 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 15(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  22:AGGREGATE [STREAMING]
+|  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=89 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  in pipelines: 15(GETNEXT)
+|  |  |
+|  |  19:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=26
+|  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=13,14,15 row-size=40B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 15(GETNEXT), 17(OPEN)
+|  |  |
+|  |  |--F83:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=26 plan-id=27 cohort-id=09
+|  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  runtime filters: RF035[min_max] <- d3.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  104:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=15 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  in pipelines: 17(GETNEXT)
+|  |  |  |
+|  |  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  17:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=15 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 17(GETNEXT)
+|  |  |
+|  |  18:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=27
+|  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=13,14 row-size=32B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 15(GETNEXT), 16(OPEN)
+|  |  |
+|  |  |--F84:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=27 plan-id=28 cohort-id=09
+|  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  runtime filters: RF037[min_max] <- iws.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  103:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  in pipelines: 16(GETNEXT)
+|  |  |  |
+|  |  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  16:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=14 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 16(GETNEXT)
+|  |  |
+|  |  15:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF035[min_max] -> ws_sold_date_sk, RF037[min_max] -> ws_item_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=13 row-size=12B cardinality=719.38K cost=8431
+|  |     in pipelines: 15(GETNEXT)
+|  |
+|  21:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=28
+|  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 97(GETNEXT), 101(OPEN)
+|  |
+|  |--F85:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  JOIN BUILD
+|  |  |  join-table-id=28 plan-id=29 cohort-id=08
+|  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |
+|  |  102:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=88 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 101(GETNEXT)
+|  |  |
+|  |  F10:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  101:AGGREGATE [FINALIZE]
+|  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=88 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 101(GETNEXT), 10(OPEN)
+|  |  |
+|  |  100:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=88 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 10(GETNEXT)
+|  |  |
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  20:AGGREGATE [STREAMING]
+|  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=88 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  in pipelines: 10(GETNEXT)
+|  |  |
+|  |  14:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=29
+|  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10,11 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 10(GETNEXT), 12(OPEN)
+|  |  |
+|  |  |--F86:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=29 plan-id=30 cohort-id=10
+|  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  runtime filters: RF031[min_max] <- d2.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  99:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=11 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 12(GETNEXT)
+|  |  |  |
+|  |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  12:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=11 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 12(GETNEXT)
+|  |  |
+|  |  13:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=30
+|  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 10(GETNEXT), 11(OPEN)
+|  |  |
+|  |  |--F87:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=30 plan-id=31 cohort-id=10
+|  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  runtime filters: RF033[min_max] <- ics.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  98:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=10 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 11(GETNEXT)
+|  |  |  |
+|  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  11:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=10 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 11(GETNEXT)
+|  |  |
+|  |  10:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF031[min_max] -> cs_sold_date_sk, RF033[min_max] -> cs_item_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=9 row-size=12B cardinality=1.44M cost=16894
+|  |     in pipelines: 10(GETNEXT)
+|  |
+|  97:AGGREGATE [FINALIZE]
+|  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 97(GETNEXT), 04(OPEN)
+|  |
+|  96:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=2325
+|  |  in pipelines: 04(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  09:AGGREGATE [STREAMING]
+|  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=8641212
+|  |  in pipelines: 04(GETNEXT)
+|  |
+|  08:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=31
+|  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5,6 row-size=40B cardinality=2.88M cost=2880404
+|  |  in pipelines: 04(GETNEXT), 06(OPEN)
+|  |
+|  |--F88:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  JOIN BUILD
+|  |  |  join-table-id=31 plan-id=32 cohort-id=08
+|  |  |  build expressions: d1.d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |
+|  |  95:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=6 row-size=8B cardinality=7.30K cost=258
+|  |  |  in pipelines: 06(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146755]
+|  |  06:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=6 row-size=8B cardinality=7.30K cost=146669
+|  |     in pipelines: 06(GETNEXT)
+|  |
+|  07:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=32
+|  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5 row-size=32B cardinality=2.88M cost=2880404
+|  |  in pipelines: 04(GETNEXT), 05(OPEN)
+|  |
+|  |--F89:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  JOIN BUILD
+|  |  |  join-table-id=32 plan-id=33 cohort-id=08
+|  |  |  build expressions: iss.i_item_sk
+|  |  |  runtime filters: RF029[min_max] <- iss.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  94:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=20B cardinality=18.00K cost=1266
+|  |  |  in pipelines: 05(GETNEXT)
+|  |  |
+|  |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  05:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF017[min_max] -> iss.i_brand_id, RF018[min_max] -> iss.i_category_id, RF019[min_max] -> iss.i_class_id, RF014[bloom] -> iss.i_brand_id, RF015[bloom] -> iss.i_category_id
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=5 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 05(GETNEXT)
+|  |
+|  04:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF029[min_max] -> ss_item_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=4 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 04(GETNEXT)
+|
+28:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=33
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=0,1,2 row-size=48B cardinality=2.88M cost=2880404
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--F90:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=20.57MB mem-reservation=20.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[75620]
+|  JOIN BUILD
+|  |  join-table-id=33 plan-id=34 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF010[bloom] <- d_date_sk
+|  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=73049
+|  |
+|  93:EXCHANGE [BROADCAST]
+|  |  mem-estimate=582.70KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=8B cardinality=73.05K cost=2571
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1428]
+|  02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     runtime filters: RF007[min_max] -> d_week_seq, RF006[bloom] -> d_week_seq
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=2 row-size=8B cardinality=73.05K cost=571
+|     in pipelines: 02(GETNEXT)
+|
+27:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=34
+|  hash predicates: ss_item_sk = i_item_sk
+|  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1 row-size=40B cardinality=2.88M cost=2880404
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--F91:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=9.12MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[19266]
+|  JOIN BUILD
+|  |  join-table-id=34 plan-id=35 cohort-id=01
+|  |  build expressions: i_item_sk
+|  |  runtime filters: RF012[bloom] <- i_item_sk, RF013[min_max] <- i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |
+|  92:EXCHANGE [BROADCAST]
+|  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1 row-size=20B cardinality=18.00K cost=1266
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB
+|  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[774]
+|  01:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     runtime filters: RF003[min_max] -> tpcds_parquet.item.i_brand_id, RF004[min_max] -> tpcds_parquet.item.i_category_id, RF005[min_max] -> tpcds_parquet.item.i_class_id, RF009[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_brand_id, RF001[bloom] -> tpcds_parquet.item.i_category_id, RF002[bloom] -> tpcds_parquet.item.i_class_id, RF008[bloom] -> tpcds_parquet.item.i_item_sk
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=1 row-size=20B cardinality=18.00K cost=352
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF009[min_max] -> ss_item_sk, RF013[min_max] -> ss_item_sk, RF008[bloom] -> ss_item_sk, RF010[bloom] -> ss_sold_date_sk, RF012[bloom] -> ss_item_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=0 row-size=20B cardinality=2.88M cost=56258
+   in pipelines: 00(GETNEXT)
+====
+# TPCDS-Q23-1 First of two queries.
+with frequent_ss_items as
+ (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,item
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = i_item_sk
+    and d_year in (2000,2000+1,2000+2,2000+3)
+  group by substr(i_item_desc,1,30),i_item_sk,d_date
+  having count(*) >4),
+ max_store_sales as
+ (select max(csales) tpcds_cmax
+  from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales
+        from store_sales
+            ,customer
+            ,date_dim
+        where ss_customer_sk = c_customer_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year in (2000,2000+1,2000+2,2000+3)
+        group by c_customer_sk) x),
+ best_ss_customer as
+ (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales
+  from store_sales
+      ,customer
+  where ss_customer_sk = c_customer_sk
+  group by c_customer_sk
+  having sum(ss_quantity*ss_sales_price) > (50/100.0) * (select
+  *
+from
+ max_store_sales))
+  select  sum(sales)
+ from (select cs_quantity*cs_list_price sales
+       from catalog_sales
+           ,date_dim
+       where d_year = 2000
+         and d_moy = 2
+         and cs_sold_date_sk = d_date_sk
+         and cs_item_sk in (select item_sk from frequent_ss_items)
+         and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer)
+      union all
+      select ws_quantity*ws_list_price sales
+       from web_sales
+           ,date_dim
+       where d_year = 2000
+         and d_moy = 2
+         and ws_sold_date_sk = d_date_sk
+         and ws_item_sk in (select item_sk from frequent_ss_items)
+         and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) y
+ limit 100;
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=572.88MB Threads=75
+Per-Host Resource Estimates: Memory=1.39GB
+F31:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[2, 0]
+PLAN-ROOT SINK
+|  output exprs: sum(sales)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0 cost=0
+|
+89:AGGREGATE [FINALIZE]
+|  output: sum:merge(sales)
+|  limit: 100
+|  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=46 row-size=16B cardinality=1 cost=1
+|  in pipelines: 89(GETNEXT), 49(OPEN)
+|
+88:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=46 row-size=16B cardinality=1 cost=1
+|  in pipelines: 49(GETNEXT)
+|
+F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Instance Resources: mem-estimate=10.93MB mem-reservation=1.94MB thread-reservation=1
+max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
+49:AGGREGATE
+|  output: sum(sales)
+|  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=46 row-size=16B cardinality=1 cost=128163
+|  in pipelines: 49(GETNEXT), 52(OPEN), 71(OPEN)
+|
+00:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=44 row-size=8B cardinality=128.16K cost=256326
+|  in pipelines: 52(GETNEXT), 71(GETNEXT)
+|
+|--48:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=08
+|  |  hash predicates: c_customer_sk = ws_bill_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=22,23 row-size=36B cardinality=42.85K cost=100000
+|  |  in pipelines: 71(GETNEXT), 25(OPEN)
+|  |
+|  |--F40:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=12 fragment-costs=[44696]
+|  |  JOIN BUILD
+|  |  |  join-table-id=08 plan-id=09 cohort-id=01
+|  |  |  build expressions: ws_bill_customer_sk
+|  |  |  runtime filters: RF016[bloom] <- ws_bill_customer_sk, RF017[min_max] <- ws_bill_customer_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=42854
+|  |  |
+|  |  87:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  |  mem-estimate=841.29KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=22,23 row-size=36B cardinality=42.85K cost=1842
+|  |  |  in pipelines: 25(GETNEXT)
+|  |  |
+|  |  F23:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=34.06MB mem-reservation=8.00MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[780941]
+|  |  47:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  |  hash-table-id=09
+|  |  |  hash predicates: ws_item_sk = i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=22,23 row-size=36B cardinality=42.85K cost=42854
+|  |  |  in pipelines: 25(GETNEXT), 85(OPEN)
+|  |  |
+|  |  |--F41:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[18397]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=09 plan-id=10 cohort-id=05
+|  |  |  |  build expressions: i_item_sk
+|  |  |  |  runtime filters: RF025[min_max] <- i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |  |
+|  |  |  86:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=68 row-size=8B cardinality=17.98K cost=422
+|  |  |  |  in pipelines: 85(GETNEXT)
+|  |  |  |
+|  |  |  F29:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  |  85:AGGREGATE [FINALIZE]
+|  |  |  |  group by: i_item_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=68 row-size=8B cardinality=17.98K cost=17975
+|  |  |  |  in pipelines: 85(GETNEXT), 83(OPEN)
+|  |  |  |
+|  |  |  84:EXCHANGE [HASH(i_item_sk)]
+|  |  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=68 row-size=8B cardinality=17.98K cost=211
+|  |  |  |  in pipelines: 83(GETNEXT)
+|  |  |  |
+|  |  |  F28:PLAN FRAGMENT [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=67.04MB mem-reservation=36.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[9542330, 235454, 211]
+|  |  |  46:AGGREGATE [STREAMING]
+|  |  |  |  group by: i_item_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=68 row-size=8B cardinality=17.98K cost=235454
+|  |  |  |  in pipelines: 83(GETNEXT)
+|  |  |  |
+|  |  |  83:AGGREGATE [FINALIZE]
+|  |  |  |  output: count:merge(*)
+|  |  |  |  group by: substr(i_item_desc, 1, 30), i_item_sk, d_date
+|  |  |  |  having: count(*) > CAST(4 AS BIGINT)
+|  |  |  |  mem-estimate=46.41MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=50B cardinality=235.45K cost=9418164
+|  |  |  |  in pipelines: 83(GETNEXT), 27(OPEN)
+|  |  |  |
+|  |  |  82:EXCHANGE [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)]
+|  |  |  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=50B cardinality=2.35M cost=124166
+|  |  |  |  in pipelines: 27(GETNEXT)
+|  |  |  |
+|  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  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=33.63MB mem-reservation=17.50MB thread-reservation=1
+|  |  |  max-parallelism=12 fragment-costs=[14686864, 124166]
+|  |  |  32:AGGREGATE [STREAMING]
+|  |  |  |  output: count(*)
+|  |  |  |  group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date
+|  |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=50B cardinality=2.35M cost=9418164
+|  |  |  |  in pipelines: 27(GETNEXT)
+|  |  |  |
+|  |  |  31:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=10
+|  |  |  |  hash predicates: ss_item_sk = i_item_sk
+|  |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=24,25,26 row-size=162B cardinality=2.35M cost=2354541
+|  |  |  |  in pipelines: 27(GETNEXT), 29(OPEN)
+|  |  |  |
+|  |  |  |--F42:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[24564]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=06
+|  |  |  |  |  build expressions: i_item_sk
+|  |  |  |  |  runtime filters: RF029[min_max] <- i_item_sk
+|  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  81:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=26 row-size=120B cardinality=18.00K cost=6564
+|  |  |  |  |  in pipelines: 29(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[4306]
+|  |  |  |  29:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |  |  |     tuple-ids=26 row-size=120B cardinality=18.00K cost=2118
+|  |  |  |     in pipelines: 29(GETNEXT)
+|  |  |  |
+|  |  |  30:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=11
+|  |  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=24,25 row-size=42B cardinality=2.35M cost=2880404
+|  |  |  |  in pipelines: 27(GETNEXT), 28(OPEN)
+|  |  |  |
+|  |  |  |--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  max-parallelism=3 fragment-costs=[1641]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=11 plan-id=12 cohort-id=06
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF030[bloom] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |
+|  |  |  |  80:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=25 row-size=30B cardinality=1.49K cost=150
+|  |  |  |  |  in pipelines: 28(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[75240]
+|  |  |  |  28:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |  |  |     tuple-ids=25 row-size=30B cardinality=1.49K cost=75190
+|  |  |  |     in pipelines: 28(GETNEXT)
+|  |  |  |
+|  |  |  27:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |  |     runtime filters: RF029[min_max] -> ss_item_sk, RF030[bloom] -> ss_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=2.88M size=200.96MB
+|  |  |       partitions: 1824/1824 rows=2.88M
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=24 row-size=12B cardinality=2.88M cost=33755
+|  |  |     in pipelines: 27(GETNEXT)
+|  |  |
+|  |  45:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=12
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=22,23 row-size=36B cardinality=42.85K cost=719384
+|  |  |  in pipelines: 25(GETNEXT), 26(OPEN)
+|  |  |
+|  |  |--F44:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=2 fragment-costs=[112]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=12 plan-id=13 cohort-id=05
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF026[bloom] <- d_date_sk, RF027[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  |
+|  |  |  79:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=23 row-size=12B cardinality=108 cost=4
+|  |  |  |  in pipelines: 26(GETNEXT)
+|  |  |  |
+|  |  |  F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146957]
+|  |  |  26:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=23 row-size=12B cardinality=108 cost=146955
+|  |  |     in pipelines: 26(GETNEXT)
+|  |  |
+|  |  25:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF027[min_max] -> ws_sold_date_sk, RF025[min_max] -> ws_item_sk, RF026[bloom] -> ws_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
+|  |     tuple-ids=22 row-size=24B cardinality=719.38K cost=16861
+|  |     in pipelines: 25(GETNEXT)
+|  |
+|  44:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  |  join table id: 13
+|  |  predicates: sum(ss_quantity * ss_sales_price) > CAST(0.500000 AS DECIMAL(10,6)) * max(csales)
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=32,39 row-size=36B cardinality=100.00K cost=0
+|  |  in pipelines: 71(GETNEXT), 77(OPEN)
+|  |
+|  |--F45:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  JOIN BUILD
+|  |  |  join-table-id=13 plan-id=14 cohort-id=01
+|  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |
+|  |  78:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=39 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 77(GETNEXT)
+|  |  |
+|  |  F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  77:AGGREGATE [FINALIZE]
+|  |  |  output: max:merge(csales)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=39 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 77(GETNEXT), 43(OPEN)
+|  |  |
+|  |  76:EXCHANGE [UNPARTITIONED]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=39 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 43(GETNEXT)
+|  |  |
+|  |  F21:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[202344, 100000, 1]
+|  |  43:AGGREGATE
+|  |  |  output: max(sum(ss_quantity * ss_sales_price))
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=39 row-size=16B cardinality=1 cost=100000
+|  |  |  in pipelines: 43(GETNEXT), 75(OPEN)
+|  |  |
+|  |  75:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  |  group by: c_customer_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=37 row-size=20B cardinality=100.00K cost=200000
+|  |  |  in pipelines: 75(GETNEXT), 37(OPEN)
+|  |  |
+|  |  74:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=37 row-size=20B cardinality=100.00K cost=2344
+|  |  |  in pipelines: 37(GETNEXT)
+|  |  |
+|  |  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=26.28MB mem-reservation=6.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[9989034, 2344]
+|  |  42:AGGREGATE [STREAMING]
+|  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  |  group by: c_customer_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=37 row-size=20B cardinality=100.00K cost=4709082
+|  |  |  in pipelines: 37(GETNEXT)
+|  |  |
+|  |  41:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=14
+|  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=34,36,35 row-size=28B cardinality=2.35M cost=2354541
+|  |  |  in pipelines: 37(GETNEXT), 38(OPEN)
+|  |  |
+|  |  |--F46:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=14 plan-id=15 cohort-id=07
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF021[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |
+|  |  |  73:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=35 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  in pipelines: 38(GETNEXT)
+|  |  |  |
+|  |  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  38:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=35 row-size=4B cardinality=100.00K cost=391
+|  |  |     in pipelines: 38(GETNEXT)
+|  |  |
+|  |  40:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=15
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=34,36 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  in pipelines: 37(GETNEXT), 39(OPEN)
+|  |  |
+|  |  |--F47:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=3 fragment-costs=[1545]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=15 plan-id=16 cohort-id=07
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF022[bloom] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |
+|  |  |  72:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=36 row-size=8B cardinality=1.49K cost=54
+|  |  |  |  in pipelines: 39(GETNEXT)
+|  |  |  |
+|  |  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[73638]
+|  |  |  39:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=36 row-size=8B cardinality=1.49K cost=73620
+|  |  |     in pipelines: 39(GETNEXT)
+|  |  |
+|  |  37:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF021[min_max] -> ss_customer_sk, RF022[bloom] -> ss_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=34 row-size=16B cardinality=2.88M cost=45007
+|  |     in pipelines: 37(GETNEXT)
+|  |
+|  71:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  group by: c_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=32 row-size=20B cardinality=100.00K cost=200000
+|  |  in pipelines: 71(GETNEXT), 33(OPEN)
+|  |
+|  70:EXCHANGE [HASH(c_customer_sk)]
+|  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=32 row-size=20B cardinality=100.00K cost=2344
+|  |  in pipelines: 33(GETNEXT)
+|  |
+|  F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=27.12MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[8674967, 2344]
+|  36:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  group by: c_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=32 row-size=20B cardinality=100.00K cost=5760808
+|  |  in pipelines: 33(GETNEXT)
+|  |
+|  35:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=17
+|  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=30,31 row-size=16B cardinality=2.88M cost=2880404
+|  |  in pipelines: 33(GETNEXT), 34(OPEN)
+|  |
+|  |--F49:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  JOIN BUILD
+|  |  |  join-table-id=17 plan-id=18 cohort-id=01
+|  |  |  build expressions: c_customer_sk
+|  |  |  runtime filters: RF019[min_max] <- c_customer_sk
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |
+|  |  69:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=31 row-size=4B cardinality=100.00K cost=2346
+|  |  |  in pipelines: 34(GETNEXT)
+|  |  |
+|  |  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1173]
+|  |  34:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     runtime filters: RF017[min_max] -> tpcds_parquet.customer.c_customer_sk, RF016[bloom] -> tpcds_parquet.customer.c_customer_sk
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=31 row-size=4B cardinality=100.00K cost=391
+|  |     in pipelines: 34(GETNEXT)
+|  |
+|  33:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF017[min_max] -> tpcds_parquet.store_sales.ss_customer_sk, RF019[min_max] -> ss_customer_sk, RF016[bloom] -> tpcds_parquet.store_sales.ss_customer_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|     tuple-ids=30 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 33(GETNEXT)
+|
+24:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: c_customer_sk = cs_bill_customer_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1 row-size=36B cardinality=85.31K cost=100000
+|  in pipelines: 52(GETNEXT), 01(OPEN)
+|
+|--F32:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=12 fragment-costs=[88975]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: cs_bill_customer_sk
+|  |  runtime filters: RF000[bloom] <- cs_bill_customer_sk, RF001[min_max] <- cs_bill_customer_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=85309
+|  |
+|  68:EXCHANGE [HASH(cs_bill_customer_sk)]
+|  |  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,1 row-size=36B cardinality=85.31K cost=3666
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=50.06MB mem-reservation=8.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[1564310]
+|  23:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=01
+|  |  hash predicates: cs_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=0,1 row-size=36B cardinality=85.31K cost=85309
+|  |  in pipelines: 01(GETNEXT), 66(OPEN)
+|  |
+|  |--F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[18608]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF009[min_max] <- i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |
+|  |  67:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=8B cardinality=17.98K cost=633
+|  |  |  in pipelines: 66(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  66:AGGREGATE [FINALIZE]
+|  |  |  group by: i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=57 row-size=8B cardinality=17.98K cost=17975
+|  |  |  in pipelines: 66(GETNEXT), 64(OPEN)
+|  |  |
+|  |  65:EXCHANGE [HASH(i_item_sk)]
+|  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=8B cardinality=17.98K cost=211
+|  |  |  in pipelines: 64(GETNEXT)
+|  |  |
+|  |  F13:PLAN FRAGMENT [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=67.04MB mem-reservation=36.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[9542330, 235454, 211]
+|  |  22:AGGREGATE [STREAMING]
+|  |  |  group by: i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=57 row-size=8B cardinality=17.98K cost=235454
+|  |  |  in pipelines: 64(GETNEXT)
+|  |  |
+|  |  64:AGGREGATE [FINALIZE]
+|  |  |  output: count:merge(*)
+|  |  |  group by: substr(i_item_desc, 1, 30), i_item_sk, d_date
+|  |  |  having: count(*) > CAST(4 AS BIGINT)
+|  |  |  mem-estimate=46.41MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=5 row-size=50B cardinality=235.45K cost=9418164
+|  |  |  in pipelines: 64(GETNEXT), 03(OPEN)
+|  |  |
+|  |  63:EXCHANGE [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)]
+|  |  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=50B cardinality=2.35M cost=124166
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  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=33.63MB mem-reservation=17.50MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[14686864, 124166]
+|  |  08:AGGREGATE [STREAMING]
+|  |  |  output: count(*)
+|  |  |  group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=5 row-size=50B cardinality=2.35M cost=9418164
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  07:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: ss_item_sk = i_item_sk
+|  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=2,3,4 row-size=162B cardinality=2.35M cost=2354541
+|  |  |  in pipelines: 03(GETNEXT), 05(OPEN)
+|  |  |
+|  |  |--F34:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[24564]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
+|  |  |  |  build expressions: i_item_sk
+|  |  |  |  runtime filters: RF013[min_max] <- i_item_sk
+|  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  62:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=4 row-size=120B cardinality=18.00K cost=6564
+|  |  |  |  in pipelines: 05(GETNEXT)
+|  |  |  |
+|  |  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[4306]
+|  |  |  05:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |  |     tuple-ids=4 row-size=120B cardinality=18.00K cost=2118
+|  |  |     in pipelines: 05(GETNEXT)
+|  |  |
+|  |  06:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=03
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=2,3 row-size=42B cardinality=2.35M cost=2880404
+|  |  |  in pipelines: 03(GETNEXT), 04(OPEN)
+|  |  |
+|  |  |--F35:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=3 fragment-costs=[1641]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF014[bloom] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |
+|  |  |  61:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=3 row-size=30B cardinality=1.49K cost=150
+|  |  |  |  in pipelines: 04(GETNEXT)
+|  |  |  |
+|  |  |  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[75240]
+|  |  |  04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |  |     tuple-ids=3 row-size=30B cardinality=1.49K cost=75190
+|  |  |     in pipelines: 04(GETNEXT)
+|  |  |
+|  |  03:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF013[min_max] -> ss_item_sk, RF014[bloom] -> ss_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=2 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 03(GETNEXT)
+|  |
+|  21:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=04
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=0,1 row-size=36B cardinality=85.31K cost=1441548
+|  |  in pipelines: 01(GETNEXT), 02(OPEN)
+|  |
+|  |--F36:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  60:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=1 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 02(GETNEXT)
+|  |  |
+|  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=1 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 02(GETNEXT)
+|  |
+|  01:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     runtime filters: RF011[min_max] -> cs_sold_date_sk, RF009[min_max] -> cs_item_sk, RF010[bloom] -> cs_sold_date_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
+|     tuple-ids=0 row-size=24B cardinality=1.44M cost=33787
+|     in pipelines: 01(GETNEXT)
+|
+20:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  join table id: 05
+|  predicates: sum(ss_quantity * ss_sales_price) > CAST(0.500000 AS DECIMAL(10,6)) * max(csales)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=10,17 row-size=36B cardinality=100.00K cost=0
+|  in pipelines: 52(GETNEXT), 58(OPEN)
+|
+|--F37:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[3]
+|  JOIN BUILD
+|  |  join-table-id=05 plan-id=06 cohort-id=01
+|  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |
+|  59:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=17 row-size=16B cardinality=1 cost=3
+|  |  in pipelines: 58(GETNEXT)
+|  |
+|  F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[2, 1]
+|  58:AGGREGATE [FINALIZE]
+|  |  output: max:merge(csales)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=17 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 58(GETNEXT), 19(OPEN)
+|  |
+|  57:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=17 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 19(GETNEXT)
+|  |
+|  F06:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=2.88MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[202344, 100000, 1]
+|  19:AGGREGATE
+|  |  output: max(sum(ss_quantity * ss_sales_price))
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=17 row-size=16B cardinality=1 cost=100000
+|  |  in pipelines: 19(GETNEXT), 56(OPEN)
+|  |
+|  56:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  group by: c_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=15 row-size=20B cardinality=100.00K cost=200000
+|  |  in pipelines: 56(GETNEXT), 13(OPEN)
+|  |
+|  55:EXCHANGE [HASH(c_customer_sk)]
+|  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=15 row-size=20B cardinality=100.00K cost=2344
+|  |  in pipelines: 13(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=26.28MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[9989034, 2344]
+|  18:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  group by: c_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=15 row-size=20B cardinality=100.00K cost=4709082
+|  |  in pipelines: 13(GETNEXT)
+|  |
+|  17:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=06
+|  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=12,14,13 row-size=28B cardinality=2.35M cost=2354541
+|  |  in pipelines: 13(GETNEXT), 14(OPEN)
+|  |
+|  |--F38:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=20.39MB mem-reservation=20.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  JOIN BUILD
+|  |  |  join-table-id=06 plan-id=07 cohort-id=04
+|  |  |  build expressions: c_customer_sk
+|  |  |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |
+|  |  54:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=13 row-size=4B cardinality=100.00K cost=2346
+|  |  |  in pipelines: 14(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1173]
+|  |  14:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=13 row-size=4B cardinality=100.00K cost=391
+|  |     in pipelines: 14(GETNEXT)
+|  |
+|  16:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=07
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=12,14 row-size=24B cardinality=2.35M cost=2880404
+|  |  in pipelines: 13(GETNEXT), 15(OPEN)
+|  |
+|  |--F39:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[1545]
+|  |  JOIN BUILD
+|  |  |  join-table-id=07 plan-id=08 cohort-id=04
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF006[bloom] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |
+|  |  53:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=14 row-size=8B cardinality=1.49K cost=54
+|  |  |  in pipelines: 15(GETNEXT)
+|  |  |
+|  |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[73638]
+|  |  15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=14 row-size=8B cardinality=1.49K cost=73620
+|  |     in pipelines: 15(GETNEXT)
+|  |
+|  13:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF005[min_max] -> ss_customer_sk, RF006[bloom] -> ss_sold_date_sk, RF004[bloom] -> ss_customer_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|     tuple-ids=12 row-size=16B cardinality=2.88M cost=45007
+|     in pipelines: 13(GETNEXT)
+|
+52:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_quantity * ss_sales_price)
+|  group by: c_customer_sk
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10 row-size=20B cardinality=100.00K cost=200000
+|  in pipelines: 52(GETNEXT), 09(OPEN)
+|
+51:EXCHANGE [HASH(c_customer_sk)]
+|  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=10 row-size=20B cardinality=100.00K cost=2344
+|  in pipelines: 09(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=27.12MB mem-reservation=6.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[8674967, 2344]
+12:AGGREGATE [STREAMING]
+|  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  group by: c_customer_sk
+|  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=10 row-size=20B cardinality=100.00K cost=5760808
+|  in pipelines: 09(GETNEXT)
+|
+11:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=16
+|  hash predicates: ss_customer_sk = c_customer_sk
+|  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=8,9 row-size=16B cardinality=2.88M cost=2880404
+|  in pipelines: 09(GETNEXT), 10(OPEN)
+|
+|--F48:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=20.39MB mem-reservation=20.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[102346]
+|  JOIN BUILD
+|  |  join-table-id=16 plan-id=17 cohort-id=01
+|  |  build expressions: c_customer_sk
+|  |  runtime filters: RF002[bloom] <- c_customer_sk, RF003[min_max] <- c_customer_sk
+|  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |
+|  50:EXCHANGE [BROADCAST]
+|  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=9 row-size=4B cardinality=100.00K cost=2346
+|  |  in pipelines: 10(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1173]
+|  10:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|     HDFS partitions=1/1 files=1 size=5.49MB
+|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_sk
+|     stored statistics:
+|       table: rows=100.00K size=5.49MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=9 row-size=4B cardinality=100.00K cost=391
+|     in pipelines: 10(GETNEXT)
+|
+09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_customer_sk, RF003[min_max] -> ss_customer_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_customer_sk, RF002[bloom] -> ss_customer_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=8 row-size=12B cardinality=2.88M cost=33755
+   in pipelines: 09(GETNEXT)
+====
+# TPCDS-Q23b second of two queries
+WITH frequent_ss_items AS
+  (SELECT itemdesc,
+          i_item_sk item_sk,
+          d_date solddate,
+          count(*) cnt
+   FROM store_sales,
+        date_dim,
+     (SELECT SUBSTRING(i_item_desc, 1, 30) itemdesc,
+             *
+      FROM item) sq1
+   WHERE ss_sold_date_sk = d_date_sk
+     AND ss_item_sk = i_item_sk
+     AND d_year IN (2000,
+                    2000+1,
+                    2000+2,
+                    2000+3)
+   GROUP BY itemdesc,
+            i_item_sk,
+            d_date
+   HAVING count(*) >4),
+     max_store_sales AS
+  (SELECT max(csales) tpcds_cmax
+   FROM
+     (SELECT c_customer_sk,
+             sum(ss_quantity*ss_sales_price) csales
+      FROM store_sales,
+           customer,
+           date_dim
+      WHERE ss_customer_sk = c_customer_sk
+        AND ss_sold_date_sk = d_date_sk
+        AND d_year IN (2000,
+                       2000+1,
+                       2000+2,
+                       2000+3)
+      GROUP BY c_customer_sk) sq2),
+     best_ss_customer AS
+  (SELECT c_customer_sk,
+          sum(ss_quantity*ss_sales_price) ssales
+   FROM store_sales,
+        customer,
+        max_store_sales
+   WHERE ss_customer_sk = c_customer_sk
+   GROUP BY c_customer_sk
+   HAVING sum(ss_quantity*ss_sales_price) > (50/100.0) * max(tpcds_cmax))
+SELECT c_last_name,
+       c_first_name,
+       sales
+FROM
+  (SELECT c_last_name,
+          c_first_name,
+          sum(cs_quantity*cs_list_price) sales
+   FROM catalog_sales,
+        customer,
+        date_dim,
+        frequent_ss_items,
+        best_ss_customer
+   WHERE d_year = 2000
+     AND d_moy = 2
+     AND cs_sold_date_sk = d_date_sk
+     AND cs_item_sk = item_sk
+     AND cs_bill_customer_sk = best_ss_customer.c_customer_sk
+     AND cs_bill_customer_sk = customer.c_customer_sk
+   GROUP BY c_last_name,
+            c_first_name
+   UNION ALL SELECT c_last_name,
+                    c_first_name,
+                    sum(ws_quantity*ws_list_price) sales
+   FROM web_sales,
+        customer,
+        date_dim,
+        frequent_ss_items,
+        best_ss_customer
+   WHERE d_year = 2000
+     AND d_moy = 2
+     AND ws_sold_date_sk = d_date_sk
+     AND ws_item_sk = item_sk
+     AND ws_bill_customer_sk = best_ss_customer.c_customer_sk
+     AND ws_bill_customer_sk = customer.c_customer_sk
+   GROUP BY c_last_name,
+            c_first_name) sq3
+ORDER BY c_last_name,
+         c_first_name,
+         sales
+LIMIT 100;
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=623.75MB Threads=75
+Per-Host Resource Estimates: Memory=1.44GB
+F35:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[306]
+PLAN-ROOT SINK
+|  output exprs: c_last_name, c_first_name, sales
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=300
+|
+96:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: c_last_name ASC, c_first_name ASC, sales ASC
+|  limit: 100
+|  mem-estimate=67.28KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=44 row-size=52B cardinality=100 cost=6
+|  in pipelines: 53(GETNEXT)
+|
+F34:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Instance Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservation=1
+max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
+53:TOP-N [LIMIT=100]
+|  order by: c_last_name ASC, c_first_name ASC, sales ASC
+|  mem-estimate=5.08KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=44 row-size=52B cardinality=100 cost=300
+|  in pipelines: 53(GETNEXT), 74(OPEN), 95(OPEN)
+|
+00:UNION
+|  pass-through-operands: all
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=42 row-size=52B cardinality=20.15K cost=0
+|  in pipelines: 74(GETNEXT), 95(GETNEXT)
+|
+|--95:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ws_quantity * ws_list_price)
+|  |  group by: c_last_name, c_first_name
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=41 row-size=52B cardinality=9.63K cost=28878
+|  |  in pipelines: 95(GETNEXT), 78(OPEN)
+|  |
+|  94:EXCHANGE [HASH(c_last_name,c_first_name)]
+|  |  mem-estimate=330.72KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=41 row-size=52B cardinality=9.63K cost=527
+|  |  in pipelines: 78(GETNEXT)
+|  |
+|  F20:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=67.04MB mem-reservation=36.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[9542330, 264332, 527]
+|  52:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price)
+|  |  group by: c_last_name, c_first_name
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=41 row-size=52B cardinality=9.63K cost=28878
+|  |  in pipelines: 78(GETNEXT)
+|  |
+|  51:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=10
+|  |  hash predicates: i_item_sk = ws_item_sk
+|  |  fk/pk conjuncts: i_item_sk = ws_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=28,21,23,39,22 row-size=162B cardinality=9.63K cost=235454
+|  |  in pipelines: 78(GETNEXT), 27(OPEN)
+|  |
+|  |--F46:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=13.78MB mem-reservation=12.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[13235]
+|  |  JOIN BUILD
+|  |  |  join-table-id=10 plan-id=11 cohort-id=05
+|  |  |  build expressions: ws_item_sk
+|  |  |  runtime filters: RF018[bloom] <- ws_item_sk, RF019[min_max] <- ws_item_sk
+|  |  |  mem-estimate=11.50MB mem-reservation=11.50MB spill-buffer=128.00KB thread-reservation=0 cost=9626
+|  |  |
+|  |  93:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=1.28MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=21,23,39,22 row-size=112B cardinality=9.63K cost=3609
+|  |  |  in pipelines: 27(GETNEXT)
+|  |  |
+|  |  F32:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=1018.27KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[11619]
+|  |  50:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=11
+|  |  |  hash predicates: ws_bill_customer_sk = customer.c_customer_sk
+|  |  |  fk/pk conjuncts: ws_bill_customer_sk = customer.c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=21,23,39,22 row-size=112B cardinality=9.63K cost=9626
+|  |  |  in pipelines: 27(GETNEXT), 28(OPEN)
+|  |  |
+|  |  |--F47:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.60MB mem-reservation=4.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[104294]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=11 plan-id=12 cohort-id=06
+|  |  |  |  build expressions: customer.c_customer_sk
+|  |  |  |  runtime filters: RF025[min_max] <- customer.c_customer_sk
+|  |  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |
+|  |  |  92:EXCHANGE [HASH(customer.c_customer_sk)]
+|  |  |  |  mem-estimate=3.85MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=22 row-size=40B cardinality=100.00K cost=4294
+|  |  |  |  in pipelines: 28(GETNEXT)
+|  |  |  |
+|  |  |  F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.34MB mem-reservation=1.00MB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[8197]
+|  |  |  28:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |  |     tuple-ids=22 row-size=40B cardinality=100.00K cost=3903
+|  |  |     in pipelines: 28(GETNEXT)
+|  |  |
+|  |  91:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  |  mem-estimate=506.41KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=21,23,39 row-size=72B cardinality=9.63K cost=790
+|  |  |  in pipelines: 27(GETNEXT)
+|  |  |
+|  |  F21:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  |  Per-Instance Resources: mem-estimate=32.66MB mem-reservation=8.00MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[779889]
+|  |  49:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=12
+|  |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=21,23,39 row-size=72B cardinality=9.63K cost=42854
+|  |  |  in pipelines: 27(GETNEXT), 89(OPEN)
+|  |  |
+|  |  |--F48:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=9.21MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=2 fragment-costs=[10782]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=12 plan-id=13 cohort-id=06
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF026[bloom] <- c_customer_sk, RF027[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=10000
+|  |  |  |
+|  |  |  90:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=471.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=36B cardinality=10.00K cost=782
+|  |  |  |  in pipelines: 89(GETNEXT)
+|  |  |  |
+|  |  |  F30:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=11.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[303907, 391]
+|  |  |  89:AGGREGATE [FINALIZE]
+|  |  |  |  output: sum:merge(ss_quantity * ss_sales_price), max:merge(tpcds_cmax)
+|  |  |  |  group by: c_customer_sk
+|  |  |  |  having: sum(ss_quantity * ss_sales_price) > CAST(0.500000 AS DECIMAL(10,6)) * max(tpcds_cmax)
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=36B cardinality=10.00K cost=300000
+|  |  |  |  in pipelines: 89(GETNEXT), 36(OPEN)
+|  |  |  |
+|  |  |  88:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=1.61MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=36B cardinality=100.00K cost=3907
+|  |  |  |  in pipelines: 36(GETNEXT)
+|  |  |  |
+|  |  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Instance Resources: mem-estimate=26.47MB mem-reservation=10.00MB thread-reservation=1
+|  |  |  max-parallelism=12 fragment-costs=[11555371, 3907]
+|  |  |  47:AGGREGATE [STREAMING]
+|  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales))
+|  |  |  |  group by: c_customer_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=36B cardinality=100.00K cost=8641212
+|  |  |  |  in pipelines: 36(GETNEXT)
+|  |  |  |
+|  |  |  46:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
+|  |  |  |  join table id: 13
+|  |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=30,31,37 row-size=32B cardinality=2.88M cost=0
+|  |  |  |  in pipelines: 36(GETNEXT), 86(OPEN)
+|  |  |  |
+|  |  |  |--F49:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=13 plan-id=14 cohort-id=07
+|  |  |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |  |  |
+|  |  |  |  87:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=37 row-size=16B cardinality=1 cost=3
+|  |  |  |  |  in pipelines: 86(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F29:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  |  |  86:AGGREGATE [FINALIZE]
+|  |  |  |  |  output: max:merge(csales)
+|  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  |  tuple-ids=37 row-size=16B cardinality=1 cost=1
+|  |  |  |  |  in pipelines: 86(GETNEXT), 44(OPEN)
+|  |  |  |  |
+|  |  |  |  85:EXCHANGE [UNPARTITIONED]
+|  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=37 row-size=16B cardinality=1 cost=1
+|  |  |  |  |  in pipelines: 44(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F28:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[202344, 100000, 1]
+|  |  |  |  44:AGGREGATE
+|  |  |  |  |  output: max(sum(ss_quantity * ss_sales_price))
+|  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  |  tuple-ids=37 row-size=16B cardinality=1 cost=100000
+|  |  |  |  |  in pipelines: 44(GETNEXT), 84(OPEN)
+|  |  |  |  |
+|  |  |  |  84:AGGREGATE [FINALIZE]
+|  |  |  |  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  |  |  |  group by: c_customer_sk
+|  |  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=35 row-size=20B cardinality=100.00K cost=200000
+|  |  |  |  |  in pipelines: 84(GETNEXT), 38(OPEN)
+|  |  |  |  |
+|  |  |  |  83:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=35 row-size=20B cardinality=100.00K cost=2344
+|  |  |  |  |  in pipelines: 38(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  |  |  Per-Instance Resources: mem-estimate=26.28MB mem-reservation=6.00MB thread-reservation=1
+|  |  |  |  max-parallelism=12 fragment-costs=[9989034, 2344]
+|  |  |  |  43:AGGREGATE [STREAMING]
+|  |  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  |  |  |  group by: c_customer_sk
+|  |  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=35 row-size=20B cardinality=100.00K cost=4709082
+|  |  |  |  |  in pipelines: 38(GETNEXT)
+|  |  |  |  |
+|  |  |  |  42:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  |  hash-table-id=14
+|  |  |  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=32,34,33 row-size=28B cardinality=2.35M cost=2354541
+|  |  |  |  |  in pipelines: 38(GETNEXT), 39(OPEN)
+|  |  |  |  |
+|  |  |  |  |--F50:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  |  |  JOIN BUILD
+|  |  |  |  |  |  join-table-id=14 plan-id=15 cohort-id=08
+|  |  |  |  |  |  build expressions: c_customer_sk
+|  |  |  |  |  |  runtime filters: RF033[min_max] <- c_customer_sk
+|  |  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  |  |
+|  |  |  |  |  82:EXCHANGE [BROADCAST]
+|  |  |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  |  tuple-ids=33 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  |  |  in pipelines: 39(GETNEXT)
+|  |  |  |  |  |
+|  |  |  |  |  F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  |  |  39:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |  |  |     stored statistics:
+|  |  |  |  |       table: rows=100.00K size=5.49MB
+|  |  |  |  |       columns: all
+|  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |  |     tuple-ids=33 row-size=4B cardinality=100.00K cost=391
+|  |  |  |  |     in pipelines: 39(GETNEXT)
+|  |  |  |  |
+|  |  |  |  41:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  |  hash-table-id=15
+|  |  |  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=32,34 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  |  |  in pipelines: 38(GETNEXT), 40(OPEN)
+|  |  |  |  |
+|  |  |  |  |--F51:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  |  max-parallelism=3 fragment-costs=[1545]
+|  |  |  |  |  JOIN BUILD
+|  |  |  |  |  |  join-table-id=15 plan-id=16 cohort-id=08
+|  |  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  |  runtime filters: RF034[bloom] <- d_date_sk
+|  |  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |  |
+|  |  |  |  |  81:EXCHANGE [BROADCAST]
+|  |  |  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  |  tuple-ids=34 row-size=8B cardinality=1.49K cost=54
+|  |  |  |  |  |  in pipelines: 40(GETNEXT)
+|  |  |  |  |  |
+|  |  |  |  |  F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  |  max-parallelism=1 fragment-costs=[73638]
+|  |  |  |  |  40:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |  |     stored statistics:
+|  |  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |  |       columns: all
+|  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |  |     tuple-ids=34 row-size=8B cardinality=1.49K cost=73620
+|  |  |  |  |     in pipelines: 40(GETNEXT)
+|  |  |  |  |
+|  |  |  |  38:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |  |  |     runtime filters: RF033[min_max] -> ss_customer_sk, RF034[bloom] -> ss_sold_date_sk
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=2.88M size=200.96MB
+|  |  |  |       partitions: 1824/1824 rows=2.88M
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |  |  |     tuple-ids=32 row-size=16B cardinality=2.88M cost=45007
+|  |  |  |     in pipelines: 38(GETNEXT)
+|  |  |  |
+|  |  |  45:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=16
+|  |  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=30,31 row-size=16B cardinality=2.88M cost=2880404
+|  |  |  |  in pipelines: 36(GETNEXT), 37(OPEN)
+|  |  |  |
+|  |  |  |--F52:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=16 plan-id=17 cohort-id=07
+|  |  |  |  |  build expressions: c_customer_sk
+|  |  |  |  |  runtime filters: RF031[min_max] <- c_customer_sk
+|  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  |
+|  |  |  |  80:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=31 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  |  in pipelines: 37(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  |  37:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |  |     runtime filters: RF025[min_max] -> tpcds_parquet.customer.c_customer_sk
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=100.00K size=5.49MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=31 row-size=4B cardinality=100.00K cost=391
+|  |  |  |     in pipelines: 37(GETNEXT)
+|  |  |  |
+|  |  |  36:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |  |     runtime filters: RF025[min_max] -> tpcds_parquet.store_sales.ss_customer_sk, RF031[min_max] -> ss_customer_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=2.88M size=200.96MB
+|  |  |       partitions: 1824/1824 rows=2.88M
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |  |     tuple-ids=30 row-size=12B cardinality=2.88M cost=33755
+|  |  |     in pipelines: 36(GETNEXT)
+|  |  |
+|  |  48:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=17
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=21,23 row-size=36B cardinality=42.85K cost=719384
+|  |  |  in pipelines: 27(GETNEXT), 29(OPEN)
+|  |  |
+|  |  |--F53:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=2 fragment-costs=[112]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=17 plan-id=18 cohort-id=06
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF028[bloom] <- d_date_sk, RF029[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  |
+|  |  |  79:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=23 row-size=12B cardinality=108 cost=4
+|  |  |  |  in pipelines: 29(GETNEXT)
+|  |  |  |
+|  |  |  F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146957]
+|  |  |  29:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=23 row-size=12B cardinality=108 cost=146955
+|  |  |     in pipelines: 29(GETNEXT)
+|  |  |
+|  |  27:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF029[min_max] -> ws_sold_date_sk, RF027[min_max] -> ws_bill_customer_sk, RF025[min_max] -> ws_bill_customer_sk, RF028[bloom] -> ws_sold_date_sk, RF026[bloom] -> ws_bill_customer_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
+|  |     tuple-ids=21 row-size=24B cardinality=719.38K cost=16861
+|  |     in pipelines: 27(GETNEXT)
+|  |
+|  78:AGGREGATE [FINALIZE]
+|  |  output: count:merge(*)
+|  |  group by: itemdesc, i_item_sk, d_date
+|  |  having: count(*) > CAST(4 AS BIGINT)
+|  |  mem-estimate=46.41MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=28 row-size=50B cardinality=235.45K cost=9418164
+|  |  in pipelines: 78(GETNEXT), 30(OPEN)
+|  |
+|  77:EXCHANGE [HASH(itemdesc,i_item_sk,d_date)]
+|  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=28 row-size=50B cardinality=2.35M cost=124166
+|  |  in pipelines: 30(GETNEXT)
+|  |
+|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=33.63MB mem-reservation=17.50MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[14686864, 124166]
+|  35:AGGREGATE [STREAMING]
+|  |  output: count(*)
+|  |  group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_parquet.item.i_item_sk, d_date
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=28 row-size=50B cardinality=2.35M cost=9418164
+|  |  in pipelines: 30(GETNEXT)
+|  |
+|  34:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=18
+|  |  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  |  fk/pk conjuncts: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=24,25,26 row-size=162B cardinality=2.35M cost=2354541
+|  |  in pipelines: 30(GETNEXT), 32(OPEN)
+|  |
+|  |--F54:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[24564]
+|  |  JOIN BUILD
+|  |  |  join-table-id=18 plan-id=19 cohort-id=05
+|  |  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  |  runtime filters: RF021[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  76:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=26 row-size=120B cardinality=18.00K cost=6564
+|  |  |  in pipelines: 32(GETNEXT)
+|  |  |
+|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[4306]
+|  |  32:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF019[min_max] -> tpcds_parquet.item.i_item_sk, RF018[bloom] -> tpcds_parquet.item.i_item_sk
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |     tuple-ids=26 row-size=120B cardinality=18.00K cost=2118
+|  |     in pipelines: 32(GETNEXT)
+|  |
+|  33:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=19
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=24,25 row-size=42B cardinality=2.35M cost=2880404
+|  |  in pipelines: 30(GETNEXT), 31(OPEN)
+|  |
+|  |--F55:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[1641]
+|  |  JOIN BUILD
+|  |  |  join-table-id=19 plan-id=20 cohort-id=05
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF022[bloom] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |
+|  |  75:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=25 row-size=30B cardinality=1.49K cost=150
+|  |  |  in pipelines: 31(GETNEXT)
+|  |  |
+|  |  F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[75240]
+|  |  31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |     tuple-ids=25 row-size=30B cardinality=1.49K cost=75190
+|  |     in pipelines: 31(GETNEXT)
+|  |
+|  30:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF019[min_max] -> tpcds_parquet.store_sales.ss_item_sk, RF021[min_max] -> ss_item_sk, RF018[bloom] -> tpcds_parquet.store_sales.ss_item_sk, RF022[bloom] -> ss_sold_date_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=24 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 30(GETNEXT)
+|
+74:AGGREGATE [FINALIZE]
+|  output: sum:merge(cs_quantity * cs_list_price)
+|  group by: c_last_name, c_first_name
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=20 row-size=52B cardinality=10.53K cost=31578
+|  in pipelines: 74(GETNEXT), 57(OPEN)
+|
+73:EXCHANGE [HASH(c_last_name,c_first_name)]
+|  mem-estimate=345.95KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=20 row-size=52B cardinality=10.53K cost=576
+|  in pipelines: 57(GETNEXT)
+|
+F03:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=67.04MB mem-reservation=36.00MB thread-reservation=1
+max-parallelism=3 fragment-costs=[9542330, 267032, 576]
+26:AGGREGATE [STREAMING]
+|  output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price)
+|  group by: c_last_name, c_first_name
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=20 row-size=52B cardinality=10.53K cost=31578
+|  in pipelines: 57(GETNEXT)
+|
+25:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=00
+|  hash predicates: i_item_sk = cs_item_sk
+|  fk/pk conjuncts: i_item_sk = cs_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=7,0,2,18,1 row-size=162B cardinality=10.53K cost=235454
+|  in pipelines: 57(GETNEXT), 01(OPEN)
+|
+|--F36:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=12.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[14474]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: cs_item_sk
+|  |  runtime filters: RF000[bloom] <- cs_item_sk, RF001[min_max] <- cs_item_sk
+|  |  mem-estimate=11.50MB mem-reservation=11.50MB spill-buffer=128.00KB thread-reservation=0 cost=10526
+|  |
+|  72:EXCHANGE [BROADCAST]
+|  |  mem-estimate=1.50MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,18,1 row-size=112B cardinality=10.53K cost=3948
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F15:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+|  Per-Instance Resources: mem-estimate=1010.56KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=3 fragment-costs=[12706]
+|  24:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=01
+|  |  hash predicates: cs_bill_customer_sk = customer.c_customer_sk
+|  |  fk/pk conjuncts: cs_bill_customer_sk = customer.c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=0,2,18,1 row-size=112B cardinality=10.53K cost=10526
+|  |  in pipelines: 01(GETNEXT), 02(OPEN)
+|  |
+|  |--F37:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.60MB mem-reservation=4.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[104294]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  build expressions: customer.c_customer_sk
+|  |  |  runtime filters: RF007[min_max] <- customer.c_customer_sk
+|  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |
+|  |  71:EXCHANGE [HASH(customer.c_customer_sk)]
+|  |  |  mem-estimate=3.85MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=1 row-size=40B cardinality=100.00K cost=4294
+|  |  |  in pipelines: 02(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.52MB mem-reservation=1.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[8197]
+|  |  02:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=1 row-size=40B cardinality=100.00K cost=3903
+|  |     in pipelines: 02(GETNEXT)
+|  |
+|  70:EXCHANGE [HASH(cs_bill_customer_sk)]
+|  |  mem-estimate=498.70KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,18 row-size=72B cardinality=10.53K cost=864
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=48.98MB mem-reservation=8.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[1561508]
+|  23:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=02
+|  |  hash predicates: cs_bill_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=0,2,18 row-size=72B cardinality=10.53K cost=85309
+|  |  in pipelines: 01(GETNEXT), 68(OPEN)
+|  |
+|  |--F38:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=9.21MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[11173]
+|  |  JOIN BUILD
+|  |  |  join-table-id=02 plan-id=03 cohort-id=02
+|  |  |  build expressions: c_customer_sk
+|  |  |  runtime filters: RF008[bloom] <- c_customer_sk, RF009[min_max] <- c_customer_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=10000
+|  |  |
+|  |  69:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=471.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=36B cardinality=10.00K cost=1173
+|  |  |  in pipelines: 68(GETNEXT)
+|  |  |
+|  |  F13:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=11.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[303907, 391]
+|  |  68:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge(ss_quantity * ss_sales_price), max:merge(tpcds_cmax)
+|  |  |  group by: c_customer_sk
+|  |  |  having: sum(ss_quantity * ss_sales_price) > CAST(0.500000 AS DECIMAL(10,6)) * max(tpcds_cmax)
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=18 row-size=36B cardinality=10.00K cost=300000
+|  |  |  in pipelines: 68(GETNEXT), 10(OPEN)
+|  |  |
+|  |  67:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  mem-estimate=1.61MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=36B cardinality=100.00K cost=3907
+|  |  |  in pipelines: 10(GETNEXT)
+|  |  |
+|  |  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=26.47MB mem-reservation=10.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[11555371, 3907]
+|  |  21:AGGREGATE [STREAMING]
+|  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales))
+|  |  |  group by: c_customer_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  tuple-ids=18 row-size=36B cardinality=100.00K cost=8641212
+|  |  |  in pipelines: 10(GETNEXT)
+|  |  |
+|  |  20:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
+|  |  |  join table id: 03
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=9,10,16 row-size=32B cardinality=2.88M cost=0
+|  |  |  in pipelines: 10(GETNEXT), 65(OPEN)
+|  |  |
+|  |  |--F39:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
+|  |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |  |
+|  |  |  66:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=16 row-size=16B cardinality=1 cost=3
+|  |  |  |  in pipelines: 65(GETNEXT)
+|  |  |  |
+|  |  |  F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  |  65:AGGREGATE [FINALIZE]
+|  |  |  |  output: max:merge(csales)
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  tuple-ids=16 row-size=16B cardinality=1 cost=1
+|  |  |  |  in pipelines: 65(GETNEXT), 18(OPEN)
+|  |  |  |
+|  |  |  64:EXCHANGE [UNPARTITIONED]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=16 row-size=16B cardinality=1 cost=1
+|  |  |  |  in pipelines: 18(GETNEXT)
+|  |  |  |
+|  |  |  F11:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[202344, 100000, 1]
+|  |  |  18:AGGREGATE
+|  |  |  |  output: max(sum(ss_quantity * ss_sales_price))
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  tuple-ids=16 row-size=16B cardinality=1 cost=100000
+|  |  |  |  in pipelines: 18(GETNEXT), 63(OPEN)
+|  |  |  |
+|  |  |  63:AGGREGATE [FINALIZE]
+|  |  |  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  |  |  group by: c_customer_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=100.00K cost=200000
+|  |  |  |  in pipelines: 63(GETNEXT), 12(OPEN)
+|  |  |  |
+|  |  |  62:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=100.00K cost=2344
+|  |  |  |  in pipelines: 12(GETNEXT)
+|  |  |  |
+|  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=26.28MB mem-reservation=6.00MB thread-reservation=1
+|  |  |  max-parallelism=12 fragment-costs=[9989034, 2344]
+|  |  |  17:AGGREGATE [STREAMING]
+|  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  |  |  group by: c_customer_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=100.00K cost=4709082
+|  |  |  |  in pipelines: 12(GETNEXT)
+|  |  |  |
+|  |  |  16:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=04
+|  |  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=11,13,12 row-size=28B cardinality=2.35M cost=2354541
+|  |  |  |  in pipelines: 12(GETNEXT), 13(OPEN)
+|  |  |  |
+|  |  |  |--F40:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=04 plan-id=05 cohort-id=04
+|  |  |  |  |  build expressions: c_customer_sk
+|  |  |  |  |  runtime filters: RF015[min_max] <- c_customer_sk
+|  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  |
+|  |  |  |  61:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=12 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  |  in pipelines: 13(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  |  13:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=100.00K size=5.49MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=12 row-size=4B cardinality=100.00K cost=391
+|  |  |  |     in pipelines: 13(GETNEXT)
+|  |  |  |
+|  |  |  15:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=05
+|  |  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=11,13 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  |  in pipelines: 12(GETNEXT), 14(OPEN)
+|  |  |  |
+|  |  |  |--F41:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  max-parallelism=3 fragment-costs=[1545]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=05 plan-id=06 cohort-id=04
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF016[bloom] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |
+|  |  |  |  60:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=13 row-size=8B cardinality=1.49K cost=54
+|  |  |  |  |  in pipelines: 14(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[73638]
+|  |  |  |  14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=13 row-size=8B cardinality=1.49K cost=73620
+|  |  |  |     in pipelines: 14(GETNEXT)
+|  |  |  |
+|  |  |  12:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |  |     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=2.88M size=200.96MB
+|  |  |       partitions: 1824/1824 rows=2.88M
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |  |     tuple-ids=11 row-size=16B cardinality=2.88M cost=45007
+|  |  |     in pipelines: 12(GETNEXT)
+|  |  |
+|  |  19:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=06
+|  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10 row-size=16B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 10(GETNEXT), 11(OPEN)
+|  |  |
+|  |  |--F42:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=06 plan-id=07 cohort-id=03
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF013[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |
+|  |  |  59:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=10 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  in pipelines: 11(GETNEXT)
+|  |  |  |
+|  |  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  11:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF007[min_max] -> tpcds_parquet.customer.c_customer_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=10 row-size=4B cardinality=100.00K cost=391
+|  |  |     in pipelines: 11(GETNEXT)
+|  |  |
+|  |  10:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF007[min_max] -> tpcds_parquet.store_sales.ss_customer_sk, RF013[min_max] -> ss_customer_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=9 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 10(GETNEXT)
+|  |
+|  22:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=07
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=0,2 row-size=36B cardinality=85.31K cost=1441548
+|  |  in pipelines: 01(GETNEXT), 03(OPEN)
+|  |
+|  |--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=07 plan-id=08 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  58:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=2 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=2 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 03(GETNEXT)
+|  |
+|  01:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     runtime filters: RF011[min_max] -> cs_sold_date_sk, RF009[min_max] -> cs_bill_customer_sk, RF007[min_max] -> cs_bill_customer_sk, RF010[bloom] -> cs_sold_date_sk, RF008[bloom] -> cs_bill_customer_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
+|     tuple-ids=0 row-size=24B cardinality=1.44M cost=33787
+|     in pipelines: 01(GETNEXT)
+|
+57:AGGREGATE [FINALIZE]
+|  output: count:merge(*)
+|  group by: itemdesc, i_item_sk, d_date
+|  having: count(*) > CAST(4 AS BIGINT)
+|  mem-estimate=46.41MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=7 row-size=50B cardinality=235.45K cost=9418164
+|  in pipelines: 57(GETNEXT), 04(OPEN)
+|
+56:EXCHANGE [HASH(itemdesc,i_item_sk,d_date)]
+|  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=7 row-size=50B cardinality=2.35M cost=124166
+|  in pipelines: 04(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=33.63MB mem-reservation=17.50MB thread-reservation=1
+max-parallelism=12 fragment-costs=[14686864, 124166]
+09:AGGREGATE [STREAMING]
+|  output: count(*)
+|  group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_parquet.item.i_item_sk, d_date
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=7 row-size=50B cardinality=2.35M cost=9418164
+|  in pipelines: 04(GETNEXT)
+|
+08:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=08
+|  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  fk/pk conjuncts: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=3,4,5 row-size=162B cardinality=2.35M cost=2354541
+|  in pipelines: 04(GETNEXT), 06(OPEN)
+|
+|--F44:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[24564]
+|  JOIN BUILD
+|  |  join-table-id=08 plan-id=09 cohort-id=01
+|  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  runtime filters: RF003[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |
+|  55:EXCHANGE [BROADCAST]
+|  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5 row-size=120B cardinality=18.00K cost=6564
+|  |  in pipelines: 06(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[4306]
+|  06:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|     tuple-ids=5 row-size=120B cardinality=18.00K cost=2118
+|     in pipelines: 06(GETNEXT)
+|
+07:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=09
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3,4 row-size=42B cardinality=2.35M cost=2880404
+|  in pipelines: 04(GETNEXT), 05(OPEN)
+|
+|--F45:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[1641]
+|  JOIN BUILD
+|  |  join-table-id=09 plan-id=10 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF004[bloom] <- d_date_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |
+|  54:EXCHANGE [BROADCAST]
+|  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=4 row-size=30B cardinality=1.49K cost=150
+|  |  in pipelines: 05(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[75240]
+|  05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|     tuple-ids=4 row-size=30B cardinality=1.49K cost=75190
+|     in pipelines: 05(GETNEXT)
+|
+04:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_item_sk, RF003[min_max] -> ss_item_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_item_sk, RF004[bloom] -> ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+   tuple-ids=3 row-size=12B cardinality=2.88M cost=33755
+   in pipelines: 04(GETNEXT)
+====
+# TPCDS-Q49
+SELECT channel,
+       item,
+       return_ratio,
+       return_rank,
+       currency_rank
+FROM
+  (SELECT 'web' AS channel,
+          web.item,
+          web.return_ratio,
+          web.return_rank,
+          web.currency_rank
+   FROM
+     (SELECT item,
+             return_ratio,
+             currency_ratio,
+             rank() OVER (
+                          ORDER BY return_ratio) AS return_rank,
+                         rank() OVER (
+                                      ORDER BY currency_ratio) AS currency_rank
+      FROM
+        (SELECT ws.ws_item_sk AS item,
+                (cast(sum(coalesce(wr.wr_return_quantity,0)) AS decimal(15,4))/ cast(sum(coalesce(ws.ws_quantity,0)) AS decimal(15,4))) AS return_ratio,
+                (cast(sum(coalesce(wr.wr_return_amt,0)) AS decimal(15,4))/ cast(sum(coalesce(ws.ws_net_paid,0)) AS decimal(15,4))) AS currency_ratio
+         FROM web_sales ws
+         LEFT OUTER JOIN web_returns wr ON (ws.ws_order_number = wr.wr_order_number
+                                            AND ws.ws_item_sk = wr.wr_item_sk) ,date_dim
+         WHERE wr.wr_return_amt > 10000
+           AND ws.ws_net_profit > 1
+           AND ws.ws_net_paid > 0
+           AND ws.ws_quantity > 0
+           AND ws_sold_date_sk = d_date_sk
+           AND d_year = 2001
+           AND d_moy = 12
+         GROUP BY ws.ws_item_sk) in_web) web
+   WHERE (web.return_rank <= 10
+          OR web.currency_rank <= 10)
+   UNION SELECT 'catalog' AS channel,
+                catalog.item,
+                catalog.return_ratio,
+                catalog.return_rank,
+                catalog.currency_rank
+   FROM
+     (SELECT item,
+             return_ratio,
+             currency_ratio,
+             rank() OVER (
+                          ORDER BY return_ratio) AS return_rank,
+                         rank() OVER (
+                                      ORDER BY currency_ratio) AS currency_rank
+      FROM
+        (SELECT cs.cs_item_sk AS item,
+                (cast(sum(coalesce(cr.cr_return_quantity,0)) AS decimal(15,4))/ cast(sum(coalesce(cs.cs_quantity,0)) AS decimal(15,4))) AS return_ratio,
+                (cast(sum(coalesce(cr.cr_return_amount,0)) AS decimal(15,4))/ cast(sum(coalesce(cs.cs_net_paid,0)) AS decimal(15,4))) AS currency_ratio
+         FROM catalog_sales cs
+         LEFT OUTER JOIN catalog_returns cr ON (cs.cs_order_number = cr.cr_order_number
+                                                AND cs.cs_item_sk = cr.cr_item_sk) ,date_dim
+         WHERE cr.cr_return_amount > 10000
+           AND cs.cs_net_profit > 1
+           AND cs.cs_net_paid > 0
+           AND cs.cs_quantity > 0
+           AND cs_sold_date_sk = d_date_sk
+           AND d_year = 2001
+           AND d_moy = 12
+         GROUP BY cs.cs_item_sk) in_cat) CATALOG
+   WHERE (catalog.return_rank <= 10
+          OR catalog.currency_rank <=10)
+   UNION SELECT 'store' AS channel,
+                store.item,
+                store.return_ratio,
+                store.return_rank,
+                store.currency_rank
+   FROM
+     (SELECT item,
+             return_ratio,
+             currency_ratio,
+             rank() OVER (
+                          ORDER BY return_ratio) AS return_rank,
+                         rank() OVER (
+                                      ORDER BY currency_ratio) AS currency_rank
+      FROM
+        (SELECT sts.ss_item_sk AS item,
+                (cast(sum(coalesce(sr.sr_return_quantity,0)) AS decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) AS decimal(15,4))) AS return_ratio,
+                (cast(sum(coalesce(sr.sr_return_amt,0)) AS decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) AS decimal(15,4))) AS currency_ratio
+         FROM store_sales sts
+         LEFT OUTER JOIN store_returns sr ON (sts.ss_ticket_number = sr.sr_ticket_number
+                                              AND sts.ss_item_sk = sr.sr_item_sk) ,date_dim
+         WHERE sr.sr_return_amt > 10000
+           AND sts.ss_net_profit > 1
+           AND sts.ss_net_paid > 0
+           AND sts.ss_quantity > 0
+           AND ss_sold_date_sk = d_date_sk
+           AND d_year = 2001
+           AND d_moy = 12
+         GROUP BY sts.ss_item_sk) in_store) store
+   WHERE (store.return_rank <= 10
+          OR store.currency_rank <= 10) ) sq1
+ORDER BY 1,
+         4,
+         5,
+         2
+LIMIT 100;
+
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=167.50MB Threads=22
+Per-Host Resource Estimates: Memory=451MB
+F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=24.61MB mem-reservation=15.94MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[8891, 17696, 35376, 119504, 400, 500]
+PLAN-ROOT SINK
+|  output exprs: channel, item, return_ratio, return_rank, currency_rank
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=500
+|
+35:TOP-N [LIMIT=100]
+|  order by: channel ASC, return_rank ASC, currency_rank ASC, item ASC
+|  mem-estimate=5.08KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=26 row-size=52B cardinality=100 cost=400
+|  in pipelines: 35(GETNEXT), 34(OPEN)
+|
+34:AGGREGATE [FINALIZE]
+|  group by: channel, item, return_ratio, return_rank, currency_rank
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=24 row-size=52B cardinality=2.99K cost=14940
+|  in pipelines: 34(GETNEXT), 09(OPEN), 20(OPEN), 31(OPEN)
+|
+00:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=24 row-size=52B cardinality=2.99K cost=44820
+|  in pipelines: 09(GETNEXT), 20(GETNEXT), 31(GETNEXT)
+|
+|--33:SELECT
+|  |  predicates: (rank() <= CAST(10 AS BIGINT) OR rank() <= CAST(10 AS BIGINT))
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=58,55 row-size=104B cardinality=1.71K cost=17055
+|  |  in pipelines: 31(GETNEXT)
+|  |
+|  32:ANALYTIC
+|  |  functions: rank()
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=58,55 row-size=104B cardinality=17.05K cost=17055
+|  |  in pipelines: 31(GETNEXT)
+|  |
+|  31:SORT
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  |  materialized: (CAST(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15,4)))
+|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=58 row-size=96B cardinality=17.05K cost=17055
+|  |  in pipelines: 31(GETNEXT), 29(OPEN)
+|  |
+|  30:ANALYTIC
+|  |  functions: rank()
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=56,53 row-size=80B cardinality=17.05K cost=17055
+|  |  in pipelines: 29(GETNEXT)
+|  |
+|  50:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  mem-estimate=627.73KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=56 row-size=72B cardinality=17.05K cost=1266
+|  |  in pipelines: 29(GETNEXT)
+|  |
+|  F13:PLAN FRAGMENT [HASH(sts.ss_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[86275, 17055, 1266]
+|  29:SORT
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  materialized: (CAST(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15,4)))
+|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=56 row-size=72B cardinality=17.05K cost=17055
+|  |  in pipelines: 29(GETNEXT), 49(OPEN)
+|  |
+|  49:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(coalesce(sr.sr_return_quantity, 0)), sum:merge(coalesce(sts.ss_quantity, 0)), sum:merge(coalesce(sr.sr_return_amt, 0)), sum:merge(coalesce(sts.ss_net_paid, 0))
+|  |  group by: sts.ss_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=19 row-size=56B cardinality=17.05K cost=85275
+|  |  in pipelines: 49(GETNEXT), 23(OPEN)
+|  |
+|  48:EXCHANGE [HASH(sts.ss_item_sk)]
+|  |  mem-estimate=1.01MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=19 row-size=56B cardinality=17.05K cost=1000
+|  |  in pipelines: 23(GETNEXT)
+|  |
+|  F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=26.70MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[9680620, 1000]
+|  28:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(coalesce(sr.sr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(sts.ss_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(sr.sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(sts.ss_net_paid, CAST(0 AS DECIMAL(7,2))))
+|  |  group by: sts.ss_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=19 row-size=56B cardinality=17.05K cost=85275
+|  |  in pipelines: 23(GETNEXT)
+|  |
+|  27:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=04
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=16,17N,18 row-size=68B cardinality=17.05K cost=288040
+|  |  in pipelines: 23(GETNEXT), 25(OPEN)
+|  |
+|  |--F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=03
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF004[bloom] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  47:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 25(GETNEXT)
+|  |  |
+|  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=18 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 25(GETNEXT)
+|  |
+|  26:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  |  hash-table-id=05
+|  |  hash predicates: sts.ss_item_sk = sr.sr_item_sk, sts.ss_ticket_number = sr.sr_ticket_number
+|  |  fk/pk conjuncts: sts.ss_item_sk = sr.sr_item_sk, sts.ss_ticket_number = sr.sr_ticket_number
+|  |  other predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=16,17N row-size=56B cardinality=288.04K cost=576080
+|  |  in pipelines: 23(GETNEXT), 24(OPEN)
+|  |
+|  |--F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=12.19MB mem-reservation=11.50MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[59863]
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=03
+|  |  |  build expressions: sr.sr_item_sk, sr.sr_ticket_number
+|  |  |  mem-estimate=11.50MB mem-reservation=11.50MB spill-buffer=128.00KB thread-reservation=0 cost=57502
+|  |  |
+|  |  46:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=701.85KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=17 row-size=24B cardinality=28.75K cost=2361
+|  |  |  in pipelines: 24(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[295040]
+|  |  24:SCAN HDFS [tpcds_parquet.store_returns sr, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=15.43MB
+|  |     predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  |     stored statistics:
+|  |       table: rows=287.51K size=15.43MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|  |     parquet statistics predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  |     parquet dictionary predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  |     mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=17 row-size=24B cardinality=28.75K cost=294253
+|  |     in pipelines: 24(GETNEXT)
+|  |
+|  23:SCAN HDFS [tpcds_parquet.store_sales sts, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=199.44MB
+|     predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
+|     runtime filters: RF004[bloom] -> ss_sold_date_sk
+|     stored statistics:
+|       table: rows=2.88M size=199.44MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     parquet statistics predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
+|     parquet dictionary predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
+|     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
+|     tuple-ids=16 row-size=32B cardinality=288.04K cost=8731225
+|     in pipelines: 23(GETNEXT)
+|
+|--22:SELECT
+|  |  predicates: (rank() <= CAST(10 AS BIGINT) OR rank() <= CAST(10 AS BIGINT))
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=47,44 row-size=104B cardinality=853 cost=8531
+|  |  in pipelines: 20(GETNEXT)
+|  |
+|  21:ANALYTIC
+|  |  functions: rank()
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=47,44 row-size=104B cardinality=8.53K cost=8531
+|  |  in pipelines: 20(GETNEXT)
+|  |
+|  20:SORT
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  |  materialized: (CAST(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15,4)))
+|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=47 row-size=96B cardinality=8.53K cost=8531
+|  |  in pipelines: 20(GETNEXT), 18(OPEN)
+|  |
+|  19:ANALYTIC
+|  |  functions: rank()
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=45,42 row-size=80B cardinality=8.53K cost=8531
+|  |  in pipelines: 18(GETNEXT)
+|  |
+|  45:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  mem-estimate=427.95KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=45 row-size=72B cardinality=8.53K cost=634
+|  |  in pipelines: 18(GETNEXT)
+|  |
+|  F08:PLAN FRAGMENT [HASH(cs.cs_item_sk)] hosts=3 instances=3
+|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[43155, 8531, 634]
+|  18:SORT
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  materialized: (CAST(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15,4)))
+|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=45 row-size=72B cardinality=8.53K cost=8531
+|  |  in pipelines: 18(GETNEXT), 44(OPEN)
+|  |
+|  44:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(coalesce(cr.cr_return_quantity, 0)), sum:merge(coalesce(cs.cs_quantity, 0)), sum:merge(coalesce(cr.cr_return_amount, 0)), sum:merge(coalesce(cs.cs_net_paid, 0))
+|  |  group by: cs.cs_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=11 row-size=56B cardinality=8.53K cost=42655
+|  |  in pipelines: 44(GETNEXT), 12(OPEN)
+|  |
+|  43:EXCHANGE [HASH(cs.cs_item_sk)]
+|  |  mem-estimate=335.51KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=11 row-size=56B cardinality=8.53K cost=500
+|  |  in pipelines: 12(GETNEXT)
+|  |
+|  F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=58.70MB mem-reservation=18.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[4844813, 500]
+|  17:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(coalesce(cr.cr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(cs.cs_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(cr.cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(cs.cs_net_paid, CAST(0 AS DECIMAL(7,2))))
+|  |  group by: cs.cs_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=11 row-size=56B cardinality=8.53K cost=42655
+|  |  in pipelines: 12(GETNEXT)
+|  |
+|  16:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=02
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=8,9N,10 row-size=68B cardinality=8.53K cost=144155
+|  |  in pipelines: 12(GETNEXT), 14(OPEN)
+|  |
+|  |--F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=02 plan-id=03 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF002[bloom] <- d_date_sk, RF003[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  42:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=10 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 14(GETNEXT)
+|  |  |
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=10 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 14(GETNEXT)
+|  |
+|  15:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  |  hash-table-id=03
+|  |  hash predicates: cs.cs_item_sk = cr.cr_item_sk, cs.cs_order_number = cr.cr_order_number
+|  |  fk/pk conjuncts: cs.cs_item_sk = cr.cr_item_sk, cs.cs_order_number = cr.cr_order_number
+|  |  other predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=8,9N row-size=56B cardinality=144.16K cost=288310
+|  |  in pipelines: 12(GETNEXT), 13(OPEN)
+|  |
+|  |--F19:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.11MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[29996]
+|  |  JOIN BUILD
+|  |  |  join-table-id=03 plan-id=04 cohort-id=02
+|  |  |  build expressions: cr.cr_item_sk, cr.cr_order_number
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=28814
+|  |  |
+|  |  41:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=365.66KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=9 row-size=24B cardinality=14.41K cost=1182
+|  |  |  in pipelines: 13(GETNEXT)
+|  |  |
+|  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[147838]
+|  |  13:SCAN HDFS [tpcds_parquet.catalog_returns cr, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=10.62MB
+|  |     predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
+|  |     stored statistics:
+|  |       table: rows=144.07K size=10.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=144.07K
+|  |     parquet statistics predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
+|  |     parquet dictionary predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
+|  |     mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=9 row-size=24B cardinality=14.41K cost=147444
+|  |     in pipelines: 13(GETNEXT)
+|  |
+|  12:SCAN HDFS [tpcds_parquet.catalog_sales cs, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
+|     runtime filters: RF003[min_max] -> cs_sold_date_sk, RF002[bloom] -> cs_sold_date_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     parquet statistics predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
+|     parquet dictionary predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
+|     mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
+|     tuple-ids=8 row-size=32B cardinality=144.16K cost=4369693
+|     in pipelines: 12(GETNEXT)
+|
+11:SELECT
+|  predicates: (rank() <= CAST(10 AS BIGINT) OR rank() <= CAST(10 AS BIGINT))
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=36,33 row-size=104B cardinality=429 cost=4286
+|  in pipelines: 09(GETNEXT)
+|
+10:ANALYTIC
+|  functions: rank()
+|  order by: (CAST(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=36,33 row-size=104B cardinality=4.29K cost=4286
+|  in pipelines: 09(GETNEXT)
+|
+09:SORT
+|  order by: (CAST(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  materialized: (CAST(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15,4)))
+|  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=36 row-size=96B cardinality=4.29K cost=4286
+|  in pipelines: 09(GETNEXT), 07(OPEN)
+|
+08:ANALYTIC
+|  functions: rank()
+|  order by: (CAST(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=34,31 row-size=80B cardinality=4.29K cost=4286
+|  in pipelines: 07(GETNEXT)
+|
+40:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: (CAST(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  mem-estimate=302.68KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=34 row-size=72B cardinality=4.29K cost=319
+|  in pipelines: 07(GETNEXT)
+|
+F03:PLAN FRAGMENT [HASH(ws.ws_item_sk)] hosts=2 instances=2
+Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservation=1
+max-parallelism=2 fragment-costs=[21682, 4286, 319]
+07:SORT
+|  order by: (CAST(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  materialized: (CAST(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15,4)))
+|  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=34 row-size=72B cardinality=4.29K cost=4286
+|  in pipelines: 07(GETNEXT), 39(OPEN)
+|
+39:AGGREGATE [FINALIZE]
+|  output: sum:merge(coalesce(wr.wr_return_quantity, 0)), sum:merge(coalesce(ws.ws_quantity, 0)), sum:merge(coalesce(wr.wr_return_amt, 0)), sum:merge(coalesce(ws.ws_net_paid, 0))
+|  group by: ws.ws_item_sk
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=4.29K cost=21430
+|  in pipelines: 39(GETNEXT), 01(OPEN)
+|
+38:EXCHANGE [HASH(ws.ws_item_sk)]
+|  mem-estimate=237.20KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=4.29K cost=252
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=42.47MB mem-reservation=18.00MB thread-reservation=1
+max-parallelism=2 fragment-costs=[2417877, 252]
+06:AGGREGATE [STREAMING]
+|  output: sum(CAST(coalesce(wr.wr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(ws.ws_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(wr.wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(ws.ws_net_paid, CAST(0 AS DECIMAL(7,2))))
+|  group by: ws.ws_item_sk
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=4.29K cost=21430
+|  in pipelines: 01(GETNEXT)
+|
+05:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=00
+|  hash predicates: ws_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2 row-size=68B cardinality=4.29K cost=71938
+|  in pipelines: 01(GETNEXT), 03(OPEN)
+|
+|--F16:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=2 fragment-costs=[112]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF000[bloom] <- d_date_sk, RF001[min_max] <- d_date_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |
+|  37:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=12B cardinality=108 cost=4
+|  |  in pipelines: 03(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[146957]
+|  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=2 row-size=12B cardinality=108 cost=146955
+|     in pipelines: 03(GETNEXT)
+|
+04:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  hash-table-id=01
+|  hash predicates: ws.ws_item_sk = wr.wr_item_sk, ws.ws_order_number = wr.wr_order_number
+|  fk/pk conjuncts: ws.ws_item_sk = wr.wr_item_sk, ws.ws_order_number = wr.wr_order_number
+|  other predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N row-size=56B cardinality=71.94K cost=143876
+|  in pipelines: 01(GETNEXT), 02(OPEN)
+|
+|--F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=7.94MB mem-reservation=7.75MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[14746]
+|  JOIN BUILD
+|  |  join-table-id=01 plan-id=02 cohort-id=01
+|  |  build expressions: wr.wr_item_sk, wr.wr_order_number
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=14352
+|  |
+|  36:EXCHANGE [BROADCAST]
+|  |  mem-estimate=196.19KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1 row-size=24B cardinality=7.18K cost=394
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.11MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[73642]
+|  02:SCAN HDFS [tpcds_parquet.web_returns wr, RANDOM]
+|     HDFS partitions=1/1 files=1 size=5.66MB
+|     predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|     stored statistics:
+|       table: rows=71.76K size=5.66MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=71.76K
+|     parquet statistics predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|     parquet dictionary predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|     tuple-ids=1 row-size=24B cardinality=7.18K cost=73445
+|     in pipelines: 02(GETNEXT)
+|
+01:SCAN HDFS [tpcds_parquet.web_sales ws, RANDOM]
+   HDFS partitions=1/1 files=2 size=45.09MB
+   predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
+   runtime filters: RF001[min_max] -> ws_sold_date_sk, RF000[bloom] -> ws_sold_date_sk
+   stored statistics:
+     table: rows=719.38K size=45.09MB
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=644.77K
+   parquet statistics predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
+   parquet dictionary predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
+   mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
+   tuple-ids=0 row-size=32B cardinality=71.94K cost=2180633
+   in pipelines: 01(GETNEXT)
+====
+# TPCDS-Q76
+SELECT channel,
+       col_name,
+       d_year,
+       d_qoy,
+       i_category,
+       COUNT(*) sales_cnt,
+       SUM(ext_sales_price) sales_amt
+FROM
+  ( SELECT 'store' AS channel,
+           'ss_store_sk' col_name,
+                         d_year,
+                         d_qoy,
+                         i_category,
+                         ss_ext_sales_price ext_sales_price
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_store_sk IS NULL
+     AND ss_sold_date_sk=d_date_sk
+     AND ss_item_sk=i_item_sk
+   UNION ALL SELECT 'web' AS channel,
+                    'ws_ship_customer_sk' col_name,
+                                          d_year,
+                                          d_qoy,
+                                          i_category,
+                                          ws_ext_sales_price ext_sales_price
+   FROM web_sales,
+        item,
+        date_dim
+   WHERE ws_ship_customer_sk IS NULL
+     AND ws_sold_date_sk=d_date_sk
+     AND ws_item_sk=i_item_sk
+   UNION ALL SELECT 'catalog' AS channel,
+                    'cs_ship_addr_sk' col_name,
+                                      d_year,
+                                      d_qoy,
+                                      i_category,
+                                      cs_ext_sales_price ext_sales_price
+   FROM catalog_sales,
+        item,
+        date_dim
+   WHERE cs_ship_addr_sk IS NULL
+     AND cs_sold_date_sk=d_date_sk
+     AND cs_item_sk=i_item_sk) foo
+GROUP BY channel,
+         col_name,
+         d_year,
+         d_qoy,
+         i_category
+ORDER BY channel,
+         col_name,
+         d_year,
+         d_qoy,
+         i_category
+LIMIT 100;
+
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=92.62MB Threads=31
+Per-Host Resource Estimates: Memory=419MB
+F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[708]
+PLAN-ROOT SINK
+|  output exprs: channel, col_name, d_year, d_qoy, i_category, count(*), sum(ext_sales_price)
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=700
+|
+30:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: channel ASC, col_name ASC, d_year ASC, d_qoy ASC, i_category ASC
+|  limit: 100
+|  mem-estimate=25.23KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=12 row-size=74B cardinality=100 cost=8
+|  in pipelines: 17(GETNEXT)
+|
+F14:PLAN FRAGMENT [HASH(channel,col_name,d_year,d_qoy,i_category)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=14.14MB mem-reservation=4.75MB thread-reservation=1
+max-parallelism=3 fragment-costs=[972054, 500, 8]
+17:TOP-N [LIMIT=100]
+|  order by: channel ASC, col_name ASC, d_year ASC, d_qoy ASC, i_category ASC
+|  mem-estimate=7.22KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=12 row-size=74B cardinality=100 cost=500
+|  in pipelines: 17(GETNEXT), 29(OPEN)
+|
+29:AGGREGATE [FINALIZE]
+|  output: count:merge(*), sum:merge(ext_sales_price)
+|  group by: channel, col_name, d_year, d_qoy, i_category
+|  mem-estimate=10.00MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=11 row-size=74B cardinality=137.37K cost=961604
+|  in pipelines: 29(GETNEXT), 01(OPEN), 07(OPEN), 11(OPEN)
+|
+28:EXCHANGE [HASH(channel,col_name,d_year,d_qoy,i_category)]
+|  mem-estimate=4.14MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=11 row-size=74B cardinality=137.37K cost=10450
+|  in pipelines: 01(GETNEXT), 07(GETNEXT), 11(GETNEXT)
+|
+F13:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=26.91MB mem-reservation=2.25MB thread-reservation=1
+max-parallelism=12 fragment-costs=[3597079, 10450]
+16:AGGREGATE [STREAMING]
+|  output: count(*), sum(ext_sales_price)
+|  group by: channel, col_name, d_year, d_qoy, i_category
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=11 row-size=74B cardinality=137.37K cost=961604
+|  in pipelines: 01(GETNEXT), 07(GETNEXT), 11(GETNEXT)
+|
+00:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=9 row-size=54B cardinality=137.37K cost=2472696
+|  in pipelines: 01(GETNEXT), 07(GETNEXT), 11(GETNEXT)
+|
+|--15:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=03
+|  |  hash predicates: cs_item_sk = i_item_sk
+|  |  fk/pk conjuncts: cs_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=6,8,7 row-size=58B cardinality=7.17K cost=7165
+|  |  in pipelines: 11(GETNEXT), 12(OPEN)
+|  |
+|  |--F19:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Instance Resources: mem-estimate=3.41MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=12 fragment-costs=[18526]
+|  |  JOIN BUILD
+|  |  |  join-table-id=03 plan-id=04 cohort-id=01
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF008[bloom] <- i_item_sk, RF009[min_max] <- i_item_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  27:EXCHANGE [HASH(i_item_sk)]
+|  |  |  mem-estimate=485.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=7 row-size=26B cardinality=18.00K cost=526
+|  |  |  in pipelines: 12(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=17.40MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[982]
+|  |  12:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=7 row-size=26B cardinality=18.00K cost=456
+|  |     in pipelines: 12(GETNEXT)
+|  |
+|  26:EXCHANGE [HASH(cs_item_sk)]
+|  |  mem-estimate=194.63KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=6,8 row-size=32B cardinality=7.17K cost=280
+|  |  in pipelines: 11(GETNEXT)
+|  |
+|  F10:PLAN FRAGMENT [HASH(cs_sold_date_sk)] hosts=3 instances=3
+|  Per-Instance Resources: mem-estimate=1.99MB mem-reservation=0B thread-reservation=1
+|  max-parallelism=3 fragment-costs=[7613]
+|  14:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=05
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: none
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=6,8 row-size=32B cardinality=7.17K cost=7165
+|  |  in pipelines: 11(GETNEXT), 13(OPEN)
+|  |
+|  |--F21:PLAN FRAGMENT [HASH(cs_sold_date_sk)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=3.79MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[74191]
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=01
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=73049
+|  |  |
+|  |  25:EXCHANGE [HASH(d_date_sk)]
+|  |  |  mem-estimate=872.04KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=8 row-size=12B cardinality=73.05K cost=1142
+|  |  |  in pipelines: 13(GETNEXT)
+|  |  |
+|  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.19MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1999]
+|  |  13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=8 row-size=12B cardinality=73.05K cost=857
+|  |     in pipelines: 13(GETNEXT)
+|  |
+|  24:EXCHANGE [HASH(cs_sold_date_sk)]
+|  |  mem-estimate=118.65KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=6 row-size=20B cardinality=7.17K cost=168
+|  |  in pipelines: 11(GETNEXT)
+|  |
+|  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=48.28MB mem-reservation=8.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[1469872]
+|  11:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     predicates: cs_ship_addr_sk IS NULL
+|     runtime filters: RF009[min_max] -> cs_item_sk, RF011[min_max] -> cs_sold_date_sk, RF008[bloom] -> cs_item_sk, RF010[bloom] -> cs_sold_date_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
+|     tuple-ids=6 row-size=20B cardinality=7.17K cost=1469704
+|     in pipelines: 11(GETNEXT)
+|
+|--10:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=01
+|  |  hash predicates: i_item_sk = ws_item_sk
+|  |  fk/pk conjuncts: i_item_sk = ws_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5,3 row-size=58B cardinality=173 cost=18000
+|  |  in pipelines: 07(GETNEXT), 08(OPEN)
+|  |
+|  |--F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[194]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=01
+|  |  |  build expressions: ws_item_sk
+|  |  |  runtime filters: RF004[bloom] <- ws_item_sk, RF005[min_max] <- ws_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=173
+|  |  |
+|  |  23:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5,3 row-size=32B cardinality=173 cost=21
+|  |  |  in pipelines: 08(GETNEXT)
+|  |  |
+|  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[73913]
+|  |  09:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: d_date_sk = ws_sold_date_sk
+|  |  |  fk/pk conjuncts: none
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=5,3 row-size=32B cardinality=173 cost=73049
+|  |  |  in pipelines: 08(GETNEXT), 06(OPEN)
+|  |  |
+|  |  |--F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=1 fragment-costs=[178]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=02
+|  |  |  |  build expressions: ws_sold_date_sk
+|  |  |  |  runtime filters: RF006[bloom] <- ws_sold_date_sk, RF007[min_max] <- ws_sold_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=173
+|  |  |  |
+|  |  |  22:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=3 row-size=20B cardinality=173 cost=5
+|  |  |  |  in pipelines: 06(GETNEXT)
+|  |  |  |
+|  |  |  F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=32.09MB mem-reservation=8.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[733440]
+|  |  |  06:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     predicates: ws_ship_customer_sk IS NULL
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
+|  |  |     tuple-ids=3 row-size=20B cardinality=173 cost=733435
+|  |  |     in pipelines: 06(GETNEXT)
+|  |  |
+|  |  08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     runtime filters: RF007[min_max] -> d_date_sk, RF006[bloom] -> d_date_sk
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=5 row-size=12B cardinality=73.05K cost=857
+|  |     in pipelines: 08(GETNEXT)
+|  |
+|  07:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     runtime filters: RF005[min_max] -> i_item_sk, RF004[bloom] -> i_item_sk
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=4 row-size=26B cardinality=18.00K cost=456
+|     in pipelines: 07(GETNEXT)
+|
+05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1,2 row-size=58B cardinality=130.03K cost=130034
+|  in pipelines: 01(GETNEXT), 03(OPEN)
+|
+|--F16:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=3.79MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=12 fragment-costs=[74191]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF000[bloom] <- d_date_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=73049
+|  |
+|  21:EXCHANGE [HASH(d_date_sk)]
+|  |  mem-estimate=872.04KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=12B cardinality=73.05K cost=1142
+|  |  in pipelines: 03(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.75MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1999]
+|  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=2 row-size=12B cardinality=73.05K cost=857
+|     in pipelines: 03(GETNEXT)
+|
+20:EXCHANGE [HASH(ss_sold_date_sk)]
+|  mem-estimate=2.06MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,1 row-size=46B cardinality=130.03K cost=6844
+|  in pipelines: 01(GETNEXT)
+|
+F02:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=3.63MB mem-reservation=0B thread-reservation=1
+max-parallelism=3 fragment-costs=[139926]
+04:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=04
+|  hash predicates: ss_item_sk = i_item_sk
+|  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1 row-size=46B cardinality=130.03K cost=130034
+|  in pipelines: 01(GETNEXT), 02(OPEN)
+|
+|--F20:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=3.41MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[18526]
+|  JOIN BUILD
+|  |  join-table-id=04 plan-id=05 cohort-id=01
+|  |  build expressions: i_item_sk
+|  |  runtime filters: RF002[bloom] <- i_item_sk, RF003[min_max] <- i_item_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |
+|  19:EXCHANGE [HASH(i_item_sk)]
+|  |  mem-estimate=485.07KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1 row-size=26B cardinality=18.00K cost=526
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.35MB mem-reservation=256.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[982]
+|  02:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=1 row-size=26B cardinality=18.00K cost=456
+|     in pipelines: 02(GETNEXT)
+|
+18:EXCHANGE [HASH(ss_item_sk)]
+|  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0 row-size=20B cardinality=130.03K cost=3048
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=16.28MB mem-reservation=1.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[2939710]
+01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=199.44MB
+   predicates: ss_store_sk IS NULL
+   runtime filters: RF003[min_max] -> ss_item_sk, RF000[bloom] -> ss_sold_date_sk, RF002[bloom] -> ss_item_sk
+   stored statistics:
+     table: rows=2.88M size=199.44MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=0 row-size=20B cardinality=130.03K cost=2936662
+   in pipelines: 01(GETNEXT)
+====
+# TPCDS-Q78
+WITH ws AS
+  (SELECT d_year AS ws_sold_year,
+          ws_item_sk,
+          ws_bill_customer_sk ws_customer_sk,
+          sum(ws_quantity) ws_qty,
+          sum(ws_wholesale_cost) ws_wc,
+          sum(ws_sales_price) ws_sp
+   FROM web_sales
+   LEFT JOIN web_returns ON wr_order_number=ws_order_number
+   AND ws_item_sk=wr_item_sk
+   JOIN date_dim ON ws_sold_date_sk = d_date_sk
+   WHERE wr_order_number IS NULL
+   GROUP BY d_year,
+            ws_item_sk,
+            ws_bill_customer_sk ),
+     cs AS
+  (SELECT d_year AS cs_sold_year,
+          cs_item_sk,
+          cs_bill_customer_sk cs_customer_sk,
+          sum(cs_quantity) cs_qty,
+          sum(cs_wholesale_cost) cs_wc,
+          sum(cs_sales_price) cs_sp
+   FROM catalog_sales
+   LEFT JOIN catalog_returns ON cr_order_number=cs_order_number
+   AND cs_item_sk=cr_item_sk
+   JOIN date_dim ON cs_sold_date_sk = d_date_sk
+   WHERE cr_order_number IS NULL
+   GROUP BY d_year,
+            cs_item_sk,
+            cs_bill_customer_sk ),
+     ss AS
+  (SELECT d_year AS ss_sold_year,
+          ss_item_sk,
+          ss_customer_sk,
+          sum(ss_quantity) ss_qty,
+          sum(ss_wholesale_cost) ss_wc,
+          sum(ss_sales_price) ss_sp
+   FROM store_sales
+   LEFT JOIN store_returns ON sr_ticket_number=ss_ticket_number
+   AND ss_item_sk=sr_item_sk
+   JOIN date_dim ON ss_sold_date_sk = d_date_sk
+   WHERE sr_ticket_number IS NULL
+   GROUP BY d_year,
+            ss_item_sk,
+            ss_customer_sk )
+SELECT ss_sold_year,
+       ss_item_sk,
+       ss_customer_sk,
+       round((ss_qty*1.00)/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio,
+       ss_qty store_qty,
+       ss_wc store_wholesale_cost,
+       ss_sp store_sales_price,
+       coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty,
+       coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost,
+       coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price
+FROM ss
+LEFT JOIN ws ON (ws_sold_year=ss_sold_year
+                 AND ws_item_sk=ss_item_sk
+                 AND ws_customer_sk=ss_customer_sk)
+LEFT JOIN cs ON (cs_sold_year=ss_sold_year
+                 AND cs_item_sk=ss_item_sk
+                 AND cs_customer_sk=ss_customer_sk)
+WHERE (coalesce(ws_qty,0)>0
+       OR coalesce(cs_qty, 0)>0)
+  AND ss_sold_year=2000
+ORDER BY ss_sold_year,
+         ss_item_sk,
+         ss_customer_sk,
+         ss_qty DESC,
+         ss_wc DESC,
+         ss_sp DESC,
+         other_chan_qty,
+         other_chan_wholesale_cost,
+         other_chan_sales_price,
+         ratio
+LIMIT 100;
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=296.62MB Threads=24
+Per-Host Resource Estimates: Memory=558MB
+F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1012]
+PLAN-ROOT SINK
+|  output exprs: ss_sold_year, ss_item_sk, ss_customer_sk, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2), ss_qty, ss_wc, ss_sp, coalesce(ws_qty, 0) + coalesce(cs_qty, 0), coalesce(ws_wc, 0) + coalesce(cs_wc, 0), coalesce(ws_sp, 0) + coalesce(cs_sp, 0)
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=1000
+|
+35:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: ss_sold_year ASC, ss_item_sk ASC, ss_customer_sk ASC, ss_qty DESC, ss_wc DESC, ss_sp DESC, coalesce(ws_qty, 0) + coalesce(cs_qty, 0) ASC, coalesce(ws_wc, 0) + coalesce(cs_wc, 0) ASC, coalesce(ws_sp, 0) + coalesce(cs_sp, 0) ASC, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2) ASC
+|  limit: 100
+|  mem-estimate=37.63KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=15 row-size=112B cardinality=100 cost=12
+|  in pipelines: 20(GETNEXT)
+|
+F03:PLAN FRAGMENT [HASH(d_year,cs_item_sk,cs_bill_customer_sk)] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reservation=1
+max-parallelism=3 fragment-costs=[1785038, 884887, 12]
+20:TOP-N [LIMIT=100]
+|  order by: ss_sold_year ASC, ss_item_sk ASC, ss_customer_sk ASC, ss_qty DESC, ss_wc DESC, ss_sp DESC, coalesce(ws_qty, 0) + coalesce(cs_qty, 0) ASC, coalesce(ws_wc, 0) + coalesce(cs_wc, 0) ASC, coalesce(ws_sp, 0) + coalesce(cs_sp, 0) ASC, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2) ASC
+|  materialized: coalesce(ws_qty, 0) + coalesce(cs_qty, 0), coalesce(ws_wc, 0) + coalesce(cs_wc, 0), coalesce(ws_sp, 0) + coalesce(cs_sp, 0), round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2)
+|  mem-estimate=10.94KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=15 row-size=112B cardinality=100 cost=1000
+|  in pipelines: 20(GETNEXT), 24(OPEN)
+|
+19:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: d_year = d_year, cs_bill_customer_sk = ss_customer_sk, cs_item_sk = ss_item_sk
+|  fk/pk conjuncts: cs_bill_customer_sk = ss_customer_sk, cs_item_sk = ss_item_sk
+|  other predicates: (coalesce(sum(ws_quantity), CAST(0 AS BIGINT)) > CAST(0 AS BIGINT) OR coalesce(sum(cs_quantity), CAST(0 AS BIGINT)) > CAST(0 AS BIGINT))
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=13N,8N,3 row-size=168B cardinality=3.00K cost=883887
+|  in pipelines: 24(GETNEXT), 28(OPEN)
+|
+|--F13:PLAN FRAGMENT [HASH(d_year,cs_item_sk,cs_bill_customer_sk)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=5.33MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=3.00MB
+|  |  max-parallelism=3 fragment-costs=[9368]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: d_year, ss_customer_sk, ss_item_sk
+|  |  runtime filters: RF000[bloom] <- d_year, RF001[bloom] <- ss_customer_sk, RF002[bloom] <- ss_item_sk, RF004[min_max] <- ss_customer_sk, RF005[min_max] <- ss_item_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=9015
+|  |
+|  34:EXCHANGE [HASH(d_year,ss_item_sk,ss_customer_sk)]
+|  |  mem-estimate=404.34KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8N,3 row-size=112B cardinality=3.00K cost=353
+|  |  in pipelines: 28(GETNEXT)
+|  |
+|  F07:PLAN FRAGMENT [HASH(d_year,ws_item_sk,ws_bill_customer_sk)] hosts=2 instances=2
+|  Per-Instance Resources: mem-estimate=14.07MB mem-reservation=8.50MB thread-reservation=1
+|  max-parallelism=2 fragment-costs=[896697, 444365]
+|  18:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
+|  |  hash-table-id=01
+|  |  hash predicates: d_year = d_year, ws_bill_customer_sk = ss_customer_sk, ws_item_sk = ss_item_sk
+|  |  fk/pk conjuncts: ws_bill_customer_sk = ss_customer_sk, ws_item_sk = ss_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=8N,3 row-size=112B cardinality=3.00K cost=444012
+|  |  in pipelines: 28(GETNEXT), 32(OPEN)
+|  |
+|  |--F14:PLAN FRAGMENT [HASH(d_year,ws_item_sk,ws_bill_customer_sk)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=5.17MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=3.00MB
+|  |  |  max-parallelism=2 fragment-costs=[9192]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  build expressions: d_year, ss_customer_sk, ss_item_sk
+|  |  |  runtime filters: RF008[bloom] <- d_year, RF009[bloom] <- ss_customer_sk, RF010[bloom] <- ss_item_sk, RF012[min_max] <- ss_customer_sk, RF013[min_max] <- ss_item_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=9015
+|  |  |
+|  |  33:EXCHANGE [HASH(d_year,ss_item_sk,ss_customer_sk)]
+|  |  |  mem-estimate=234.78KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=56B cardinality=3.00K cost=177
+|  |  |  in pipelines: 32(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [HASH(d_year,ss_item_sk,ss_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=27.70MB mem-reservation=17.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[3568694, 177]
+|  |  32:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge(ss_quantity), sum:merge(ss_wholesale_cost), sum:merge(ss_sales_price)
+|  |  |  group by: d_year, ss_item_sk, ss_customer_sk
+|  |  |  having: d_year = CAST(2000 AS INT)
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=56B cardinality=3.00K cost=3534180
+|  |  |  in pipelines: 32(GETNEXT), 00(OPEN)
+|  |  |
+|  |  31:EXCHANGE [HASH(d_year,ss_item_sk,ss_customer_sk)]
+|  |  |  mem-estimate=10.70MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=56B cardinality=589.03K cost=34514
+|  |  |  in pipelines: 00(GETNEXT)
+|  |  |
+|  |  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=26.70MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[12276657, 34514]
+|  |  05:AGGREGATE [STREAMING]
+|  |  |  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  |  |  group by: d_year, ss_item_sk, ss_customer_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=56B cardinality=589.03K cost=3534180
+|  |  |  in pipelines: 00(GETNEXT)
+|  |  |
+|  |  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=0,1N,2 row-size=60B cardinality=589.03K cost=2880404
+|  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
+|  |  |
+|  |  |--F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=3 fragment-costs=[388]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF016[bloom] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |  |  |
+|  |  |  30:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=2 row-size=8B cardinality=373 cost=15
+|  |  |  |  in pipelines: 02(GETNEXT)
+|  |  |  |
+|  |  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[73625]
+|  |  |  02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=2 row-size=8B cardinality=373 cost=73620
+|  |  |     in pipelines: 02(GETNEXT)
+|  |  |
+|  |  03:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  |  |  hash-table-id=03
+|  |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
+|  |  |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
+|  |  |  other predicates: sr_ticket_number IS NULL
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=0,1N row-size=52B cardinality=2.88M cost=5760808
+|  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
+|  |  |
+|  |  |--F16:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=72.41MB mem-reservation=68.00MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[591876]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
+|  |  |  |  build expressions: sr_item_sk, sr_ticket_number
+|  |  |  |  mem-estimate=68.00MB mem-reservation=68.00MB spill-buffer=1.00MB thread-reservation=0 cost=575028
+|  |  |  |
+|  |  |  29:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=4.41MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=1 row-size=16B cardinality=287.51K cost=16848
+|  |  |  |  in pipelines: 01(GETNEXT)
+|  |  |  |
+|  |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=24.08MB mem-reservation=4.00MB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[10109]
+|  |  |  01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=15.43MB
+|  |  |     stored statistics:
+|  |  |       table: rows=287.51K size=15.43MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|  |  |     mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=1 row-size=16B cardinality=287.51K cost=4493
+|  |  |     in pipelines: 01(GETNEXT)
+|  |  |
+|  |  00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF016[bloom] -> ss_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=0 row-size=36B cardinality=2.88M cost=101265
+|  |     in pipelines: 00(GETNEXT)
+|  |
+|  28:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ws_quantity), sum:merge(ws_wholesale_cost), sum:merge(ws_sales_price)
+|  |  group by: d_year, ws_item_sk, ws_bill_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  tuple-ids=8 row-size=56B cardinality=148.00K cost=888024
+|  |  in pipelines: 28(GETNEXT), 06(OPEN)
+|  |
+|  27:EXCHANGE [HASH(d_year,ws_item_sk,ws_bill_customer_sk)]
+|  |  mem-estimate=4.07MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=56B cardinality=148.00K cost=8673
+|  |  in pipelines: 06(GETNEXT)
+|  |
+|  F04:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+|  Per-Instance Resources: mem-estimate=42.47MB mem-reservation=21.00MB thread-reservation=1
+|  max-parallelism=2 fragment-costs=[3071467, 8673]
+|  11:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ws_quantity AS BIGINT)), sum(ws_wholesale_cost), sum(ws_sales_price)
+|  |  group by: d_year, ws_item_sk, ws_bill_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=8 row-size=56B cardinality=148.00K cost=888024
+|  |  in pipelines: 06(GETNEXT)
+|  |
+|  10:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=04
+|  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=5,6N,7 row-size=60B cardinality=148.00K cost=719384
+|  |  in pipelines: 06(GETNEXT), 08(OPEN)
+|  |
+|  |--F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=2 fragment-costs=[383]
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |  |
+|  |  26:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=7 row-size=8B cardinality=373 cost=10
+|  |  |  in pipelines: 08(GETNEXT)
+|  |  |
+|  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[73625]
+|  |  08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |     runtime filters: RF008[bloom] -> tpcds_parquet.date_dim.d_year
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=7 row-size=8B cardinality=373 cost=73620
+|  |     in pipelines: 08(GETNEXT)
+|  |
+|  09:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  |  hash-table-id=05
+|  |  hash predicates: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
+|  |  fk/pk conjuncts: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
+|  |  other predicates: wr_order_number IS NULL
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=5,6N row-size=52B cardinality=719.38K cost=1438768
+|  |  in pipelines: 06(GETNEXT), 07(OPEN)
+|  |
+|  |--F18:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=20.11MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[146330]
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=02
+|  |  |  build expressions: wr_item_sk, wr_order_number
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=143526
+|  |  |
+|  |  25:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=6 row-size=16B cardinality=71.76K cost=2804
+|  |  |  in pipelines: 07(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=1.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2524]
+|  |  07:SCAN HDFS [tpcds_parquet.web_returns, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.66MB
+|  |     runtime filters: RF013[min_max] -> tpcds_parquet.web_returns.wr_item_sk, RF010[bloom] -> tpcds_parquet.web_returns.wr_item_sk
+|  |     stored statistics:
+|  |       table: rows=71.76K size=5.66MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=71.76K
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=6 row-size=16B cardinality=71.76K cost=1122
+|  |     in pipelines: 07(GETNEXT)
+|  |
+|  06:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|     HDFS partitions=1/1 files=2 size=45.09MB
+|     runtime filters: RF012[min_max] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF013[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF015[min_max] -> ws_sold_date_sk, RF009[bloom] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_item_sk, RF014[bloom] -> ws_sold_date_sk
+|     stored statistics:
+|       table: rows=719.38K size=45.09MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
+|     tuple-ids=5 row-size=36B cardinality=719.38K cost=25291
+|     in pipelines: 06(GETNEXT)
+|
+24:AGGREGATE [FINALIZE]
+|  output: sum:merge(cs_quantity), sum:merge(cs_wholesale_cost), sum:merge(cs_sales_price)
+|  group by: d_year, cs_item_sk, cs_bill_customer_sk
+|  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=13 row-size=56B cardinality=294.63K cost=1767774
+|  in pipelines: 24(GETNEXT), 12(OPEN)
+|
+23:EXCHANGE [HASH(d_year,cs_item_sk,cs_bill_customer_sk)]
+|  mem-estimate=5.42MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=13 row-size=56B cardinality=294.63K cost=17264
+|  in pipelines: 12(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+Per-Instance Resources: mem-estimate=58.70MB mem-reservation=25.00MB thread-reservation=1
+max-parallelism=3 fragment-costs=[6143098, 17264]
+17:AGGREGATE [STREAMING]
+|  output: sum(CAST(cs_quantity AS BIGINT)), sum(cs_wholesale_cost), sum(cs_sales_price)
+|  group by: d_year, cs_item_sk, cs_bill_customer_sk
+|  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=13 row-size=56B cardinality=294.63K cost=1767774
+|  in pipelines: 12(GETNEXT)
+|
+16:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=06
+|  hash predicates: cs_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10,11N,12 row-size=60B cardinality=294.63K cost=1441548
+|  in pipelines: 12(GETNEXT), 14(OPEN)
+|
+|--F19:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[388]
+|  JOIN BUILD
+|  |  join-table-id=06 plan-id=07 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |
+|  22:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=12 row-size=8B cardinality=373 cost=15
+|  |  in pipelines: 14(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[73625]
+|  14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|     runtime filters: RF000[bloom] -> tpcds_parquet.date_dim.d_year
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=12 row-size=8B cardinality=373 cost=73620
+|     in pipelines: 14(GETNEXT)
+|
+15:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  hash-table-id=07
+|  hash predicates: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
+|  fk/pk conjuncts: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
+|  other predicates: cr_order_number IS NULL
+|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=10,11N row-size=52B cardinality=1.44M cost=2883096
+|  in pipelines: 12(GETNEXT), 13(OPEN)
+|
+|--F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=36.22MB mem-reservation=34.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[296576]
+|  JOIN BUILD
+|  |  join-table-id=07 plan-id=08 cohort-id=01
+|  |  build expressions: cr_item_sk, cr_order_number
+|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=512.00KB thread-reservation=0 cost=288134
+|  |
+|  21:EXCHANGE [BROADCAST]
+|  |  mem-estimate=2.22MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=11 row-size=16B cardinality=144.07K cost=8442
+|  |  in pipelines: 13(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
... 984 lines suppressed ...