You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2023/03/23 16:20:40 UTC

[impala] 01/03: IMPALA-12005: Describe executor group set selection in query profile

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

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

commit 5c1693d03d67df49236ed393ac722732986727dc
Author: Riza Suminto <ri...@cloudera.com>
AuthorDate: Wed Mar 15 22:07:01 2023 -0700

    IMPALA-12005: Describe executor group set selection in query profile
    
    This patch adds new profile counters under the Frontend profile node to
    describe executor group set selection during query planning. It modifies
    FrontendProfile.java to allow one level of TRuntimeProfileNode nesting
    under the Frontend profile node. This makes it possible to group profile
    counters specific to each executor group set in consideration.
    "fragment-costs" hint is renamed to "segment-costs". A new
    "cpu-comparison-result" hint is added after "segment-costs" to help
    navigate how cpu sizing decision is made.
    
    This patch also adds some function overloading in runtime-profile.cc to
    hide TotalTime and InactiveTotalTime that is meaningless for anything
    under the Frontend profile node. Additional context also added into
    AnalysisException threw when none of the executor group sets fits the
    query requirement.
    
    This is how the Frontend profile node looks like after running
    TestExecutorGroups::test_query_cpu_count_divisor_fraction
    
        Frontend:
          Referenced Tables: tpcds_parquet.store_sales
           - CpuCountDivisor: 0.20
           - ExecutorGroupsConsidered: 3 (3)
          Executor group 1 (root.tiny):
            Verdict: not enough cpu cores
             - CpuAsk: 15 (15)
             - CpuMax: 2 (2)
             - EffectiveParallelism: 3 (3)
             - MemoryAsk: 36.83 MB (38617088)
             - MemoryMax: 64.00 MB (67108864)
          Executor group 2 (root.small):
            Verdict: not enough cpu cores
             - CpuAsk: 25 (25)
             - CpuMax: 16 (16)
             - EffectiveParallelism: 5 (5)
             - MemoryAsk: 36.83 MB (38624004)
             - MemoryMax: 64.00 MB (67108864)
          Executor group 3 (root.large):
            Verdict: Match
             - CpuAsk: 35 (35)
             - CpuMax: 192 (192)
             - EffectiveParallelism: 7 (7)
             - MemoryAsk: 36.84 MB (38633570)
             - MemoryMax: 8388608.00 GB (9007199254740992)
    
    Testing:
    - Pass core tests
    
    Change-Id: I6c0ac7f5216d631e4439fe97702e21e06d2eda8a
    Reviewed-on: http://gerrit.cloudera.org:8080/19628
    Reviewed-by: Riza Suminto <ri...@cloudera.com>
    Tested-by: Riza Suminto <ri...@cloudera.com>
---
 be/src/service/client-request-state.cc             |  12 +-
 be/src/service/client-request-state.h              |   2 +-
 be/src/service/impala-server.cc                    |   2 +-
 be/src/util/runtime-profile.cc                     |  48 +-
 be/src/util/runtime-profile.h                      |  21 +-
 common/thrift/Frontend.thrift                      |   3 +
 .../org/apache/impala/planner/PlanFragment.java    |  28 +-
 .../java/org/apache/impala/planner/Planner.java    |   9 -
 .../java/org/apache/impala/service/Frontend.java   | 109 ++-
 .../org/apache/impala/service/FrontendProfile.java |  64 +-
 .../queries/PlannerTest/tpcds-processing-cost.test | 814 ++++++++++-----------
 tests/custom_cluster/test_executor_groups.py       |  30 +-
 12 files changed, 668 insertions(+), 474 deletions(-)

diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index e0c066de6..5ef85b7fa 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -109,7 +109,7 @@ ClientRequestState::ClientRequestState(const TQueryCtx& query_ctx, Frontend* fro
     coord_exec_called_(false),
     // Profile is assigned name w/ id after planning
     profile_(RuntimeProfile::Create(&profile_pool_, "Query")),
-    frontend_profile_(RuntimeProfile::Create(&profile_pool_, "Frontend")),
+    frontend_profile_(RuntimeProfile::Create(&profile_pool_, "Frontend", false)),
     server_profile_(RuntimeProfile::Create(&profile_pool_, "ImpalaServer")),
     summary_profile_(RuntimeProfile::Create(&profile_pool_, "Summary")),
     exec_request_(exec_request),
@@ -198,11 +198,15 @@ void ClientRequestState::SetRemoteSubmitTime(int64_t remote_submit_time) {
   query_events_->Start(remote_submit_time);
 }
 
-void ClientRequestState::SetFrontendProfile(TRuntimeProfileNode profile) {
+void ClientRequestState::SetFrontendProfile(const TExecRequest& exec_request) {
   // Should we defer creating and adding the child until here? probably.
   TRuntimeProfileTree prof_tree;
-  prof_tree.nodes.emplace_back(std::move(profile));
-  frontend_profile_->Update(prof_tree);
+  prof_tree.nodes.emplace_back(std::move(exec_request.profile));
+  for (auto& child : exec_request.profile_children) {
+    prof_tree.nodes.emplace_back(std::move(child));
+  }
+  prof_tree.nodes.at(0).num_children = prof_tree.nodes.size() - 1;
+  frontend_profile_->Update(prof_tree, false);
 }
 
 void ClientRequestState::AddBlacklistedExecutorAddress(const NetworkAddressPB& addr) {
diff --git a/be/src/service/client-request-state.h b/be/src/service/client-request-state.h
index 4a98b9cef..2c9da9d2a 100644
--- a/be/src/service/client-request-state.h
+++ b/be/src/service/client-request-state.h
@@ -89,7 +89,7 @@ class ClientRequestState {
   /// Sets the profile that is produced by the frontend. The frontend creates the
   /// profile during planning and returns it to the backend via TExecRequest,
   /// which then sets the frontend profile.
-  void SetFrontendProfile(TRuntimeProfileNode profile);
+  void SetFrontendProfile(const TExecRequest& exec_request);
 
   /// Sets the coordinator time that the plan request was submitted at so that
   /// the backend timeline starts where the frontend timeline ends
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 2e1046c1e..4d6f64840 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -1271,7 +1271,7 @@ Status ImpalaServer::ExecuteInternal(const TQueryCtx& query_ctx,
     (*query_handle)->set_user_profile_access(result.user_has_profile_access);
     (*query_handle)->summary_profile()->AddEventSequence(
         result.timeline.name, result.timeline);
-    (*query_handle)->SetFrontendProfile(result.profile);
+    (*query_handle)->SetFrontendProfile(result);
     if (result.__isset.result_set_metadata) {
       (*query_handle)->set_result_metadata(result.result_set_metadata);
     }
diff --git a/be/src/util/runtime-profile.cc b/be/src/util/runtime-profile.cc
index 1995ebc15..44dc8df11 100644
--- a/be/src/util/runtime-profile.cc
+++ b/be/src/util/runtime-profile.cc
@@ -180,7 +180,7 @@ const char* ProfileEntryPrototype::SignificanceDescription(
 }
 
 RuntimeProfileBase::RuntimeProfileBase(ObjectPool* pool, const string& name,
-    Counter* total_time_counter, Counter* inactive_timer)
+    Counter* total_time_counter, Counter* inactive_timer, bool add_default_counters)
   : pool_(pool),
     name_(name),
     total_time_counter_(total_time_counter),
@@ -188,10 +188,12 @@ RuntimeProfileBase::RuntimeProfileBase(ObjectPool* pool, const string& name,
   DCHECK(total_time_counter != nullptr);
   DCHECK(inactive_timer != nullptr);
   set<string>& root_counters = child_counter_map_[ROOT_COUNTER];
-  counter_map_[TOTAL_TIME_COUNTER_NAME] = total_time_counter;
-  root_counters.emplace(TOTAL_TIME_COUNTER_NAME);
-  counter_map_[INACTIVE_TIME_COUNTER_NAME] = inactive_timer;
-  root_counters.emplace(INACTIVE_TIME_COUNTER_NAME);
+  if (add_default_counters) {
+    counter_map_[TOTAL_TIME_COUNTER_NAME] = total_time_counter;
+    root_counters.emplace(TOTAL_TIME_COUNTER_NAME);
+    counter_map_[INACTIVE_TIME_COUNTER_NAME] = inactive_timer;
+    root_counters.emplace(INACTIVE_TIME_COUNTER_NAME);
+  }
 }
 
 RuntimeProfileBase::~RuntimeProfileBase() {}
@@ -236,13 +238,15 @@ void RuntimeProfileBase::UpdateChildCountersLocked(const unique_lock<SpinLock>&
   }
 }
 
-RuntimeProfile* RuntimeProfile::Create(ObjectPool* pool, const string& name) {
-  return pool->Add(new RuntimeProfile(pool, name));
+RuntimeProfile* RuntimeProfile::Create(
+    ObjectPool* pool, const string& name, bool add_default_counters) {
+  return pool->Add(new RuntimeProfile(pool, name, add_default_counters));
 }
 
-RuntimeProfile::RuntimeProfile(ObjectPool* pool, const string& name)
-  : RuntimeProfileBase(
-        pool, name, &builtin_counter_total_time_, &builtin_inactive_timer_) {}
+RuntimeProfile::RuntimeProfile(
+    ObjectPool* pool, const string& name, bool add_default_counters)
+  : RuntimeProfileBase(pool, name, &builtin_counter_total_time_, &builtin_inactive_timer_,
+      add_default_counters) {}
 
 RuntimeProfile::~RuntimeProfile() {
   DCHECK(!has_active_periodic_counters_);
@@ -736,15 +740,17 @@ void AggregatedRuntimeProfile::UpdateEventSequencesFromInstances(
   }
 }
 
-void RuntimeProfile::Update(const TRuntimeProfileTree& thrift_profile) {
+void RuntimeProfile::Update(
+    const TRuntimeProfileTree& thrift_profile, bool add_default_counters) {
   int idx = 0;
-  Update(thrift_profile.nodes, &idx);
+  Update(thrift_profile.nodes, &idx, add_default_counters);
   DCHECK_EQ(idx, thrift_profile.nodes.size());
   // Re-compute the total time for the entire profile tree.
   ComputeTimeInProfile();
 }
 
-void RuntimeProfile::Update(const vector<TRuntimeProfileNode>& nodes, int* idx) {
+void RuntimeProfile::Update(
+    const vector<TRuntimeProfileNode>& nodes, int* idx, bool add_default_counters) {
   if (UNLIKELY(nodes.size()) == 0) return;
   DCHECK_LT(*idx, nodes.size());
   const TRuntimeProfileNode& node = nodes[*idx];
@@ -853,14 +859,16 @@ void RuntimeProfile::Update(const vector<TRuntimeProfileNode>& nodes, int* idx)
     // Update children with matching names; create new ones if they don't match.
     for (int i = 0; i < node.num_children; ++i) {
       const TRuntimeProfileNode& tchild = nodes[*idx];
-      RuntimeProfile* child = dynamic_cast<RuntimeProfile*>(
-          AddOrCreateChild(tchild.name, &insert_pos, [this, tchild] () {
-              RuntimeProfile* child2 = Create(pool_, tchild.name);
-              child2->metadata_ = tchild.node_metadata;
-              return child2;
-            }, tchild.indent));
+      RuntimeProfile* child = dynamic_cast<RuntimeProfile*>(AddOrCreateChild(
+          tchild.name, &insert_pos,
+          [this, tchild, add_default_counters]() {
+            RuntimeProfile* child2 = Create(pool_, tchild.name, add_default_counters);
+            child2->metadata_ = tchild.node_metadata;
+            return child2;
+          },
+          tchild.indent));
       DCHECK(child != nullptr);
-      child->Update(nodes, idx);
+      child->Update(nodes, idx, add_default_counters);
     }
   }
 }
diff --git a/be/src/util/runtime-profile.h b/be/src/util/runtime-profile.h
index 4e070f281..2b6a2cde6 100644
--- a/be/src/util/runtime-profile.h
+++ b/be/src/util/runtime-profile.h
@@ -313,7 +313,8 @@ class RuntimeProfileBase {
   AtomicInt64 total_time_ns_{0};
 
   RuntimeProfileBase(ObjectPool* pool, const std::string& name,
-      Counter* total_time_counter, Counter* inactive_timer);
+      Counter* total_time_counter, Counter* inactive_timer,
+      bool add_default_counters = true);
 
   ///  Inserts 'child' before the iterator 'insert_pos' in 'children_'.
   /// 'children_lock_' must be held by the caller.
@@ -434,7 +435,10 @@ class RuntimeProfile : public RuntimeProfileBase {
 
   /// Create a runtime profile object with 'name'. The profile, counters and any other
   /// structures owned by the profile are allocated from 'pool'.
-  static RuntimeProfile* Create(ObjectPool* pool, const std::string& name);
+  /// If add_default_counters is false, TotalTime and InactiveTotalTime will be
+  /// hidden in the newly created RuntimeProfile node.
+  static RuntimeProfile* Create(
+      ObjectPool* pool, const std::string& name, bool add_default_counters = true);
 
   ~RuntimeProfile();
 
@@ -479,7 +483,10 @@ class RuntimeProfile : public RuntimeProfileBase {
   /// are updated. Counters that do not already exist are created.
   /// Info strings matched up by key and are updated or added, depending on whether
   /// the key has already been registered.
-  void Update(const TRuntimeProfileTree& thrift_profile);
+  /// If add_default_counters is false, TotalTime and InactiveTotalTime will be
+  /// hidden in any newly created RuntimeProfile node.
+  void Update(
+      const TRuntimeProfileTree& thrift_profile, bool add_default_counters = true);
 
   /// Add a counter with 'name'/'unit'.  Returns a counter object that the caller can
   /// update.  The counter is owned by the RuntimeProfile object.
@@ -781,11 +788,15 @@ class RuntimeProfile : public RuntimeProfileBase {
   mutable SpinLock t_exec_summary_lock_;
 
   /// Constructor used by Create().
-  RuntimeProfile(ObjectPool* pool, const std::string& name);
+  RuntimeProfile(
+      ObjectPool* pool, const std::string& name, bool add_default_counters = true);
 
   /// Update a subtree of profiles from nodes, rooted at *idx.
   /// On return, *idx points to the node immediately following this subtree.
-  void Update(const std::vector<TRuntimeProfileNode>& nodes, int* idx);
+  /// If add_default_counters is false, TotalTime and InactiveTotalTime will be
+  /// hidden in any newly created RuntimeProfile node.
+  void Update(
+      const std::vector<TRuntimeProfileNode>& nodes, int* idx, bool add_default_counters);
 
   /// Send exec_summary to thrift
   void ExecSummaryToThrift(TRuntimeProfileTree* tree) const;
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index 77408ba6f..8c3f20797 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -638,6 +638,9 @@ struct TExecRequest {
 
   // Coordinator time when plan was submitted by external frontend
   16: optional i64 remote_submit_time
+
+  // Additional profile nodes to be displayed nested right under 'profile' field.
+  17: optional list<RuntimeProfile.TRuntimeProfileNode> profile_children
 }
 
 // Parameters to FeSupport.cacheJar().
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 c50877cbf..a3b9a21e5 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -167,6 +167,11 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   // the original plan vs the ProcessingCost based plan.
   private int originalInstanceCount_ = -1;
 
+  // Information about any cpu comparison that was made (if any) at this fragment.
+  // Set in maxCore().
+  private int thisTreeCpuCore_ = -1;
+  private int subtreeCpuCore_ = -1;
+
   public long getProducedRuntimeFiltersMemReservationBytes() {
     return producedRuntimeFiltersMemReservationBytes_;
   }
@@ -231,7 +236,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
    * <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]
+   * segment-costs=[34550429, 2159270, 23752870, 1]
    * 08:TOP-N [LIMIT=100]
    * |  cost=900
    * |
@@ -727,8 +732,17 @@ public class PlanFragment extends TreeNode<PlanFragment> {
       } else {
         builder.append(getCostBasedMaxParallelism());
       }
-      builder.append(" fragment-costs=");
+      builder.append(" segment-costs=");
       builder.append(costingSegmentSummary());
+      if (thisTreeCpuCore_ > 0 && subtreeCpuCore_ > 0) {
+        builder.append(" cpu-comparison-result=");
+        builder.append(Math.max(thisTreeCpuCore_, subtreeCpuCore_));
+        builder.append(" [max(");
+        builder.append(thisTreeCpuCore_);
+        builder.append(" (self) vs ");
+        builder.append(subtreeCpuCore_);
+        builder.append(" (sum children))]");
+      }
       builder.append("\n");
       if (explainLevel.ordinal() >= TExplainLevel.VERBOSE.ordinal()) {
         builder.append(explainProcessingCosts(detailPrefix, false));
@@ -1098,12 +1112,14 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     fragmentCoreState.put(getId(), Pair.create(coreReq, subtreeCoreBuilder.build()));
   }
 
-  protected CoreCount maxCore(CoreCount core1, CoreCount core2) {
+  protected CoreCount maxCore(CoreCount thisTreeCpuCore, CoreCount subtreeCpuCore) {
+    thisTreeCpuCore_ = thisTreeCpuCore.total();
+    subtreeCpuCore_ = subtreeCpuCore.total();
     if (LOG.isTraceEnabled()) {
-      LOG.trace("At {}, compare {} ({}) vs {} ({})", getId(), core1, core1.total(), core2,
-          core2.total());
+      LOG.trace("At {}, compare {} ({}) vs {} ({})", getId(), thisTreeCpuCore,
+          thisTreeCpuCore.total(), subtreeCpuCore, subtreeCpuCore.total());
     }
-    return CoreCount.max(core1, core2);
+    return CoreCount.max(thisTreeCpuCore, subtreeCpuCore);
   }
 
   /**
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 a12b08824..cdded30c4 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -308,15 +308,6 @@ 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.
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 cde002df3..28f097f3a 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
 import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.math.IntMath;
@@ -36,6 +37,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -49,6 +51,7 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
@@ -153,6 +156,7 @@ import org.apache.impala.thrift.TColumn;
 import org.apache.impala.thrift.TColumnValue;
 import org.apache.impala.thrift.TCommentOnParams;
 import org.apache.impala.thrift.TCopyTestCaseReq;
+import org.apache.impala.thrift.TCounter;
 import org.apache.impala.thrift.TCreateDropRoleParams;
 import org.apache.impala.thrift.TDdlExecRequest;
 import org.apache.impala.thrift.TDdlQueryOptions;
@@ -184,12 +188,14 @@ import org.apache.impala.thrift.TResetMetadataRequest;
 import org.apache.impala.thrift.TResultRow;
 import org.apache.impala.thrift.TResultSet;
 import org.apache.impala.thrift.TResultSetMetadata;
+import org.apache.impala.thrift.TRuntimeProfileNode;
 import org.apache.impala.thrift.TShowFilesParams;
 import org.apache.impala.thrift.TShowStatsOp;
 import org.apache.impala.thrift.TStmtType;
 import org.apache.impala.thrift.TTableName;
 import org.apache.impala.thrift.TTruncateParams;
 import org.apache.impala.thrift.TUniqueId;
+import org.apache.impala.thrift.TUnit;
 import org.apache.impala.thrift.TUpdateCatalogCacheRequest;
 import org.apache.impala.thrift.TUpdateCatalogCacheResponse;
 import org.apache.impala.util.AcidUtils;
@@ -236,6 +242,16 @@ public class Frontend {
   // (i.e., ExecutorMembershipSnapshot.numExecutors_ == 0).
   private static final String DEFAULT_POOL_NAME = "default-pool";
 
+  // Labels for various query profile counters.
+  private static final String EXECUTOR_GROUPS_CONSIDERED = "ExecutorGroupsConsidered";
+  private static final String CPU_COUNT_DIVISOR = "CpuCountDivisor";
+  private static final String EFFECTIVE_PARALLELISM = "EffectiveParallelism";
+  private static final String VERDICT = "Verdict";
+  private static final String MEMORY_MAX = "MemoryMax";
+  private static final String MEMORY_ASK = "MemoryAsk";
+  private static final String CPU_MAX = "CpuMax";
+  private static final String CPU_ASK = "CpuAsk";
+
   /**
    * Plan-time context that allows capturing various artifacts created
    * during the process.
@@ -1807,6 +1823,7 @@ public class Frontend {
       timeline.markEvent("Planning finished");
       result.setTimeline(timeline.toThrift());
       result.setProfile(FrontendProfile.getCurrent().emitAsThrift());
+      result.setProfile_children(FrontendProfile.getCurrent().emitChildrenAsThrift());
       return result;
     }
   }
@@ -1990,6 +2007,11 @@ public class Frontend {
     // Capture the current state.
     planCtx.compilationState_.captureState();
 
+    if (ProcessingCost.isComputeCost(queryOptions)) {
+      FrontendProfile.getCurrent().setToCounter(CPU_COUNT_DIVISOR, TUnit.DOUBLE_VALUE,
+          Double.doubleToLongBits(BackendConfig.INSTANCE.getQueryCpuCountDivisor()));
+    }
+
     TExecutorGroupSet group_set = null;
     String reason = "Unknown";
     int attempt = 0;
@@ -1997,6 +2019,8 @@ public class Frontend {
       group_set = executorGroupSetsToUse.get(i);
       planCtx.compilationState_.setGroupSet(group_set);
       LOG.info("Consider executor group set: " + group_set);
+      FrontendProfile.getCurrent().addToCounter(
+          EXECUTOR_GROUPS_CONSIDERED, TUnit.UNIT, 1);
 
       String retryMsg = "";
       while (true) {
@@ -2034,6 +2058,17 @@ public class Frontend {
         break;
       }
 
+      // Counters about this group set.
+      String profileName = "Executor group " + (i + 1);
+      if (group_set.isSetExec_group_name_prefix()
+          && !group_set.getExec_group_name_prefix().isEmpty()) {
+        profileName += " (" + group_set.getExec_group_name_prefix() + ")";
+      }
+      TRuntimeProfileNode groupSetProfile = createTRuntimeProfileNode(profileName);
+      addCounter(groupSetProfile,
+          new TCounter(MEMORY_MAX, TUnit.BYTES, group_set.getMax_mem_limit()));
+      FrontendProfile.getCurrent().addChildrenProfile(groupSetProfile);
+
       // Find out the per host memory estimated from two possible sources.
       long per_host_mem_estimate = -1;
       int cores_requirement = -1;
@@ -2048,21 +2083,33 @@ public class Frontend {
       }
 
       Preconditions.checkState(per_host_mem_estimate >= 0);
+      boolean memReqSatisfied = per_host_mem_estimate <= group_set.getMax_mem_limit();
+      addCounter(
+          groupSetProfile, new TCounter(MEMORY_ASK, TUnit.BYTES, per_host_mem_estimate));
+
       boolean cpuReqSatisfied = true;
       int scaled_cores_requirement = -1;
+      int available_cores = -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);
+        available_cores = expectedTotalCores(group_set);
+        cpuReqSatisfied = scaled_cores_requirement <= available_cores;
+        addCounter(groupSetProfile, new TCounter(CPU_MAX, TUnit.UNIT, available_cores));
+        addCounter(
+            groupSetProfile, new TCounter(CPU_ASK, TUnit.UNIT, scaled_cores_requirement));
+        addCounter(groupSetProfile,
+            new TCounter(EFFECTIVE_PARALLELISM, TUnit.UNIT, cores_requirement));
       }
 
-      if (per_host_mem_estimate <= group_set.getMax_mem_limit() && cpuReqSatisfied) {
+      if (memReqSatisfied && cpuReqSatisfied) {
         reason = "suitable group found (estimated per-host memory="
             + PrintUtils.printBytes(per_host_mem_estimate)
             + ", estimated cpu cores required=" + cores_requirement
             + ", scaled cpu cores=" + scaled_cores_requirement + ")";
+        addInfoString(groupSetProfile, VERDICT, "Match");
 
         // Set the group name prefix in both the returned query options and
         // the query context for non default group setup.
@@ -2077,12 +2124,36 @@ public class Frontend {
         break;
       }
 
+      List<String> verdicts = Lists.newArrayListWithCapacity(2);
+      List<String> reasons = Lists.newArrayListWithCapacity(2);
+      if (!memReqSatisfied) {
+        String verdict = "not enough per-host memory";
+        verdicts.add(verdict);
+        reasons.add(verdict + " (require=" + per_host_mem_estimate
+            + ", max=" + group_set.getMax_mem_limit() + ")");
+      }
+      if (!cpuReqSatisfied) {
+        String verdict = "not enough cpu cores";
+        verdicts.add(verdict);
+        reasons.add(verdict + " (require=" + scaled_cores_requirement
+            + ", max=" + available_cores + ")");
+      }
+      reason = String.join(", ", reasons);
+      addInfoString(groupSetProfile, VERDICT, String.join(", ", verdicts));
+      group_set = null;
+
       // Restore to the captured state.
       planCtx.compilationState_.restoreState();
     }
 
-    if (reason.equals("Unknown") && group_set.getMax_mem_limit() > 0) {
-      throw new AnalysisException("The query does not fit any executor group sets.");
+    if (group_set == null) {
+      if (reason.equals("Unknown")) {
+        throw new AnalysisException("The query does not fit any executor group sets.");
+      } else {
+        throw new AnalysisException(
+            "The query does not fit largest executor group sets. Reason: " + reason
+            + ".");
+      }
     }
 
     LOG.info("Selected executor group: " + group_set + ", reason: " + reason);
@@ -2093,6 +2164,36 @@ public class Frontend {
     return req;
   }
 
+  private static TRuntimeProfileNode createTRuntimeProfileNode(
+      String childrenProfileName) {
+    return new TRuntimeProfileNode(childrenProfileName,
+        /*num_children=*/0,
+        /*counters=*/new ArrayList<>(),
+        /*metadata=*/-1L,
+        /*indent=*/true,
+        /*info_strings=*/new HashMap<>(),
+        /*info_strings_display_order*/ new ArrayList<>(),
+        /*child_counters_map=*/ImmutableMap.of("", new HashSet<>()));
+  }
+
+  /**
+   * Add counter into node profile.
+   * <p>
+   * Caller must make sure that there is no other counter existing in node profile that
+   * share the same counter name.
+   */
+  private static void addCounter(TRuntimeProfileNode node, TCounter counter) {
+    Preconditions.checkNotNull(node.child_counters_map.get(""));
+    node.addToCounters(counter);
+    node.child_counters_map.get("").add(counter.getName());
+  }
+
+  private static void addInfoString(TRuntimeProfileNode node, String key, String value) {
+    if (node.getInfo_strings().put(key, value) == null) {
+      node.getInfo_strings_display_order().add(key);
+    }
+  }
+
   private TExecRequest doCreateExecRequest(PlanCtx planCtx,
       EventSequence timeline) throws ImpalaException {
     TQueryCtx queryCtx = planCtx.getQueryContext();
diff --git a/fe/src/main/java/org/apache/impala/service/FrontendProfile.java b/fe/src/main/java/org/apache/impala/service/FrontendProfile.java
index 8a471ccaa..9deb138e8 100644
--- a/fe/src/main/java/org/apache/impala/service/FrontendProfile.java
+++ b/fe/src/main/java/org/apache/impala/service/FrontendProfile.java
@@ -19,12 +19,15 @@ package org.apache.impala.service;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.TreeMap;
 
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
 
+import com.google.common.collect.Lists;
 import org.apache.impala.thrift.TCounter;
 import org.apache.impala.thrift.TRuntimeProfileNode;
 import org.apache.impala.thrift.TUnit;
@@ -58,6 +61,12 @@ public class FrontendProfile {
   @GuardedBy("this")
   private final Map<String, TCounter> countersByName_ = new HashMap<>();
 
+  /**
+   * Additional profile node to be displayed under {@link #profile_}.
+   */
+  @GuardedBy("this")
+  private Map<String, TRuntimeProfileNode> childrenProfiles_ = new TreeMap<>();
+
   FrontendProfile() {
     profile_ = new TRuntimeProfileNode("Frontend",
         /*num_children=*/ 0,
@@ -100,9 +109,11 @@ public class FrontendProfile {
   }
 
   /**
-   * Return the profile in Thrift format. This may be called only once, and after it is
-   * called, no further methods may be used on this PlannerProfile object. Any attempts
-   * to do so will result in IllegalStateExceptions.
+   * Return the Frontend profile in Thrift format.
+   * <p>
+   * This may be called only once, and after it is called, no further methods may be used
+   * on this PlannerProfile object except {@link #emitChildrenAsThrift()}. Any attempts to
+   * do so will result in IllegalStateExceptions.
    */
   public synchronized TRuntimeProfileNode emitAsThrift() {
     Preconditions.checkState(profile_ != null, "already emitted profile");
@@ -111,6 +122,22 @@ public class FrontendProfile {
     return ret;
   }
 
+  /**
+   * Return the Frontend's children profiles in Thrift format.
+   * <p>
+   * {@link #emitAsThrift()} must be called ahead of this method.
+   * This may be called only once, and after it is called, no further methods may be used
+   * on this PlannerProfile object. Any attempts to do so will result in
+   * IllegalStateExceptions.
+   */
+  public synchronized List<TRuntimeProfileNode> emitChildrenAsThrift() {
+    Preconditions.checkState(profile_ == null, "emitAsThrift() must be called first");
+    Preconditions.checkState(childrenProfiles_ != null, "already emitted profile");
+    List<TRuntimeProfileNode> ret = Lists.newArrayList(childrenProfiles_.values());
+    childrenProfiles_ = null;
+    return ret;
+  }
+
   /**
    * Add an informational key/value string pair to the profile. These are written out
    * as is to the user. Subsequent calls with the same key will overwrite previous ones.
@@ -154,11 +181,17 @@ public class FrontendProfile {
   }
 
   /**
-   * Add 'delta' to the counter with the given name and unit. Counters are created
-   * on-demand.
+   * Add 'child' profile under 'Frontend' profile node in query profile.
    */
-  public synchronized void addToCounter(String name, TUnit unit, long delta) {
+  public synchronized void addChildrenProfile(TRuntimeProfileNode child) {
     Preconditions.checkState(profile_ != null, "already emitted profile");
+    Preconditions.checkState(childrenProfiles_ != null, "already emitted profile");
+    Preconditions.checkNotNull(child.getName());
+    Preconditions.checkArgument(!childrenProfiles_.containsKey(child.getName()));
+    childrenProfiles_.put(child.getName(), child);
+  }
+
+  private TCounter getOrCreateCounter(String name, TUnit unit) {
     TCounter counter = countersByName_.get(Preconditions.checkNotNull(name));
     if (counter == null) {
       // Need to create the counter.
@@ -168,9 +201,28 @@ public class FrontendProfile {
       // Currently we don't support hierarchical counters in the frontend.
       profile_.child_counters_map.get(ROOT_COUNTER_NAME).add(name);
     }
+    return counter;
+  }
+
+  /**
+   * Add 'delta' to the counter with the given name and unit. Counters are created
+   * on-demand.
+   */
+  public synchronized void addToCounter(String name, TUnit unit, long delta) {
+    Preconditions.checkState(profile_ != null, "already emitted profile");
+    TCounter counter = getOrCreateCounter(name, unit);
     counter.value += delta;
   }
 
+  /**
+   * Set 'value' to the counter with the given name. Counters are created
+   * on-demand.
+   */
+  public synchronized void setToCounter(String name, TUnit unit, long value) {
+    Preconditions.checkState(profile_ != null, "already emitted profile");
+    TCounter counter = getOrCreateCounter(name, unit);
+    counter.value = value;
+  }
 
   public static class Scope implements AutoCloseable {
     private final FrontendProfile oldThreadLocalValue_;
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
index d930c697f..9a0abd0a3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
@@ -27,7 +27,7 @@ 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]
+|  max-parallelism=1 segment-costs=[406] cpu-comparison-result=15 [max(1 (self) vs 15 (sum children))]
 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
@@ -41,7 +41,7 @@ PLAN-ROOT SINK
 |
 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]
+max-parallelism=3 segment-costs=[12331, 300, 6] cpu-comparison-result=15 [max(3 (self) vs 15 (sum children))]
 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
@@ -62,7 +62,7 @@ max-parallelism=3 fragment-costs=[12331, 300, 6]
 |
 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]
+max-parallelism=3 segment-costs=[15408, 167] cpu-comparison-result=15 [max(15 (self) vs 8 (sum children))]
 05:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price)
 |  group by: dt.d_year, item.i_brand, item.i_brand_id
@@ -80,7 +80,7 @@ max-parallelism=3 fragment-costs=[15408, 167]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[6183]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: dt.d_date_sk
@@ -94,7 +94,7 @@ max-parallelism=3 fragment-costs=[15408, 167]
 |  |
 |  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]
+|  max-parallelism=1 segment-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)
@@ -116,7 +116,7 @@ max-parallelism=3 fragment-costs=[15408, 167]
 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]
+max-parallelism=12 segment-costs=[2925614]
 03:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=01
 |  hash predicates: store_sales.ss_item_sk = item.i_item_sk
@@ -127,7 +127,7 @@ max-parallelism=12 fragment-costs=[2925614]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[22]
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: item.i_item_sk
@@ -141,7 +141,7 @@ max-parallelism=12 fragment-costs=[2925614]
 |  |
 |  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]
+|  max-parallelism=1 segment-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)
@@ -274,7 +274,7 @@ 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]
+|  max-parallelism=1 segment-costs=[606] cpu-comparison-result=148 [max(1 (self) vs 148 (sum children))]
 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
@@ -288,7 +288,7 @@ PLAN-ROOT SINK
 |
 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]
+max-parallelism=3 segment-costs=[17011772, 3936100, 400, 6] cpu-comparison-result=148 [max(3 (self) vs 148 (sum children))]
 129:TOP-N [LIMIT=100]
 |  order by: CASE valid_tid(104,105,106,107,108) WHEN 104 THEN channel WHEN 105 THEN channel WHEN 106 THEN channel WHEN 107 THEN channel WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_brand_id WHEN 105 THEN i_brand_id WHEN 106 THEN i_brand_id WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_class_id WHEN 105 THEN i_class_id WHEN 106 THEN NULL WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(10 [...]
 |  mem-estimate=4.69KB mem-reservation=0B thread-reservation=0
@@ -329,7 +329,7 @@ max-parallelism=3 fragment-costs=[17011772, 3936100, 400, 6]
 |
 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]
+max-parallelism=12 segment-costs=[749813, 429878, 215944, 13294224, 142772] cpu-comparison-result=148 [max(12 (self) vs 148 (sum children))]
 127:AGGREGATE [STREAMING]
 |  Class 0
 |    output: sum(sales), sum(number_sales)
@@ -364,7 +364,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=08 plan-id=09 cohort-id=01
 |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -376,7 +376,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  |  212:AGGREGATE [FINALIZE]
 |  |  |  output: avg:merge(quantity * list_price)
 |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -390,7 +390,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[40389768, 1] cpu-comparison-result=12 [max(12 (self) vs 12 (sum children))]
 |  |  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
@@ -412,7 +412,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=11 plan-id=12 cohort-id=04
 |  |  |  |  |  build expressions: d_date_sk
@@ -426,7 +426,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -461,7 +461,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=04
 |  |  |  |  |  build expressions: d_date_sk
@@ -475,7 +475,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -510,7 +510,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=09 plan-id=10 cohort-id=04
 |  |  |  |  build expressions: d_date_sk
@@ -523,7 +523,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -563,7 +563,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  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]
+|  max-parallelism=2 segment-costs=[1033413, 1674] cpu-comparison-result=36 [max(2 (self) vs 36 (sum children))]
 |  114:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price), count(*)
 |  |  group by: i_brand_id, i_class_id, i_category_id
@@ -580,7 +580,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=2 segment-costs=[18397] cpu-comparison-result=30 [max(2 (self) vs 30 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=36 plan-id=37 cohort-id=01
 |  |  |  build expressions: tpcds_parquet.item.i_item_sk
@@ -594,7 +594,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[18186, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  |  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
@@ -608,7 +608,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[448725, 1488000, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  |  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
@@ -625,7 +625,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-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
@@ -639,7 +639,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  88:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |     stored statistics:
@@ -659,7 +659,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=6 [max(3 (self) vs 6 (sum children))]
 |  |  |  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
@@ -672,7 +672,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=2 segment-costs=[448725, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  |  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
@@ -686,7 +686,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=2 segment-costs=[3605351, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  |  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
@@ -703,7 +703,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=2 segment-costs=[7477]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=39 plan-id=40 cohort-id=12
 |  |  |  |  |  build expressions: d3.d_date_sk
@@ -717,7 +717,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -741,7 +741,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=2 segment-costs=[18844]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=40 plan-id=41 cohort-id=12
 |  |  |  |  |  build expressions: iws.i_item_sk
@@ -755,7 +755,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  |  101:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
 |  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |  |     stored statistics:
@@ -786,7 +786,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -799,7 +799,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[448725, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -813,7 +813,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[7224634, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -830,7 +830,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=42 plan-id=43 cohort-id=13
 |  |  |  |  |  build expressions: d2.d_date_sk
@@ -844,7 +844,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -868,7 +868,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=43 plan-id=44 cohort-id=13
 |  |  |  |  |  build expressions: ics.i_item_sk
@@ -882,7 +882,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  |  96:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
 |  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |  |     stored statistics:
@@ -917,7 +917,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[14435775, 2325] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  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
@@ -934,7 +934,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=44 plan-id=45 cohort-id=11
 |  |  |  |  build expressions: d1.d_date_sk
@@ -947,7 +947,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -971,7 +971,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=45 plan-id=46 cohort-id=11
 |  |  |  |  build expressions: iss.i_item_sk
@@ -985,7 +985,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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
@@ -1019,7 +1019,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=2 segment-costs=[18844]
 |  |  JOIN BUILD
 |  |  |  join-table-id=46 plan-id=47 cohort-id=01
 |  |  |  build expressions: i_item_sk
@@ -1033,7 +1033,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -1055,7 +1055,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=2 segment-costs=[112]
 |  |  JOIN BUILD
 |  |  |  join-table-id=47 plan-id=48 cohort-id=01
 |  |  |  build expressions: d_date_sk
@@ -1069,7 +1069,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -1103,7 +1103,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -1115,7 +1115,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  |  184:AGGREGATE [FINALIZE]
 |  |  |  output: avg:merge(quantity * list_price)
 |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -1129,7 +1129,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[40389768, 1] cpu-comparison-result=12 [max(12 (self) vs 12 (sum children))]
 |  |  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
@@ -1151,7 +1151,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=07 plan-id=08 cohort-id=03
 |  |  |  |  |  build expressions: d_date_sk
@@ -1165,7 +1165,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -1200,7 +1200,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=06 plan-id=07 cohort-id=03
 |  |  |  |  |  build expressions: d_date_sk
@@ -1214,7 +1214,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -1249,7 +1249,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=05 plan-id=06 cohort-id=03
 |  |  |  |  build expressions: d_date_sk
@@ -1262,7 +1262,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -1302,7 +1302,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  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]
+|  max-parallelism=3 segment-costs=[2066867, 3333] cpu-comparison-result=38 [max(3 (self) vs 38 (sum children))]
 |  72:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price), count(*)
 |  |  group by: i_brand_id, i_class_id, i_category_id
@@ -1319,7 +1319,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[18608] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=24 plan-id=25 cohort-id=01
 |  |  |  build expressions: tpcds_parquet.item.i_item_sk
@@ -1333,7 +1333,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[18186, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  |  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
@@ -1347,7 +1347,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[448725, 1488000, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  |  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
@@ -1364,7 +1364,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-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
@@ -1378,7 +1378,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  46:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |     stored statistics:
@@ -1398,7 +1398,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=6 [max(3 (self) vs 6 (sum children))]
 |  |  |  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
@@ -1411,7 +1411,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=2 segment-costs=[448725, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  |  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
@@ -1425,7 +1425,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=2 segment-costs=[3605351, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  |  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
@@ -1442,7 +1442,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=2 segment-costs=[7477]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=27 plan-id=28 cohort-id=09
 |  |  |  |  |  build expressions: d3.d_date_sk
@@ -1456,7 +1456,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -1480,7 +1480,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=2 segment-costs=[18844]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=28 plan-id=29 cohort-id=09
 |  |  |  |  |  build expressions: iws.i_item_sk
@@ -1494,7 +1494,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  |  59:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
 |  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |  |     stored statistics:
@@ -1525,7 +1525,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -1538,7 +1538,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[448725, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -1552,7 +1552,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[7224634, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -1569,7 +1569,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=30 plan-id=31 cohort-id=10
 |  |  |  |  |  build expressions: d2.d_date_sk
@@ -1583,7 +1583,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -1607,7 +1607,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=31 plan-id=32 cohort-id=10
 |  |  |  |  |  build expressions: ics.i_item_sk
@@ -1621,7 +1621,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  |  54:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
 |  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |  |     stored statistics:
@@ -1656,7 +1656,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[14435775, 2325] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  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
@@ -1673,7 +1673,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=32 plan-id=33 cohort-id=08
 |  |  |  |  build expressions: d1.d_date_sk
@@ -1686,7 +1686,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -1710,7 +1710,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=33 plan-id=34 cohort-id=08
 |  |  |  |  build expressions: iss.i_item_sk
@@ -1724,7 +1724,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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
@@ -1758,7 +1758,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  JOIN BUILD
 |  |  |  join-table-id=34 plan-id=35 cohort-id=01
 |  |  |  build expressions: i_item_sk
@@ -1772,7 +1772,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -1794,7 +1794,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=35 plan-id=36 cohort-id=01
 |  |  |  build expressions: d_date_sk
@@ -1808,7 +1808,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -1842,7 +1842,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -1854,7 +1854,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  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]
+|  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  156:AGGREGATE [FINALIZE]
 |  |  output: avg:merge(quantity * list_price)
 |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -1868,7 +1868,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  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]
+|  max-parallelism=12 segment-costs=[40389768, 1] cpu-comparison-result=12 [max(12 (self) vs 12 (sum children))]
 |  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
@@ -1890,7 +1890,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=02
 |  |  |  |  build expressions: d_date_sk
@@ -1904,7 +1904,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -1939,7 +1939,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  |  build expressions: d_date_sk
@@ -1953,7 +1953,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -1988,7 +1988,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: d_date_sk
@@ -2001,7 +2001,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -2041,7 +2041,7 @@ max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
 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]
+max-parallelism=12 segment-costs=[4130519, 5813] cpu-comparison-result=38 [max(12 (self) vs 38 (sum children))]
 30:AGGREGATE [STREAMING]
 |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*)
 |  group by: i_brand_id, i_class_id, i_category_id
@@ -2058,7 +2058,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[18608] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=12 plan-id=13 cohort-id=01
 |  |  build expressions: tpcds_parquet.item.i_item_sk
@@ -2072,7 +2072,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[18186, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  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
@@ -2086,7 +2086,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[448725, 1488000, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  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
@@ -2103,7 +2103,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-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
@@ -2117,7 +2117,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[774]
 |  |  04:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |     stored statistics:
@@ -2137,7 +2137,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=6 [max(3 (self) vs 6 (sum children))]
 |  |  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
@@ -2151,7 +2151,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  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]
+|  |  max-parallelism=2 segment-costs=[448725, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  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
@@ -2165,7 +2165,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  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]
+|  |  max-parallelism=2 segment-costs=[3605351, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  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
@@ -2182,7 +2182,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[7477]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=15 plan-id=16 cohort-id=06
 |  |  |  |  build expressions: d3.d_date_sk
@@ -2196,7 +2196,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -2220,7 +2220,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[18844]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=16 plan-id=17 cohort-id=06
 |  |  |  |  build expressions: iws.i_item_sk
@@ -2234,7 +2234,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  17:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |     stored statistics:
@@ -2265,7 +2265,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  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
@@ -2278,7 +2278,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[448725, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  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
@@ -2292,7 +2292,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[7224634, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  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
@@ -2309,7 +2309,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=18 plan-id=19 cohort-id=07
 |  |  |  |  build expressions: d2.d_date_sk
@@ -2323,7 +2323,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -2347,7 +2347,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=19 plan-id=20 cohort-id=07
 |  |  |  |  build expressions: ics.i_item_sk
@@ -2361,7 +2361,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  12:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |     stored statistics:
@@ -2396,7 +2396,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  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]
+|  max-parallelism=12 segment-costs=[14435775, 2325] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  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
@@ -2413,7 +2413,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  JOIN BUILD
 |  |  |  join-table-id=20 plan-id=21 cohort-id=05
 |  |  |  build expressions: d1.d_date_sk
@@ -2426,7 +2426,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -2450,7 +2450,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  JOIN BUILD
 |  |  |  join-table-id=21 plan-id=22 cohort-id=05
 |  |  |  build expressions: iss.i_item_sk
@@ -2465,7 +2465,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -2499,7 +2499,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[19266]
 |  JOIN BUILD
 |  |  join-table-id=22 plan-id=23 cohort-id=01
 |  |  build expressions: i_item_sk
@@ -2514,7 +2514,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  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]
+|  max-parallelism=1 segment-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
@@ -2536,7 +2536,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[114]
 |  JOIN BUILD
 |  |  join-table-id=23 plan-id=24 cohort-id=01
 |  |  build expressions: d_date_sk
@@ -2550,7 +2550,7 @@ max-parallelism=12 fragment-costs=[4130519, 5813]
 |  |
 |  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]
+|  max-parallelism=1 segment-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)
@@ -2711,7 +2711,7 @@ 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]
+|  max-parallelism=1 segment-costs=[61] cpu-comparison-result=106 [max(1 (self) vs 106 (sum children))]
 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
@@ -2725,7 +2725,7 @@ PLAN-ROOT SINK
 |
 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]
+max-parallelism=3 segment-costs=[1366, 833, 1] cpu-comparison-result=106 [max(3 (self) vs 106 (sum children))]
 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
@@ -2742,7 +2742,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[1366, 813] cpu-comparison-result=53 [max(3 (self) vs 53 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: i_brand_id, i_category_id, i_class_id
@@ -2758,7 +2758,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -2770,7 +2770,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  |  150:AGGREGATE [FINALIZE]
 |  |  |  output: avg:merge(quantity * list_price)
 |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -2784,7 +2784,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[40389768, 1] cpu-comparison-result=12 [max(12 (self) vs 12 (sum children))]
 |  |  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
@@ -2806,7 +2806,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=04 plan-id=05 cohort-id=03
 |  |  |  |  |  build expressions: d_date_sk
@@ -2820,7 +2820,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -2855,7 +2855,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  |  |  build expressions: d_date_sk
@@ -2869,7 +2869,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -2904,7 +2904,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: d_date_sk
@@ -2917,7 +2917,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -2956,7 +2956,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  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]
+|  max-parallelism=12 segment-costs=[11579229, 11] cpu-comparison-result=41 [max(12 (self) vs 41 (sum children))]
 |  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
@@ -2973,7 +2973,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[4] cpu-comparison-result=3 [max(3 (self) vs 2 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=02
 |  |  |  build expressions: d_week_seq
@@ -2987,7 +2987,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[1, 1]
 |  |  71:CARDINALITY CHECK
 |  |  |  limit: 1
 |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -3002,7 +3002,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -3026,7 +3026,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[18608] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=06 plan-id=07 cohort-id=02
 |  |  |  build expressions: tpcds_parquet.item.i_item_sk
@@ -3040,7 +3040,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[18186, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  |  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
@@ -3054,7 +3054,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[448725, 1488000, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  |  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
@@ -3071,7 +3071,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-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
@@ -3085,7 +3085,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  48:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |     stored statistics:
@@ -3105,7 +3105,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=6 [max(3 (self) vs 6 (sum children))]
 |  |  |  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
@@ -3118,7 +3118,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=2 segment-costs=[448725, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  |  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
@@ -3132,7 +3132,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=2 segment-costs=[3605351, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  |  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
@@ -3149,7 +3149,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=2 segment-costs=[7477]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=09 plan-id=10 cohort-id=05
 |  |  |  |  |  build expressions: d3.d_date_sk
@@ -3163,7 +3163,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -3187,7 +3187,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=2 segment-costs=[18844]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=05
 |  |  |  |  |  build expressions: iws.i_item_sk
@@ -3201,7 +3201,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  |  61:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
 |  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |  |     stored statistics:
@@ -3232,7 +3232,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -3245,7 +3245,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[448725, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -3259,7 +3259,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[7224634, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  |  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
@@ -3276,7 +3276,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=12 plan-id=13 cohort-id=06
 |  |  |  |  |  build expressions: d2.d_date_sk
@@ -3290,7 +3290,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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)
@@ -3314,7 +3314,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=13 plan-id=14 cohort-id=06
 |  |  |  |  |  build expressions: ics.i_item_sk
@@ -3328,7 +3328,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  |  56:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
 |  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |  |     stored statistics:
@@ -3363,7 +3363,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[14435775, 2325] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  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
@@ -3380,7 +3380,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=14 plan-id=15 cohort-id=04
 |  |  |  |  build expressions: d1.d_date_sk
@@ -3393,7 +3393,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -3417,7 +3417,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=15 plan-id=16 cohort-id=04
 |  |  |  |  build expressions: iss.i_item_sk
@@ -3431,7 +3431,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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
@@ -3465,7 +3465,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[75620]
 |  |  JOIN BUILD
 |  |  |  join-table-id=16 plan-id=17 cohort-id=02
 |  |  |  build expressions: d_date_sk
@@ -3479,7 +3479,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -3501,7 +3501,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  JOIN BUILD
 |  |  |  join-table-id=17 plan-id=18 cohort-id=02
 |  |  |  build expressions: i_item_sk
@@ -3515,7 +3515,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -3548,7 +3548,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=18 plan-id=19 cohort-id=01
 |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -3560,7 +3560,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  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]
+|  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  120:AGGREGATE [FINALIZE]
 |  |  output: avg:merge(quantity * list_price)
 |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -3574,7 +3574,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  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]
+|  max-parallelism=12 segment-costs=[40389768, 1] cpu-comparison-result=12 [max(12 (self) vs 12 (sum children))]
 |  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
@@ -3596,7 +3596,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=21 plan-id=22 cohort-id=07
 |  |  |  |  build expressions: d_date_sk
@@ -3610,7 +3610,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -3645,7 +3645,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=20 plan-id=21 cohort-id=07
 |  |  |  |  build expressions: d_date_sk
@@ -3659,7 +3659,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -3694,7 +3694,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  JOIN BUILD
 |  |  |  join-table-id=19 plan-id=20 cohort-id=07
 |  |  |  build expressions: d_date_sk
@@ -3707,7 +3707,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -3747,7 +3747,7 @@ max-parallelism=3 fragment-costs=[1366, 833, 1]
 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]
+max-parallelism=12 segment-costs=[11579229, 11] cpu-comparison-result=41 [max(12 (self) vs 41 (sum children))]
 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
@@ -3764,7 +3764,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[4] cpu-comparison-result=3 [max(3 (self) vs 2 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=22 plan-id=23 cohort-id=01
 |  |  build expressions: d_week_seq
@@ -3778,7 +3778,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  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]
+|  max-parallelism=1 segment-costs=[1, 1]
 |  26:CARDINALITY CHECK
 |  |  limit: 1
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
@@ -3793,7 +3793,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  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]
+|  max-parallelism=1 segment-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)
@@ -3817,7 +3817,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[18608] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=23 plan-id=24 cohort-id=01
 |  |  build expressions: tpcds_parquet.item.i_item_sk
@@ -3831,7 +3831,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[18186, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  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
@@ -3845,7 +3845,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[448725, 1488000, 211] cpu-comparison-result=30 [max(3 (self) vs 30 (sum children))]
 |  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
@@ -3862,7 +3862,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-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
@@ -3876,7 +3876,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[774]
 |  |  03:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |     stored statistics:
@@ -3896,7 +3896,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=6 [max(3 (self) vs 6 (sum children))]
 |  |  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
@@ -3909,7 +3909,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  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]
+|  |  max-parallelism=2 segment-costs=[448725, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  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
@@ -3923,7 +3923,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  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]
+|  |  max-parallelism=2 segment-costs=[3605351, 2325] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  |  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
@@ -3940,7 +3940,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[7477]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=26 plan-id=27 cohort-id=09
 |  |  |  |  build expressions: d3.d_date_sk
@@ -3954,7 +3954,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -3978,7 +3978,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[18844]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=27 plan-id=28 cohort-id=09
 |  |  |  |  build expressions: iws.i_item_sk
@@ -3992,7 +3992,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  16:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |     stored statistics:
@@ -4023,7 +4023,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[448725] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  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
@@ -4036,7 +4036,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[448725, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  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
@@ -4050,7 +4050,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[7224634, 2325] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  |  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
@@ -4067,7 +4067,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=29 plan-id=30 cohort-id=10
 |  |  |  |  build expressions: d2.d_date_sk
@@ -4081,7 +4081,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -4105,7 +4105,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=30 plan-id=31 cohort-id=10
 |  |  |  |  build expressions: ics.i_item_sk
@@ -4119,7 +4119,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[774]
 |  |  |  11:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |     stored statistics:
@@ -4154,7 +4154,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  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]
+|  max-parallelism=12 segment-costs=[14435775, 2325] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  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
@@ -4171,7 +4171,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[7563]
 |  |  JOIN BUILD
 |  |  |  join-table-id=31 plan-id=32 cohort-id=08
 |  |  |  build expressions: d1.d_date_sk
@@ -4184,7 +4184,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -4208,7 +4208,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[19266]
 |  |  JOIN BUILD
 |  |  |  join-table-id=32 plan-id=33 cohort-id=08
 |  |  |  build expressions: iss.i_item_sk
@@ -4223,7 +4223,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -4257,7 +4257,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[75620]
 |  JOIN BUILD
 |  |  join-table-id=33 plan-id=34 cohort-id=01
 |  |  build expressions: d_date_sk
@@ -4272,7 +4272,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  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]
+|  max-parallelism=1 segment-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
@@ -4294,7 +4294,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[19266]
 |  JOIN BUILD
 |  |  join-table-id=34 plan-id=35 cohort-id=01
 |  |  build expressions: i_item_sk
@@ -4309,7 +4309,7 @@ max-parallelism=12 fragment-costs=[11579229, 11]
 |  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]
+|  max-parallelism=1 segment-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
@@ -4388,7 +4388,7 @@ 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]
+|  max-parallelism=1 segment-costs=[2, 0] cpu-comparison-result=79 [max(1 (self) vs 79 (sum children))]
 PLAN-ROOT SINK
 |  output exprs: sum(sales)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0 cost=0
@@ -4407,7 +4407,7 @@ PLAN-ROOT SINK
 |
 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]
+max-parallelism=12 segment-costs=[202344, 202344, 584489, 1] cpu-comparison-result=79 [max(12 (self) vs 79 (sum children))]
 49:AGGREGATE
 |  output: sum(sales)
 |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -4428,7 +4428,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=12 segment-costs=[44696] cpu-comparison-result=15 [max(14 (self) vs 15 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=08 plan-id=09 cohort-id=01
 |  |  |  build expressions: ws_bill_customer_sk
@@ -4443,7 +4443,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  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]
+|  |  max-parallelism=2 segment-costs=[780941]
 |  |  47:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  |  hash-table-id=09
 |  |  |  hash predicates: ws_item_sk = i_item_sk
@@ -4453,7 +4453,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[18397] cpu-comparison-result=12 [max(2 (self) vs 12 (sum children))]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=09 plan-id=10 cohort-id=05
 |  |  |  |  build expressions: i_item_sk
@@ -4467,7 +4467,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[18186, 211] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  |  85:AGGREGATE [FINALIZE]
 |  |  |  |  group by: i_item_sk
 |  |  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
@@ -4481,7 +4481,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[9542330, 235454, 211] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  |  46:AGGREGATE [STREAMING]
 |  |  |  |  group by: i_item_sk
 |  |  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
@@ -4504,7 +4504,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  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]
+|  |  |  max-parallelism=12 segment-costs=[14686864, 124166] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  |  32:AGGREGATE [STREAMING]
 |  |  |  |  output: count(*)
 |  |  |  |  group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date
@@ -4522,7 +4522,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[24564]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=06
 |  |  |  |  |  build expressions: i_item_sk
@@ -4536,7 +4536,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[4306]
 |  |  |  |  29:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |  |     stored statistics:
@@ -4557,7 +4557,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[1641]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=11 plan-id=12 cohort-id=06
 |  |  |  |  |  build expressions: d_date_sk
@@ -4571,7 +4571,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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))
@@ -4607,7 +4607,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[112]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=12 plan-id=13 cohort-id=05
 |  |  |  |  build expressions: d_date_sk
@@ -4621,7 +4621,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -4655,7 +4655,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=13 plan-id=14 cohort-id=01
 |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -4667,7 +4667,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  |  77:AGGREGATE [FINALIZE]
 |  |  |  output: max:merge(csales)
 |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -4681,7 +4681,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[202344, 100000, 1] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  43:AGGREGATE
 |  |  |  output: max(sum(ss_quantity * ss_sales_price))
 |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -4703,7 +4703,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[9989034, 2344] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  42:AGGREGATE [STREAMING]
 |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  |  group by: c_customer_sk
@@ -4721,7 +4721,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=14 plan-id=15 cohort-id=07
 |  |  |  |  build expressions: c_customer_sk
@@ -4735,7 +4735,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[1173]
 |  |  |  38:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |  |     stored statistics:
@@ -4756,7 +4756,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[1545]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=15 plan-id=16 cohort-id=07
 |  |  |  |  build expressions: d_date_sk
@@ -4770,7 +4770,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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))
@@ -4811,7 +4811,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  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]
+|  max-parallelism=12 segment-costs=[8674967, 2344] cpu-comparison-result=12 [max(12 (self) vs 4 (sum children))]
 |  36:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  group by: c_customer_sk
@@ -4829,7 +4829,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  JOIN BUILD
 |  |  |  join-table-id=17 plan-id=18 cohort-id=01
 |  |  |  build expressions: c_customer_sk
@@ -4844,7 +4844,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -4877,7 +4877,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |
 |--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]
+|  |  max-parallelism=12 segment-costs=[88975] cpu-comparison-result=16 [max(15 (self) vs 16 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: cs_bill_customer_sk
@@ -4892,7 +4892,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  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]
+|  max-parallelism=3 segment-costs=[1564310]
 |  23:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
 |  |  hash-table-id=01
 |  |  hash predicates: cs_item_sk = i_item_sk
@@ -4902,7 +4902,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[18608] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: i_item_sk
@@ -4916,7 +4916,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[18186, 211] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  66:AGGREGATE [FINALIZE]
 |  |  |  group by: i_item_sk
 |  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
@@ -4930,7 +4930,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[9542330, 235454, 211] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  22:AGGREGATE [STREAMING]
 |  |  |  group by: i_item_sk
 |  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
@@ -4953,7 +4953,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[14686864, 124166] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  08:AGGREGATE [STREAMING]
 |  |  |  output: count(*)
 |  |  |  group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date
@@ -4971,7 +4971,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[24564]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: i_item_sk
@@ -4985,7 +4985,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[4306]
 |  |  |  05:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |  |     stored statistics:
@@ -5006,7 +5006,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[1641]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  |  build expressions: d_date_sk
@@ -5020,7 +5020,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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))
@@ -5056,7 +5056,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=02
 |  |  |  build expressions: d_date_sk
@@ -5070,7 +5070,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -5104,7 +5104,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -5116,7 +5116,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  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]
+|  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  58:AGGREGATE [FINALIZE]
 |  |  output: max:merge(csales)
 |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -5130,7 +5130,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[202344, 100000, 1] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  19:AGGREGATE
 |  |  output: max(sum(ss_quantity * ss_sales_price))
 |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -5152,7 +5152,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  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]
+|  max-parallelism=12 segment-costs=[9989034, 2344] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  18:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  group by: c_customer_sk
@@ -5170,7 +5170,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  JOIN BUILD
 |  |  |  join-table-id=06 plan-id=07 cohort-id=04
 |  |  |  build expressions: c_customer_sk
@@ -5184,7 +5184,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[1173]
 |  |  14:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |     stored statistics:
@@ -5205,7 +5205,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[1545]
 |  |  JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=04
 |  |  |  build expressions: d_date_sk
@@ -5219,7 +5219,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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))
@@ -5260,7 +5260,7 @@ max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
 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]
+max-parallelism=12 segment-costs=[8674967, 2344] cpu-comparison-result=12 [max(12 (self) vs 4 (sum children))]
 12:AGGREGATE [STREAMING]
 |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  group by: c_customer_sk
@@ -5278,7 +5278,7 @@ max-parallelism=12 fragment-costs=[8674967, 2344]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[102346]
 |  JOIN BUILD
 |  |  join-table-id=16 plan-id=17 cohort-id=01
 |  |  build expressions: c_customer_sk
@@ -5293,7 +5293,7 @@ max-parallelism=12 fragment-costs=[8674967, 2344]
 |  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]
+|  max-parallelism=1 segment-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
@@ -5407,7 +5407,7 @@ 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]
+|  max-parallelism=1 segment-costs=[306] cpu-comparison-result=70 [max(1 (self) vs 70 (sum children))]
 PLAN-ROOT SINK
 |  output exprs: c_last_name, c_first_name, sales
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=300
@@ -5421,7 +5421,7 @@ PLAN-ROOT SINK
 |
 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]
+max-parallelism=12 segment-costs=[32154, 29405, 300, 6] cpu-comparison-result=70 [max(12 (self) vs 70 (sum children))]
 53:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, c_first_name ASC, sales ASC
 |  mem-estimate=5.08KB mem-reservation=0B thread-reservation=0
@@ -5448,7 +5448,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[9542330, 264332, 527] cpu-comparison-result=34 [max(3 (self) vs 34 (sum children))]
 |  52:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price)
 |  |  group by: c_last_name, c_first_name
@@ -5466,7 +5466,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[13235] cpu-comparison-result=22 [max(7 (self) vs 22 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=10 plan-id=11 cohort-id=05
 |  |  |  build expressions: ws_item_sk
@@ -5480,7 +5480,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |
 |  |  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]
+|  |  max-parallelism=2 segment-costs=[11619]
 |  |  50:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  |  hash-table-id=11
 |  |  |  hash predicates: ws_bill_customer_sk = customer.c_customer_sk
@@ -5491,7 +5491,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[104294]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=11 plan-id=12 cohort-id=06
 |  |  |  |  build expressions: customer.c_customer_sk
@@ -5505,7 +5505,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[8197]
 |  |  |  28:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |  |     stored statistics:
@@ -5524,7 +5524,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  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]
+|  |  max-parallelism=2 segment-costs=[779889]
 |  |  49:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  |  hash-table-id=12
 |  |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
@@ -5535,7 +5535,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[10782] cpu-comparison-result=16 [max(2 (self) vs 16 (sum children))]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=12 plan-id=13 cohort-id=06
 |  |  |  |  build expressions: c_customer_sk
@@ -5549,7 +5549,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[303907, 391] cpu-comparison-result=16 [max(3 (self) vs 16 (sum children))]
 |  |  |  89:AGGREGATE [FINALIZE]
 |  |  |  |  output: sum:merge(ss_quantity * ss_sales_price), max:merge(tpcds_cmax)
 |  |  |  |  group by: c_customer_sk
@@ -5565,7 +5565,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=12 segment-costs=[11555371, 3907] cpu-comparison-result=16 [max(12 (self) vs 16 (sum children))]
 |  |  |  47:AGGREGATE [STREAMING]
 |  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales))
 |  |  |  |  group by: c_customer_sk
@@ -5581,7 +5581,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=13 plan-id=14 cohort-id=07
 |  |  |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -5593,7 +5593,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  |  |  |  86:AGGREGATE [FINALIZE]
 |  |  |  |  |  output: max:merge(csales)
 |  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -5607,7 +5607,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=3 segment-costs=[202344, 100000, 1] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  |  |  44:AGGREGATE
 |  |  |  |  |  output: max(sum(ss_quantity * ss_sales_price))
 |  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -5629,7 +5629,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |  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]
+|  |  |  |  max-parallelism=12 segment-costs=[9989034, 2344] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  |  |  43:AGGREGATE [STREAMING]
 |  |  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  |  |  |  group by: c_customer_sk
@@ -5647,7 +5647,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |  |
 |  |  |  |  |--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]
+|  |  |  |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  |  |  |  JOIN BUILD
 |  |  |  |  |  |  join-table-id=14 plan-id=15 cohort-id=08
 |  |  |  |  |  |  build expressions: c_customer_sk
@@ -5661,7 +5661,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |  |  |
 |  |  |  |  |  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]
+|  |  |  |  |  max-parallelism=1 segment-costs=[1173]
 |  |  |  |  |  39:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |  |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |  |  |  |     stored statistics:
@@ -5682,7 +5682,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |  |
 |  |  |  |  |--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]
+|  |  |  |  |  |  max-parallelism=3 segment-costs=[1545]
 |  |  |  |  |  JOIN BUILD
 |  |  |  |  |  |  join-table-id=15 plan-id=16 cohort-id=08
 |  |  |  |  |  |  build expressions: d_date_sk
@@ -5696,7 +5696,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |  |  |
 |  |  |  |  |  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]
+|  |  |  |  |  max-parallelism=1 segment-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))
@@ -5732,7 +5732,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=16 plan-id=17 cohort-id=07
 |  |  |  |  |  build expressions: c_customer_sk
@@ -5746,7 +5746,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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
@@ -5780,7 +5780,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=2 segment-costs=[112]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=17 plan-id=18 cohort-id=06
 |  |  |  |  build expressions: d_date_sk
@@ -5794,7 +5794,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -5835,7 +5835,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  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]
+|  max-parallelism=12 segment-costs=[14686864, 124166] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  35:AGGREGATE [STREAMING]
 |  |  output: count(*)
 |  |  group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_parquet.item.i_item_sk, d_date
@@ -5853,7 +5853,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[24564]
 |  |  JOIN BUILD
 |  |  |  join-table-id=18 plan-id=19 cohort-id=05
 |  |  |  build expressions: tpcds_parquet.item.i_item_sk
@@ -5868,7 +5868,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -5890,7 +5890,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[1641]
 |  |  JOIN BUILD
 |  |  |  join-table-id=19 plan-id=20 cohort-id=05
 |  |  |  build expressions: d_date_sk
@@ -5904,7 +5904,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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))
@@ -5944,7 +5944,7 @@ max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
 |
 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]
+max-parallelism=3 segment-costs=[9542330, 267032, 576] cpu-comparison-result=36 [max(3 (self) vs 36 (sum children))]
 26:AGGREGATE [STREAMING]
 |  output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price)
 |  group by: c_last_name, c_first_name
@@ -5962,7 +5962,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[14474] cpu-comparison-result=24 [max(9 (self) vs 24 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: cs_item_sk
@@ -5976,7 +5976,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[12706]
 |  24:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=01
 |  |  hash predicates: cs_bill_customer_sk = customer.c_customer_sk
@@ -5987,7 +5987,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[104294]
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: customer.c_customer_sk
@@ -6001,7 +6001,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[8197]
 |  |  02:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |     stored statistics:
@@ -6020,7 +6020,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  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]
+|  max-parallelism=3 segment-costs=[1561508]
 |  23:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  hash-table-id=02
 |  |  hash predicates: cs_bill_customer_sk = c_customer_sk
@@ -6031,7 +6031,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[11173] cpu-comparison-result=16 [max(3 (self) vs 16 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  build expressions: c_customer_sk
@@ -6045,7 +6045,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[303907, 391] cpu-comparison-result=16 [max(3 (self) vs 16 (sum children))]
 |  |  68:AGGREGATE [FINALIZE]
 |  |  |  output: sum:merge(ss_quantity * ss_sales_price), max:merge(tpcds_cmax)
 |  |  |  group by: c_customer_sk
@@ -6061,7 +6061,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[11555371, 3907] cpu-comparison-result=16 [max(12 (self) vs 16 (sum children))]
 |  |  21:AGGREGATE [STREAMING]
 |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales))
 |  |  |  group by: c_customer_sk
@@ -6077,7 +6077,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[3] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
@@ -6089,7 +6089,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[2, 1] cpu-comparison-result=12 [max(1 (self) vs 12 (sum children))]
 |  |  |  65:AGGREGATE [FINALIZE]
 |  |  |  |  output: max:merge(csales)
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -6103,7 +6103,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=3 segment-costs=[202344, 100000, 1] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  |  18:AGGREGATE
 |  |  |  |  output: max(sum(ss_quantity * ss_sales_price))
 |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
@@ -6125,7 +6125,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |  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]
+|  |  |  max-parallelism=12 segment-costs=[9989034, 2344] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  |  17:AGGREGATE [STREAMING]
 |  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
 |  |  |  |  group by: c_customer_sk
@@ -6143,7 +6143,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=04 plan-id=05 cohort-id=04
 |  |  |  |  |  build expressions: c_customer_sk
@@ -6157,7 +6157,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-costs=[1173]
 |  |  |  |  13:SCAN HDFS [tpcds_parquet.customer, RANDOM]
 |  |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
 |  |  |  |     stored statistics:
@@ -6178,7 +6178,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |  |
 |  |  |  |--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]
+|  |  |  |  |  max-parallelism=3 segment-costs=[1545]
 |  |  |  |  JOIN BUILD
 |  |  |  |  |  join-table-id=05 plan-id=06 cohort-id=04
 |  |  |  |  |  build expressions: d_date_sk
@@ -6192,7 +6192,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |  |  |
 |  |  |  |  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]
+|  |  |  |  max-parallelism=1 segment-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))
@@ -6228,7 +6228,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[102346]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=06 plan-id=07 cohort-id=03
 |  |  |  |  build expressions: c_customer_sk
@@ -6242,7 +6242,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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
@@ -6276,7 +6276,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=02
 |  |  |  build expressions: d_date_sk
@@ -6290,7 +6290,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -6331,7 +6331,7 @@ max-parallelism=3 fragment-costs=[9542330, 267032, 576]
 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]
+max-parallelism=12 segment-costs=[14686864, 124166] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 09:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_parquet.item.i_item_sk, d_date
@@ -6349,7 +6349,7 @@ max-parallelism=12 fragment-costs=[14686864, 124166]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[24564]
 |  JOIN BUILD
 |  |  join-table-id=08 plan-id=09 cohort-id=01
 |  |  build expressions: tpcds_parquet.item.i_item_sk
@@ -6364,7 +6364,7 @@ max-parallelism=12 fragment-costs=[14686864, 124166]
 |  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]
+|  max-parallelism=1 segment-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
@@ -6386,7 +6386,7 @@ max-parallelism=12 fragment-costs=[14686864, 124166]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[1641]
 |  JOIN BUILD
 |  |  join-table-id=09 plan-id=10 cohort-id=01
 |  |  build expressions: d_date_sk
@@ -6400,7 +6400,7 @@ max-parallelism=12 fragment-costs=[14686864, 124166]
 |  |
 |  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]
+|  max-parallelism=1 segment-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))
@@ -6534,7 +6534,7 @@ 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]
+|  max-parallelism=1 segment-costs=[8891, 17696, 35376, 119504, 400, 500] cpu-comparison-result=26 [max(1 (self) vs 26 (sum children))]
 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
@@ -6593,7 +6593,7 @@ PLAN-ROOT SINK
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[86275, 17055, 1266] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  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)))
@@ -6616,7 +6616,7 @@ PLAN-ROOT SINK
 |  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]
+|  max-parallelism=12 segment-costs=[9680620, 1000] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  28:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(coalesce(sr.sr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(sts.ss_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(sr.sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(sts.ss_net_paid, CAST(0 AS DECIMAL(7,2))))
 |  |  group by: sts.ss_item_sk
@@ -6634,7 +6634,7 @@ PLAN-ROOT SINK
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=03
 |  |  |  build expressions: d_date_sk
@@ -6648,7 +6648,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -6673,7 +6673,7 @@ PLAN-ROOT SINK
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[59863]
 |  |  JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=03
 |  |  |  build expressions: sr.sr_item_sk, sr.sr_ticket_number
@@ -6686,7 +6686,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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))
@@ -6752,7 +6752,7 @@ PLAN-ROOT SINK
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[43155, 8531, 634] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  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)))
@@ -6775,7 +6775,7 @@ PLAN-ROOT SINK
 |  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]
+|  max-parallelism=3 segment-costs=[4844813, 500] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 |  17:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(coalesce(cr.cr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(cs.cs_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(cr.cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(cs.cs_net_paid, CAST(0 AS DECIMAL(7,2))))
 |  |  group by: cs.cs_item_sk
@@ -6793,7 +6793,7 @@ PLAN-ROOT SINK
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[114]
 |  |  JOIN BUILD
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  build expressions: d_date_sk
@@ -6807,7 +6807,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -6832,7 +6832,7 @@ PLAN-ROOT SINK
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[29996]
 |  |  JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=02
 |  |  |  build expressions: cr.cr_item_sk, cr.cr_order_number
@@ -6845,7 +6845,7 @@ PLAN-ROOT SINK
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-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))
@@ -6910,7 +6910,7 @@ PLAN-ROOT SINK
 |
 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]
+max-parallelism=2 segment-costs=[21682, 4286, 319] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 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)))
@@ -6933,7 +6933,7 @@ max-parallelism=2 fragment-costs=[21682, 4286, 319]
 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]
+max-parallelism=2 segment-costs=[2417877, 252] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 06:AGGREGATE [STREAMING]
 |  output: sum(CAST(coalesce(wr.wr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(ws.ws_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(wr.wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(ws.ws_net_paid, CAST(0 AS DECIMAL(7,2))))
 |  group by: ws.ws_item_sk
@@ -6951,7 +6951,7 @@ max-parallelism=2 fragment-costs=[2417877, 252]
 |
 |--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]
+|  |  max-parallelism=2 segment-costs=[112]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: d_date_sk
@@ -6965,7 +6965,7 @@ max-parallelism=2 fragment-costs=[2417877, 252]
 |  |
 |  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]
+|  max-parallelism=1 segment-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)
@@ -6990,7 +6990,7 @@ max-parallelism=2 fragment-costs=[2417877, 252]
 |
 |--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]
+|  |  max-parallelism=2 segment-costs=[14746]
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: wr.wr_item_sk, wr.wr_order_number
@@ -7003,7 +7003,7 @@ max-parallelism=2 fragment-costs=[2417877, 252]
 |  |
 |  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]
+|  max-parallelism=1 segment-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))
@@ -7093,7 +7093,7 @@ 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]
+|  max-parallelism=1 segment-costs=[708] cpu-comparison-result=38 [max(1 (self) vs 38 (sum children))]
 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
@@ -7107,7 +7107,7 @@ PLAN-ROOT SINK
 |
 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]
+max-parallelism=3 segment-costs=[972054, 500, 8] cpu-comparison-result=38 [max(3 (self) vs 38 (sum children))]
 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
@@ -7129,7 +7129,7 @@ max-parallelism=3 fragment-costs=[972054, 500, 8]
 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]
+max-parallelism=12 segment-costs=[3597079, 10450] cpu-comparison-result=38 [max(33 (self) vs 38 (sum children))]
 16:AGGREGATE [STREAMING]
 |  output: count(*), sum(ext_sales_price)
 |  group by: channel, col_name, d_year, d_qoy, i_category
@@ -7152,7 +7152,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |
 |  |--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]
+|  |  |  max-parallelism=12 segment-costs=[18526]
 |  |  JOIN BUILD
 |  |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  |  build expressions: i_item_sk
@@ -7166,7 +7166,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[982]
 |  |  12:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |     stored statistics:
@@ -7184,7 +7184,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |
 |  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]
+|  max-parallelism=3 segment-costs=[7613]
 |  14:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash-table-id=05
 |  |  hash predicates: cs_sold_date_sk = d_date_sk
@@ -7195,7 +7195,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[74191]
 |  |  JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=01
 |  |  |  build expressions: d_date_sk
@@ -7209,7 +7209,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |  |
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[1999]
 |  |  13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |     stored statistics:
@@ -7228,7 +7228,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  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]
+|  max-parallelism=3 segment-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
@@ -7251,7 +7251,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |
 |  |--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]
+|  |  |  max-parallelism=3 segment-costs=[194] cpu-comparison-result=4 [max(4 (self) vs 3 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  |  build expressions: ws_item_sk
@@ -7266,7 +7266,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |  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]
+|  |  max-parallelism=1 segment-costs=[73913]
 |  |  09:HASH JOIN [INNER JOIN, BROADCAST]
 |  |  |  hash-table-id=02
 |  |  |  hash predicates: d_date_sk = ws_sold_date_sk
@@ -7277,7 +7277,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=1 segment-costs=[178]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  |  build expressions: ws_sold_date_sk
@@ -7291,7 +7291,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=2 segment-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
@@ -7335,7 +7335,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |
 |--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]
+|  |  max-parallelism=12 segment-costs=[74191]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: d_date_sk
@@ -7349,7 +7349,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |  |
 |  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]
+|  max-parallelism=1 segment-costs=[1999]
 |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |     HDFS partitions=1/1 files=1 size=2.15MB
 |     stored statistics:
@@ -7367,7 +7367,7 @@ max-parallelism=12 fragment-costs=[3597079, 10450]
 |
 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]
+max-parallelism=3 segment-costs=[139926]
 04:HASH JOIN [INNER JOIN, PARTITIONED]
 |  hash-table-id=04
 |  hash predicates: ss_item_sk = i_item_sk
@@ -7378,7 +7378,7 @@ max-parallelism=3 fragment-costs=[139926]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[18526]
 |  JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: i_item_sk
@@ -7392,7 +7392,7 @@ max-parallelism=3 fragment-costs=[139926]
 |  |
 |  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]
+|  max-parallelism=1 segment-costs=[982]
 |  02:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |     HDFS partitions=1/1 files=1 size=1.73MB
 |     stored statistics:
@@ -7411,7 +7411,7 @@ max-parallelism=3 fragment-costs=[139926]
 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]
+max-parallelism=12 segment-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
@@ -7507,7 +7507,7 @@ 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]
+|  max-parallelism=1 segment-costs=[1012] cpu-comparison-result=26 [max(1 (self) vs 26 (sum children))]
 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
@@ -7521,7 +7521,7 @@ PLAN-ROOT SINK
 |
 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]
+max-parallelism=3 segment-costs=[1785038, 884887, 12] cpu-comparison-result=26 [max(3 (self) vs 26 (sum children))]
 20:TOP-N [LIMIT=100]
 |  order by: ss_sold_year ASC, ss_item_sk ASC, ss_customer_sk ASC, ss_qty DESC, ss_wc DESC, ss_sp DESC, coalesce(ws_qty, 0) + coalesce(cs_qty, 0) ASC, coalesce(ws_wc, 0) + coalesce(cs_wc, 0) ASC, coalesce(ws_sp, 0) + coalesce(cs_sp, 0) ASC, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2) ASC
 |  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)
@@ -7540,7 +7540,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[9368] cpu-comparison-result=18 [max(3 (self) vs 18 (sum children))]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: d_year, ss_customer_sk, ss_item_sk
@@ -7554,7 +7554,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |
 |  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]
+|  max-parallelism=2 segment-costs=[896697, 444365] cpu-comparison-result=18 [max(2 (self) vs 18 (sum children))]
 |  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
@@ -7565,7 +7565,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |
 |  |--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]
+|  |  |  max-parallelism=2 segment-costs=[9192] cpu-comparison-result=12 [max(2 (self) vs 12 (sum children))]
 |  |  JOIN BUILD
 |  |  |  join-table-id=01 plan-id=02 cohort-id=02
 |  |  |  build expressions: d_year, ss_customer_sk, ss_item_sk
@@ -7579,7 +7579,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |  |
 |  |  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]
+|  |  max-parallelism=3 segment-costs=[3568694, 177] cpu-comparison-result=12 [max(3 (self) vs 12 (sum children))]
 |  |  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
@@ -7596,7 +7596,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |  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]
+|  |  max-parallelism=12 segment-costs=[12276657, 34514] cpu-comparison-result=12 [max(12 (self) vs 8 (sum children))]
 |  |  05:AGGREGATE [STREAMING]
 |  |  |  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
 |  |  |  group by: d_year, ss_item_sk, ss_customer_sk
@@ -7614,7 +7614,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[388]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
 |  |  |  |  build expressions: d_date_sk
@@ -7628,7 +7628,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-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)
@@ -7653,7 +7653,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |  |
 |  |  |--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]
+|  |  |  |  max-parallelism=3 segment-costs=[591876]
 |  |  |  JOIN BUILD
 |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
 |  |  |  |  build expressions: sr_item_sk, sr_ticket_number
@@ -7666,7 +7666,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |  |  |
 |  |  |  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]
+|  |  |  max-parallelism=1 segment-costs=[10109]
 |  |  |  01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM]
 |  |  |     HDFS partitions=1/1 files=1 size=15.43MB
 |  |  |     stored statistics:
@@ -7704,7 +7704,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  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]
+|  max-parallelism=2 segment-costs=[3071467, 8673] cpu-comparison-result=6 [max(2 (self) vs 6 (sum children))]
 |  11:AGGREGATE [STREAMING]
 |  |  output: sum(CAST(ws_quantity AS BIGINT)), sum(ws_wholesale_cost), sum(ws_sales_price)
 |  |  group by: d_year, ws_item_sk, ws_bill_customer_sk
@@ -7722,7 +7722,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |
 |  |--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]
+|  |  |  max-parallelism=2 segment-costs=[383]
 |  |  JOIN BUILD
 |  |  |  join-table-id=04 plan-id=05 cohort-id=02
 |  |  |  build expressions: d_date_sk
@@ -7737,7 +7737,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |  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]
+|  |  max-parallelism=1 segment-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)
@@ -7763,7 +7763,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |
 |  |--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]
+|  |  |  max-parallelism=2 segment-costs=[146330]
 |  |  JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=02
 |  |  |  build expressions: wr_item_sk, wr_order_number
@@ -7777,7 +7777,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 |  |  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]
+|  |  max-parallelism=1 segment-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
@@ -7815,7 +7815,7 @@ max-parallelism=3 fragment-costs=[1785038, 884887, 12]
 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]
+max-parallelism=3 segment-costs=[6143098, 17264] cpu-comparison-result=8 [max(3 (self) vs 8 (sum children))]
 17:AGGREGATE [STREAMING]
 |  output: sum(CAST(cs_quantity AS BIGINT)), sum(cs_wholesale_cost), sum(cs_sales_price)
 |  group by: d_year, cs_item_sk, cs_bill_customer_sk
@@ -7833,7 +7833,7 @@ max-parallelism=3 fragment-costs=[6143098, 17264]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[388]
 |  JOIN BUILD
 |  |  join-table-id=06 plan-id=07 cohort-id=01
 |  |  build expressions: d_date_sk
@@ -7848,7 +7848,7 @@ max-parallelism=3 fragment-costs=[6143098, 17264]
 |  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]
+|  max-parallelism=1 segment-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)
@@ -7874,7 +7874,7 @@ max-parallelism=3 fragment-costs=[6143098, 17264]
 |
 |--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]
+|  |  max-parallelism=3 segment-costs=[296576]
 |  JOIN BUILD
 |  |  join-table-id=07 plan-id=08 cohort-id=01
 |  |  build expressions: cr_item_sk, cr_order_number
@@ -7888,7 +7888,7 @@ max-parallelism=3 fragment-costs=[6143098, 17264]
 |  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=24.08MB mem-reservation=2.00MB thread-reservation=1
-|  max-parallelism=1 fragment-costs=[5066]
+|  max-parallelism=1 segment-costs=[5066]
 |  13:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM]
 |     HDFS partitions=1/1 files=1 size=10.62MB
 |     runtime filters: RF005[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk, RF002[bloom] -> tpcds_parquet.catalog_returns.cr_item_sk
@@ -8010,7 +8010,7 @@ Max Per-Host Resource Reservation: Memory=358.62MB Threads=42
 Per-Host Resource Estimates: Memory=951MB
 F23: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=[508]
+|  max-parallelism=1 segment-costs=[508] cpu-comparison-result=55 [max(1 (self) vs 55 (sum children))]
 PLAN-ROOT SINK
 |  output exprs: CASE valid_tid(26,27,28) WHEN 26 THEN channel WHEN 27 THEN channel WHEN 28 THEN NULL END, CASE valid_tid(26,27,28) WHEN 26 THEN id WHEN 27 THEN NULL WHEN 28 THEN NULL END, aggif(valid_tid(26,27,28) IN (26, 27, 28), CASE valid_tid(26,27,28) WHEN 26 THEN sum(sales) WHEN 27 THEN sum(sales) WHEN 28 THEN sum(sales) END), aggif(valid_tid(26,27,28) IN (26, 27, 28), CASE valid_tid(26,27,28) WHEN 26 THEN sum(`returns`) WHEN 27 THEN sum(`returns`) WHEN 28 THEN sum(`returns`) END), [...]
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=500
@@ -8024,7 +8024,7 @@ PLAN-ROOT SINK
 |
 F22:PLAN FRAGMENT [HASH(CASE valid_tid(26,27,28) WHEN 26 THEN murmur_hash(channel) WHEN 27 THEN murmur_hash(channel) WHEN 28 THEN murmur_hash(NULL) END,CASE valid_tid(26,27,28) WHEN 26 THEN murmur_hash(id) WHEN 27 THEN murmur_hash(NULL) WHEN 28 THEN murmur_hash(NULL) END)] hosts=3 instances=3 (adjusted from 12)
 Per-Instance Resources: mem-estimate=40.00MB mem-reservation=7.75MB thread-reservation=1
-max-parallelism=3 fragment-costs=[176035, 69384, 200, 8]
+max-parallelism=3 segment-costs=[176035, 69384, 200, 8] cpu-comparison-result=55 [max(3 (self) vs 55 (sum children))]
 39:TOP-N [LIMIT=100]
 |  order by: CASE valid_tid(26,27,28) WHEN 26 THEN channel WHEN 27 THEN channel WHEN 28 THEN NULL END ASC, CASE valid_tid(26,27,28) WHEN 26 THEN id WHEN 27 THEN NULL WHEN 28 THEN NULL END ASC
 |  mem-estimate=7.03KB mem-reservation=0B thread-reservation=0
@@ -8059,7 +8059,7 @@ max-parallelism=3 fragment-costs=[176035, 69384, 200, 8]
 |
 F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
 Per-Instance Resources: mem-estimate=43.19MB mem-reservation=7.94MB thread-reservation=1
-max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
+max-parallelism=12 segment-costs=[25, 47058, 62, 346800, 2575] cpu-comparison-result=55 [max(12 (self) vs 55 (sum children))]
 37:AGGREGATE [STREAMING]
 |  Class 0
 |    output: sum(sales), sum(returns), sum(profit)
@@ -8094,7 +8094,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  F14: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=45.75MB mem-reservation=18.00MB thread-reservation=1
-|  max-parallelism=2 fragment-costs=[2675586, 2]
+|  max-parallelism=2 segment-costs=[2675586, 2] cpu-comparison-result=15 [max(2 (self) vs 15 (sum children))]
 |  36:AGGREGATE [STREAMING]
 |  |  output: sum(ws_ext_sales_price), sum(coalesce(wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(ws_net_profit - coalesce(wr_net_loss, CAST(0 AS DECIMAL(7,2))))
 |  |  group by: web_site_id
@@ -8112,7 +8112,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F34:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=7.77MB mem-reservation=7.75MB thread-reservation=1
-|  |  |  max-parallelism=2 fragment-costs=[34]
+|  |  |  max-parallelism=2 segment-costs=[34]
 |  |  JOIN BUILD
 |  |  |  join-table-id=10 plan-id=11 cohort-id=03
 |  |  |  build expressions: web_site_sk
@@ -8126,7 +8126,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  |
 |  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[3]
+|  |  max-parallelism=1 segment-costs=[3]
 |  |  28:SCAN HDFS [tpcds_parquet.web_site, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=11.91KB
 |  |     stored statistics:
@@ -8147,7 +8147,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F35:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=7.96MB mem-reservation=7.75MB thread-reservation=1
-|  |  |  max-parallelism=2 fragment-costs=[7735]
+|  |  |  max-parallelism=2 segment-costs=[7735]
 |  |  JOIN BUILD
 |  |  |  join-table-id=11 plan-id=12 cohort-id=03
 |  |  |  build expressions: d_date_sk
@@ -8161,7 +8161,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  |
 |  |  F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[148168]
+|  |  max-parallelism=1 segment-costs=[148168]
 |  |  27:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |     predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04'
@@ -8185,7 +8185,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F36: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=[308]
+|  |  |  max-parallelism=2 segment-costs=[308]
 |  |  JOIN BUILD
 |  |  |  join-table-id=12 plan-id=13 cohort-id=03
 |  |  |  build expressions: p_promo_sk
@@ -8199,7 +8199,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  |
 |  |  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=32.00KB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[312]
+|  |  max-parallelism=1 segment-costs=[312]
 |  |  30:SCAN HDFS [tpcds_parquet.promotion, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=23.30KB
 |  |     predicates: p_channel_tv = 'N'
@@ -8223,7 +8223,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F37:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=8.79MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  |  max-parallelism=2 fragment-costs=[1858]
+|  |  |  max-parallelism=2 segment-costs=[1858]
 |  |  JOIN BUILD
 |  |  |  join-table-id=13 plan-id=14 cohort-id=03
 |  |  |  build expressions: i_item_sk
@@ -8237,7 +8237,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  |
 |  |  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[18240]
+|  |  max-parallelism=1 segment-costs=[18240]
 |  |  29:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |     predicates: i_current_price > CAST(50 AS DECIMAL(3,0))
@@ -8261,7 +8261,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F38:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  |  |  Per-Instance Resources: mem-estimate=20.67MB mem-reservation=19.00MB thread-reservation=1
-|  |  |  max-parallelism=2 fragment-costs=[147452]
+|  |  |  max-parallelism=2 segment-costs=[147452]
 |  |  JOIN BUILD
 |  |  |  join-table-id=14 plan-id=15 cohort-id=03
 |  |  |  build expressions: wr_item_sk, wr_order_number
@@ -8275,7 +8275,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  F15: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.11MB mem-reservation=2.00MB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[3645]
+|  |  max-parallelism=1 segment-costs=[3645]
 |  |  26:SCAN HDFS [tpcds_parquet.web_returns, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=5.66MB
 |  |     runtime filters: RF023[min_max] -> tpcds_parquet.web_returns.wr_item_sk, RF022[bloom] -> tpcds_parquet.web_returns.wr_item_sk
@@ -8313,7 +8313,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  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=61.75MB mem-reservation=18.00MB thread-reservation=1
-|  max-parallelism=3 fragment-costs=[5361501, 902]
+|  max-parallelism=3 segment-costs=[5361501, 902] cpu-comparison-result=20 [max(3 (self) vs 20 (sum children))]
 |  24:AGGREGATE [STREAMING]
 |  |  output: sum(cs_ext_sales_price), sum(coalesce(cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(cs_net_profit - coalesce(cr_net_loss, CAST(0 AS DECIMAL(7,2))))
 |  |  group by: cp_catalog_page_id
@@ -8331,7 +8331,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=9.14MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  |  max-parallelism=3 fragment-costs=[12954]
+|  |  |  max-parallelism=3 segment-costs=[12954]
 |  |  JOIN BUILD
 |  |  |  join-table-id=05 plan-id=06 cohort-id=02
 |  |  |  build expressions: cp_catalog_page_sk
@@ -8345,7 +8345,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  |
 |  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.14MB mem-reservation=256.00KB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[779]
+|  |  max-parallelism=1 segment-costs=[779]
 |  |  16:SCAN HDFS [tpcds_parquet.catalog_page, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=739.17KB
 |  |     stored statistics:
@@ -8366,7 +8366,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=8.96MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  |  max-parallelism=3 fragment-costs=[7950]
+|  |  |  max-parallelism=3 segment-costs=[7950]
 |  |  JOIN BUILD
 |  |  |  join-table-id=06 plan-id=07 cohort-id=02
 |  |  |  build expressions: d_date_sk
@@ -8380,7 +8380,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  |
 |  |  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[148168]
+|  |  max-parallelism=1 segment-costs=[148168]
 |  |  15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=2.15MB
 |  |     predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04'
@@ -8404,7 +8404,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F31: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=[315]
+|  |  |  max-parallelism=3 segment-costs=[315]
 |  |  JOIN BUILD
 |  |  |  join-table-id=07 plan-id=08 cohort-id=02
 |  |  |  build expressions: p_promo_sk
@@ -8418,7 +8418,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  |
 |  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=32.00KB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[312]
+|  |  max-parallelism=1 segment-costs=[312]
 |  |  18:SCAN HDFS [tpcds_parquet.promotion, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=23.30KB
 |  |     predicates: p_channel_tv = 'N'
@@ -8442,7 +8442,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F32:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=8.79MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  |  max-parallelism=3 fragment-costs=[1887]
+|  |  |  max-parallelism=3 segment-costs=[1887]
 |  |  JOIN BUILD
 |  |  |  join-table-id=08 plan-id=09 cohort-id=02
 |  |  |  build expressions: i_item_sk
@@ -8456,7 +8456,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  |
 |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[18240]
+|  |  max-parallelism=1 segment-costs=[18240]
 |  |  17:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=1.73MB
 |  |     predicates: i_current_price > CAST(50 AS DECIMAL(3,0))
@@ -8480,7 +8480,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |
 |  |--F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=37.32MB mem-reservation=34.00MB thread-reservation=1
-|  |  |  max-parallelism=3 fragment-costs=[299954]
+|  |  |  max-parallelism=3 segment-costs=[299954]
 |  |  JOIN BUILD
 |  |  |  join-table-id=09 plan-id=10 cohort-id=02
 |  |  |  build expressions: cr_item_sk, cr_order_number
@@ -8494,7 +8494,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
 |  |  Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1
-|  |  max-parallelism=1 fragment-costs=[7317]
+|  |  max-parallelism=1 segment-costs=[7317]
 |  |  14:SCAN HDFS [tpcds_parquet.catalog_returns, RANDOM]
 |  |     HDFS partitions=1/1 files=1 size=10.62MB
 |  |     runtime filters: RF015[min_max] -> tpcds_parquet.catalog_returns.cr_item_sk, RF014[bloom] -> tpcds_parquet.catalog_returns.cr_item_sk
@@ -8532,7 +8532,7 @@ max-parallelism=12 fragment-costs=[25, 47058, 62, 346800, 2575]
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
 Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB
 Per-Instance Resources: mem-estimate=29.75MB mem-reservation=6.00MB thread-reservation=1
-max-parallelism=12 fragment-costs=[10712984, 1]
+max-parallelism=12 segment-costs=[10712984, 1] cpu-comparison-result=20 [max(12 (self) vs 20 (sum children))]
 12:AGGREGATE [STREAMING]
 |  output: sum(ss_ext_sales_price), sum(coalesce(sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(ss_net_profit - coalesce(sr_net_loss, CAST(0 AS DECIMAL(7,2))))
 |  group by: s_store_id
@@ -8550,7 +8550,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |
 |--F24: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=[15]
+|  |  max-parallelism=3 segment-costs=[15]
 |  JOIN BUILD
 |  |  join-table-id=00 plan-id=01 cohort-id=01
 |  |  build expressions: s_store_sk
@@ -8564,7 +8564,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |  |
 |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.14MB mem-reservation=16.00KB thread-reservation=1
-|  max-parallelism=1 fragment-costs=[2]
+|  max-parallelism=1 segment-costs=[2]
 |  04:SCAN HDFS [tpcds_parquet.store, RANDOM]
 |     HDFS partitions=1/1 files=1 size=9.93KB
 |     stored statistics:
@@ -8585,7 +8585,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |
 |--F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=8.96MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  max-parallelism=3 fragment-costs=[7950]
+|  |  max-parallelism=3 segment-costs=[7950]
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: d_date_sk
@@ -8599,7 +8599,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |  |
 |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.12MB mem-reservation=1.00MB thread-reservation=1
-|  max-parallelism=1 fragment-costs=[148168]
+|  max-parallelism=1 segment-costs=[148168]
 |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
 |     HDFS partitions=1/1 files=1 size=2.15MB
 |     predicates: CAST(d_date AS DATE) <= DATE '1998-09-03', CAST(d_date AS DATE) >= DATE '1998-08-04'
@@ -8623,7 +8623,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |
 |--F26: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=[315]
+|  |  max-parallelism=3 segment-costs=[315]
 |  JOIN BUILD
 |  |  join-table-id=02 plan-id=03 cohort-id=01
 |  |  build expressions: p_promo_sk
@@ -8637,7 +8637,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |  |
 |  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=32.00KB thread-reservation=1
-|  max-parallelism=1 fragment-costs=[312]
+|  max-parallelism=1 segment-costs=[312]
 |  06:SCAN HDFS [tpcds_parquet.promotion, RANDOM]
 |     HDFS partitions=1/1 files=1 size=23.30KB
 |     predicates: p_channel_tv = 'N'
@@ -8661,7 +8661,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |
 |--F27:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=8.79MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
-|  |  max-parallelism=3 fragment-costs=[1887]
+|  |  max-parallelism=3 segment-costs=[1887]
 |  JOIN BUILD
 |  |  join-table-id=03 plan-id=04 cohort-id=01
 |  |  build expressions: i_item_sk
@@ -8675,7 +8675,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |  |
 |  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=256.00KB thread-reservation=1
-|  max-parallelism=1 fragment-costs=[18240]
+|  max-parallelism=1 segment-costs=[18240]
 |  05:SCAN HDFS [tpcds_parquet.item, RANDOM]
 |     HDFS partitions=1/1 files=1 size=1.73MB
 |     predicates: i_current_price > CAST(50 AS DECIMAL(3,0))
@@ -8699,7 +8699,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |
 |--F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  |  Per-Instance Resources: mem-estimate=74.61MB mem-reservation=68.00MB thread-reservation=1
-|  |  max-parallelism=3 fragment-costs=[598614]
+|  |  max-parallelism=3 segment-costs=[598614]
 |  JOIN BUILD
 |  |  join-table-id=04 plan-id=05 cohort-id=01
 |  |  build expressions: sr_item_sk, sr_ticket_number
@@ -8713,7 +8713,7 @@ max-parallelism=12 fragment-costs=[10712984, 1]
 |  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=24.11MB mem-reservation=8.00MB thread-reservation=1
-|  max-parallelism=1 fragment-costs=[14601]
+|  max-parallelism=1 segment-costs=[14601]
 |  02:SCAN HDFS [tpcds_parquet.store_returns, RANDOM]
 |     HDFS partitions=1/1 files=1 size=15.43MB
 |     runtime filters: RF007[min_max] -> tpcds_parquet.store_returns.sr_item_sk, RF006[bloom] -> tpcds_parquet.store_returns.sr_item_sk
diff --git a/tests/custom_cluster/test_executor_groups.py b/tests/custom_cluster/test_executor_groups.py
index 79d6d5c6d..7df6c27bb 100644
--- a/tests/custom_cluster/test_executor_groups.py
+++ b/tests/custom_cluster/test_executor_groups.py
@@ -33,6 +33,9 @@ LOG = logging.getLogger("test_auto_scaling")
 # Non-trivial query that gets scheduled on all executors within a group.
 TEST_QUERY = "select count(*) from functional.alltypes where month + random() < 3"
 
+# A query to test Cpu requirement. Estimated memory per host is 37MB.
+CPU_TEST_QUERY = "select * from tpcds_parquet.store_sales where ss_item_sk = 1 limit 50;"
+
 DEFAULT_RESOURCE_POOL = "default-pool"
 
 
@@ -785,7 +788,8 @@ class TestExecutorGroups(CustomClusterTestSuite):
     # Force to run the large query on the small group should fail
     self.client.set_configuration({'request_pool': 'small'})
     result = self.execute_query_expect_failure(self.client, LARGE_QUERY)
-    assert "The query does not fit any executor group set" in str(result)
+    assert ("The query does not fit largest executor group sets. "
+        "Reason: not enough per-host memory") in str(result)
 
     self.client.close()
 
@@ -845,24 +849,28 @@ class TestExecutorGroups(CustomClusterTestSuite):
     self.client.close()
 
   @pytest.mark.execute_serially
-  def test_query_cpu_count_divisor(self):
-    # A query with estimated memory per host of 37MB.
-    TEST_QUERY = "select * from tpcds_parquet.store_sales where ss_item_sk = 1 limit 50;"
-
+  def test_query_cpu_count_divisor_default(self):
     # Expect to run the query on the small group by default.
     coordinator_test_args = ""
-    self._run_with_compute_processing_cost(coordinator_test_args, TEST_QUERY,
-        ["Executor Group: root.small-group", "Effective parallelism: 5"])
+    self._run_with_compute_processing_cost(coordinator_test_args, CPU_TEST_QUERY,
+        ["Executor Group: root.small-group", "EffectiveParallelism: 5",
+         "ExecutorGroupsConsidered: 2"])
 
+  @pytest.mark.execute_serially
+  def test_query_cpu_count_divisor_two(self):
     # Expect to run the query on the tiny group
     coordinator_test_args = "-query_cpu_count_divisor=2 "
-    self._run_with_compute_processing_cost(coordinator_test_args, TEST_QUERY,
-        ["Executor Group: root.tiny-group", "Effective parallelism: 3"])
+    self._run_with_compute_processing_cost(coordinator_test_args, CPU_TEST_QUERY,
+        ["Executor Group: root.tiny-group", "EffectiveParallelism: 3",
+         "ExecutorGroupsConsidered: 1"])
 
+  @pytest.mark.execute_serially
+  def test_query_cpu_count_divisor_fraction(self):
     # Expect to run the query on the large group
     coordinator_test_args = "-query_cpu_count_divisor=0.2 "
-    self._run_with_compute_processing_cost(coordinator_test_args, TEST_QUERY,
-        ["Executor Group: root.large-group", "Effective parallelism: 7"])
+    self._run_with_compute_processing_cost(coordinator_test_args, CPU_TEST_QUERY,
+        ["Executor Group: root.large-group", "EffectiveParallelism: 7",
+         "ExecutorGroupsConsidered: 3"])
 
   @pytest.mark.execute_serially
   def test_per_exec_group_set_metrics(self):