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:39 UTC

[impala] branch master updated (f2cb2c9ce -> 1cfd41e8b)

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

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


    from f2cb2c9ce IMPALA-11964: Make sure Impala returns error for Iceberg tables with equality deletes
     new 5c1693d03 IMPALA-12005: Describe executor group set selection in query profile
     new f2b01c1dd IMPALA-12017: Skip memory and cpu limit check if REQUEST_POOL is set
     new 1cfd41e8b IMPALA-11886: Data cache should support asynchronous writes

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/runtime/io/data-cache-test.cc               | 166 ++++-
 be/src/runtime/io/data-cache-trace.cc              |   3 +-
 be/src/runtime/io/data-cache.cc                    | 168 ++++-
 be/src/runtime/io/data-cache.h                     |  39 +-
 be/src/runtime/io/disk-io-mgr.cc                   |   9 +-
 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/impalad-metrics.cc                     |  22 +
 be/src/util/impalad-metrics.h                      |  18 +
 be/src/util/runtime-profile.cc                     |  48 +-
 be/src/util/runtime-profile.h                      |  21 +-
 bin/run-all-tests.sh                               |   6 +
 bin/start-impala-cluster.py                        |   8 +
 common/thrift/Frontend.thrift                      |   3 +
 common/thrift/metrics.json                         |  40 +
 .../org/apache/impala/planner/PlanFragment.java    |  28 +-
 .../java/org/apache/impala/planner/Planner.java    |   9 -
 .../java/org/apache/impala/service/Frontend.java   | 122 ++-
 .../org/apache/impala/service/FrontendProfile.java |  64 +-
 .../queries/PlannerTest/tpcds-processing-cost.test | 814 ++++++++++-----------
 tests/custom_cluster/test_executor_groups.py       |  92 ++-
 22 files changed, 1160 insertions(+), 536 deletions(-)


[impala] 03/03: IMPALA-11886: Data cache should support asynchronous writes

Posted by jo...@apache.org.
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 1cfd41e8b10f6e91fc79d50ab58e671d63b65eec
Author: Eyizoha <18...@163.com>
AuthorDate: Thu Feb 16 10:48:39 2023 +0800

    IMPALA-11886: Data cache should support asynchronous writes
    
    This patch implements asynchronous writes to the data cache to improve
    scan performance when a cache miss happens.
    Previously, writes to the data cache are synchronous with hdfs file
    reads, and both are handled by remote hdfs IO threads. In other words,
    if a cache miss occurs,  the IO thread needs to take additional
    responsibility for cache writes,  which will lead to scan performance
    deterioration.
    This patch uses a thread pool for asynchronous writes, and the number of
    threads in the pool is determined by the new configuration
    'data_cache_num_write_threads'. In asynchronous write mode, the IO
    thread only needs to copy data to the temporary buffer when storing data
    into the data cache. The additional memory consumption caused by
    temporary buffers can be limited, depending on the new configuration
    'data_cache_write_buffer_limit'.
    
    Testing:
    - Add test cases for asynchronous data writing to the original
    DataCacheTest using different number of threads.
    - Add DataCacheTest,#OutOfWriteBufferLimit
    Used to test the limit of memory consumed by temporary buffers in the
    case of asynchronous writes
    - Add a timer to the MultiThreadedReadWrite function to get the average
    time of multithreaded writes. Here are some test cases and their time
    that differ significantly between synchronous and asynchronous:
    Test case                | Policy | Sync/Async | write time in ms
    MultiThreadedNoMisses    | LRU    | Sync       |   12.20
    MultiThreadedNoMisses    | LRU    | Async      |   20.74
    MultiThreadedNoMisses    | LIRS   | Sync       |    9.42
    MultiThreadedNoMisses    | LIRS   | Async      |   16.75
    MultiThreadedWithMisses  | LRU    | Sync       |  510.87
    MultiThreadedWithMisses  | LRU    | Async      |   10.06
    MultiThreadedWithMisses  | LIRS   | Sync       | 1872.11
    MultiThreadedWithMisses  | LIRS   | Async      |   11.02
    MultiPartitions          | LRU    | Sync       |    1.20
    MultiPartitions          | LRU    | Async      |    5.23
    MultiPartitions          | LIRS   | Sync       |    1.26
    MultiPartitions          | LIRS   | Async      |    7.91
    AccessTraceAnonymization | LRU    | Sync       | 1963.89
    AccessTraceAnonymization | LRU    | Sync       | 2073.62
    AccessTraceAnonymization | LRU    | Async      |    9.43
    AccessTraceAnonymization | LRU    | Async      |   13.13
    AccessTraceAnonymization | LIRS   | Sync       | 1663.93
    AccessTraceAnonymization | LIRS   | Sync       | 1501.86
    AccessTraceAnonymization | LIRS   | Async      |   12.83
    AccessTraceAnonymization | LIRS   | Async      |   12.74
    
    Change-Id: I878f7486d485b6288de1a9145f49576b7155d312
    Reviewed-on: http://gerrit.cloudera.org:8080/19475
    Reviewed-by: Joe McDonnell <jo...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/runtime/io/data-cache-test.cc  | 166 +++++++++++++++++++++++++++------
 be/src/runtime/io/data-cache-trace.cc |   3 +-
 be/src/runtime/io/data-cache.cc       | 168 ++++++++++++++++++++++++++++++----
 be/src/runtime/io/data-cache.h        |  39 +++++++-
 be/src/runtime/io/disk-io-mgr.cc      |   9 +-
 be/src/util/impalad-metrics.cc        |  22 +++++
 be/src/util/impalad-metrics.h         |  18 ++++
 bin/run-all-tests.sh                  |   6 ++
 bin/start-impala-cluster.py           |   8 ++
 common/thrift/metrics.json            |  40 ++++++++
 10 files changed, 427 insertions(+), 52 deletions(-)

diff --git a/be/src/runtime/io/data-cache-test.cc b/be/src/runtime/io/data-cache-test.cc
index f025b32a4..b1540cf59 100644
--- a/be/src/runtime/io/data-cache-test.cc
+++ b/be/src/runtime/io/data-cache-test.cc
@@ -70,12 +70,22 @@ DECLARE_string(data_cache_eviction_policy);
 DECLARE_string(data_cache_trace_dir);
 DECLARE_int32(max_data_cache_trace_file_size);
 DECLARE_int32(data_cache_trace_percentage);
+DECLARE_int32(data_cache_num_async_write_threads);
+DECLARE_string(data_cache_async_write_buffer_limit);
 
 namespace impala {
 namespace io {
 
 using boost::filesystem::path;
 
+struct DataCacheTestParam {
+  DataCacheTestParam(const std::string& eviction_policy, int32 num_write_threads)
+    : eviction_policy(eviction_policy),
+      num_write_threads(num_write_threads) { }
+  std::string eviction_policy;
+  int32 num_write_threads;
+};
+
 class DataCacheBaseTest : public testing::Test {
  public:
   const uint8_t* test_buffer() {
@@ -90,6 +100,14 @@ class DataCacheBaseTest : public testing::Test {
     return data_cache_trace_dir_;
   }
 
+  /// A function for testing, we need to make sure that all the store tasks have completed
+  /// before any lookup when running test case in async write mode.
+  static void WaitForAsyncWrite(const DataCache& cache) {
+    while (cache.current_buffer_size_.Load() != 0) {
+      usleep(500);
+    }
+  }
+
   //
   // Use multiple threads to insert and read back a set of ranges from test_buffer().
   // Depending on the setting, the working set may or may not fit in the cache.
@@ -104,13 +122,14 @@ class DataCacheBaseTest : public testing::Test {
   //                             from the cache
   //
   void MultiThreadedReadWrite(DataCache* cache, int64_t max_start_offset,
-      bool use_per_thread_filename, bool expect_misses) {
+      bool use_per_thread_filename, bool expect_misses, double* write_time_ms) {
     // Barrier to synchronize all threads so no thread will start probing the cache until
     // all insertions are done.
     CountingBarrier barrier(NUM_THREADS);
 
     vector<unique_ptr<Thread>> threads;
     int num_misses[NUM_THREADS];
+    int64_t write_times_us[NUM_THREADS];
     for (int i = 0; i < NUM_THREADS; ++i) {
       unique_ptr<Thread> thread;
       num_misses[i] = 0;
@@ -118,14 +137,17 @@ class DataCacheBaseTest : public testing::Test {
       ASSERT_OK(Thread::Create("data-cache-test", thread_name,
           boost::bind(&DataCacheBaseTest::ThreadFn, this,
              use_per_thread_filename ? thread_name : "", cache, max_start_offset,
-             &barrier, &num_misses[i]), &thread));
+             &barrier, &num_misses[i], &write_times_us[i]), &thread));
       threads.emplace_back(std::move(thread));
     }
     int cache_misses = 0;
+    int64_t avg_write_time_us = 0;
     for (int i = 0; i < NUM_THREADS; ++i) {
       threads[i]->Join();
       cache_misses += num_misses[i];
+      avg_write_time_us += write_times_us[i];
     }
+    *write_time_ms = static_cast<double>(avg_write_time_us / NUM_THREADS) / 1000;
     if (expect_misses) {
       ASSERT_GT(cache_misses, 0);
     } else {
@@ -168,7 +190,7 @@ class DataCacheBaseTest : public testing::Test {
   }
 
   // Create a bunch of test directories in which the data cache will reside.
-  void SetupWithParameters(std::string eviction_policy) {
+  void SetupWithParameters(DataCacheTestParam param) {
     test_env_.reset(new TestEnv());
     flag_saver_.reset(new google::FlagSaver());
     ASSERT_OK(test_env_->Init());
@@ -189,7 +211,11 @@ class DataCacheBaseTest : public testing::Test {
     FLAGS_data_cache_write_concurrency = NUM_THREADS;
 
     // This is parameterized to allow testing LRU and LIRS
-    FLAGS_data_cache_eviction_policy = eviction_policy;
+    FLAGS_data_cache_eviction_policy = param.eviction_policy;
+
+    // This is parameterized to allow testing asynchronous write.
+    FLAGS_data_cache_num_async_write_threads = param.num_write_threads;
+    FLAGS_data_cache_async_write_buffer_limit = "1G";
   }
 
   // Delete all the test directories created.
@@ -234,23 +260,29 @@ class DataCacheBaseTest : public testing::Test {
   // of cache keys which indirectly control the cache footprint. 'num_misses' records
   // the number of cache misses.
   void ThreadFn(const string& fname_prefix, DataCache* cache, int64_t max_start_offset,
-      CountingBarrier* store_barrier, int* num_misses) {
+      CountingBarrier* store_barrier, int* num_misses, int64_t* write_time_us) {
     const string& custom_fname = Substitute("$0file", fname_prefix);
     vector<int64_t> offsets;
     for (int64_t offset = 0; offset < max_start_offset; ++offset) {
       offsets.push_back(offset);
     }
     random_shuffle(offsets.begin(), offsets.end());
+    int64_t write_start_time = UnixMicros();
     for (int64_t offset : offsets) {
       cache->Store(custom_fname, MTIME, offset, test_buffer() + offset,
           TEMP_BUFFER_SIZE);
     }
+    *write_time_us = UnixMicros() - write_start_time;
     // Wait until all threads have finished inserting. Since different threads may be
     // inserting the same cache key and collide, only one thread which wins the race will
     // insert the cache entry. Make sure other threads which lose out on the race will
     // wait for the insertion to complete first before proceeding.
     store_barrier->Notify();
     store_barrier->Wait();
+
+    // Before continue, we should wait for all asynchronous store tasks (if there are)
+    // finished.
+    WaitForAsyncWrite(*cache);
     for (int64_t offset : offsets) {
       uint8_t buffer[TEMP_BUFFER_SIZE];
       memset(buffer, 0, TEMP_BUFFER_SIZE);
@@ -268,7 +300,7 @@ class DataCacheBaseTest : public testing::Test {
 
 class DataCacheTest :
     public DataCacheBaseTest,
-    public ::testing::WithParamInterface<std::string> {
+    public ::testing::WithParamInterface<DataCacheTestParam> {
  public:
   DataCacheTest()
       : DataCacheBaseTest() {
@@ -280,15 +312,19 @@ class DataCacheTest :
   }
 };
 
-INSTANTIATE_TEST_CASE_P(DataCacheTestTypes, DataCacheTest,
-    ::testing::Values("LRU", "LIRS"));
+INSTANTIATE_TEST_CASE_P(DataCacheTestTypes, DataCacheTest,::testing::Values(
+    DataCacheTestParam("LRU", 0),
+    DataCacheTestParam("LRU", NUM_THREADS),
+    DataCacheTestParam("LIRS", 0),
+    DataCacheTestParam("LIRS", NUM_THREADS)));
 
 // This test exercises the basic insertion and lookup paths by inserting a known set of
 // offsets which fit in the cache entirely. Also tries reading entries which are never
 // inserted into the cache.
 TEST_P(DataCacheTest, TestBasics) {
   const int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
 
   // Temporary buffer for holding results read from the cache.
@@ -308,6 +344,9 @@ TEST_P(DataCacheTest, TestBasics) {
         ASSERT_EQ(0, memcmp(test_buffer() + offset, buffer, TEMP_BUFFER_SIZE));
       }
     }
+    // Before continue, we should wait for all asynchronous store tasks (if there are)
+    // finished.
+    WaitForAsyncWrite(cache);
   }
 
   // Read the same range inserted previously but with a different filename.
@@ -345,6 +384,7 @@ TEST_P(DataCacheTest, TestBasics) {
   uint8_t buffer2[TEMP_BUFFER_SIZE + 10];
   const int64_t larger_entry_size = TEMP_BUFFER_SIZE + 10;
   ASSERT_TRUE(cache.Store(FNAME, MTIME, 0, test_buffer(), larger_entry_size));
+  WaitForAsyncWrite(cache);
   memset(buffer2, 0, larger_entry_size);
   ASSERT_EQ(larger_entry_size,
       cache.Lookup(FNAME, MTIME, 0, larger_entry_size, buffer2));
@@ -373,13 +413,15 @@ TEST_P(DataCacheTest, TestBasics) {
 }
 
 TEST_P(DataCacheTest, NonCanonicalPath) {
-  DataCache cache("/noncanonical/../file/./system/path:1M");
+  DataCache cache("/noncanonical/../file/./system/path:1M",
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_ERROR_MSG(cache.Init(), "/noncanonical/../file/./system/path is not a canonical"
       " path");
 }
 
 TEST_P(DataCacheTest, NonExistantPath) {
-  DataCache cache("/invalid/file/system/path:1M");
+  DataCache cache("/invalid/file/system/path:1M",
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_ERROR_MSG(cache.Init(), "Encountered exception while verifying existence of "
       "directory path /invalid/file/system/path: No such file or directory");
 }
@@ -390,7 +432,8 @@ TEST_P(DataCacheTest, RotateFiles) {
   // Set the maximum size of backing files to be 1/4 of the cache size.
   FLAGS_data_cache_file_max_size_bytes = DEFAULT_CACHE_SIZE / 4;
   const int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
 
   // Read and then insert a range of offsets. Expected all misses in the first iteration
@@ -409,6 +452,7 @@ TEST_P(DataCacheTest, RotateFiles) {
         ASSERT_EQ(0, memcmp(test_buffer() + offset, buffer, TEMP_BUFFER_SIZE));
       }
     }
+    WaitForAsyncWrite(cache);
   }
 
   // Make sure the cache's destructor removes all backing files.
@@ -432,7 +476,8 @@ TEST_P(DataCacheTest, RotateAndDeleteFiles) {
   FLAGS_data_cache_max_opened_files = 1;
 
   const int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
 
   // Read and insert a working set the same size of the cache. Expected all misses
@@ -444,6 +489,7 @@ TEST_P(DataCacheTest, RotateAndDeleteFiles) {
       ASSERT_EQ(0, cache.Lookup(FNAME, MTIME, offset, TEMP_BUFFER_SIZE, buffer));
       ASSERT_TRUE(cache.Store(FNAME, MTIME, offset, test_buffer() + offset,
           TEMP_BUFFER_SIZE));
+      WaitForAsyncWrite(cache);
     }
   }
 
@@ -486,7 +532,8 @@ TEST_P(DataCacheTest, LRUEviction) {
   // This test is specific to LRU
   if (FLAGS_data_cache_eviction_policy != "LRU") return;
   const int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
 
   // Read and then insert range chunks of size TEMP_BUFFER_SIZE from the test buffer.
@@ -498,6 +545,7 @@ TEST_P(DataCacheTest, LRUEviction) {
       ASSERT_EQ(0, cache.Lookup(FNAME, MTIME, offset, TEMP_BUFFER_SIZE, buffer));
       ASSERT_TRUE(cache.Store(FNAME, MTIME, offset, test_buffer() + offset,
           TEMP_BUFFER_SIZE));
+      WaitForAsyncWrite(cache);
     }
   }
   // Verifies that the cache is full.
@@ -518,6 +566,7 @@ TEST_P(DataCacheTest, LRUEviction) {
   const string& alt_fname = "random";
   offset = 0;
   ASSERT_TRUE(cache.Store(alt_fname, MTIME, offset, large_buffer.get(), cache_size));
+  WaitForAsyncWrite(cache);
 
   // Verifies that all previous entries are all evicted.
   for (offset = 0; offset < 1028; ++offset) {
@@ -539,28 +588,40 @@ TEST_P(DataCacheTest, LRUEviction) {
 // collision during insertion, all entries in the working set should be found.
 TEST_P(DataCacheTest, MultiThreadedNoMisses) {
   int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
 
   int64_t max_start_offset = NUM_CACHE_ENTRIES_NO_EVICT;
   bool use_per_thread_filename = false;
   bool expect_misses = false;
+  double wait_time_ms;
   MultiThreadedReadWrite(&cache, max_start_offset, use_per_thread_filename,
-      expect_misses);
+      expect_misses, &wait_time_ms);
+  LOG(INFO) << "MultiThreadedNoMisses, "
+            << FLAGS_data_cache_eviction_policy << ", "
+            << (FLAGS_data_cache_num_async_write_threads > 0 ? "Async" : "Sync")
+            << ", write_time_ms: " << wait_time_ms;
 }
 
 // Inserts a working set which is known to be larger than the cache's capacity.
 // Expect some cache misses in lookups.
 TEST_P(DataCacheTest, MultiThreadedWithMisses) {
   int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
 
   int64_t max_start_offset = 1024;
   bool use_per_thread_filename = true;
   bool expect_misses = true;
+  double wait_time_ms;
   MultiThreadedReadWrite(&cache, max_start_offset, use_per_thread_filename,
-      expect_misses);
+      expect_misses, &wait_time_ms);
+  LOG(INFO) << "MultiThreadedWithMisses, "
+            << FLAGS_data_cache_eviction_policy << ", "
+            << (FLAGS_data_cache_num_async_write_threads > 0 ? "Async" : "Sync")
+            << ", write_time_ms: " << wait_time_ms;
 }
 
 // Test insertion and lookup with a cache configured with multiple partitions.
@@ -568,14 +629,20 @@ TEST_P(DataCacheTest, MultiPartitions) {
   StringPiece delimiter(",");
   string cache_base = JoinStrings(data_cache_dirs(), delimiter);
   const int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", cache_base, std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", cache_base, std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
 
   int64_t max_start_offset = 512;
   bool use_per_thread_filename = false;
   bool expect_misses = false;
+  double wait_time_ms;
   MultiThreadedReadWrite(&cache, max_start_offset, use_per_thread_filename,
-      expect_misses);
+      expect_misses, &wait_time_ms);
+  LOG(INFO) << "MultiPartitions, "
+            << FLAGS_data_cache_eviction_policy << ", "
+            << (FLAGS_data_cache_num_async_write_threads > 0 ? "Async" : "Sync")
+            << ", write_time_ms: " << wait_time_ms;
 }
 
 // Tests insertion of a working set whose size is 1/8 of the total memory size.
@@ -585,8 +652,11 @@ TEST_P(DataCacheTest, LargeFootprint) {
   struct sysinfo info;
   ASSERT_EQ(0, sysinfo(&info));
   ASSERT_GT(info.totalram, 0);
+  // Set sufficient buffer size limits to prevent async store failures.
+  FLAGS_data_cache_async_write_buffer_limit = std::to_string(info.totalram);
   DataCache cache(
-      Substitute("$0:$1", data_cache_dirs()[0], std::to_string(info.totalram)));
+      Substitute("$0:$1", data_cache_dirs()[0], std::to_string(info.totalram)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
 
   const int64_t footprint = info.totalram / 8;
@@ -594,6 +664,7 @@ TEST_P(DataCacheTest, LargeFootprint) {
     int64_t offset = i * TEST_BUFFER_SIZE;
     ASSERT_TRUE(cache.Store(FNAME, MTIME, offset, test_buffer(), TEST_BUFFER_SIZE));
   }
+  WaitForAsyncWrite(cache);
   uint8_t buffer[TEST_BUFFER_SIZE];
   for (int64_t i = 0; i < footprint / TEST_BUFFER_SIZE; ++i) {
     int64_t offset = i * TEST_BUFFER_SIZE;
@@ -620,13 +691,19 @@ TEST_P(DataCacheTest, AccessTraceAnonymization) {
     ASSERT_OK(FileSystemUtil::RemoveAndCreateDirectory(FLAGS_data_cache_trace_dir));
     {
       DataCache cache(Substitute(
-          "$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+          "$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+          FLAGS_data_cache_num_async_write_threads);
       ASSERT_OK(cache.Init());
 
       bool use_per_thread_filename = true;
       bool expect_misses = true;
+      double wait_time_ms;
       MultiThreadedReadWrite(&cache, max_start_offset, use_per_thread_filename,
-                             expect_misses);
+          expect_misses, &wait_time_ms);
+      LOG(INFO) << "AccessTraceAnonymization, "
+                << FLAGS_data_cache_eviction_policy << ", "
+                << (FLAGS_data_cache_num_async_write_threads > 0 ? "Async" : "Sync")
+                << ", write_time_ms: " << wait_time_ms;
     }
 
     // Part 1: Read the trace files and ensure that the JSON entries are valid
@@ -686,13 +763,19 @@ TEST_P(DataCacheTest, AccessTraceSubsetPercentage) {
     ASSERT_OK(FileSystemUtil::RemoveAndCreateDirectory(FLAGS_data_cache_trace_dir));
     {
       DataCache cache(Substitute(
-          "$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+          "$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+          FLAGS_data_cache_num_async_write_threads);
       ASSERT_OK(cache.Init());
 
       bool use_per_thread_filename = true;
       bool expect_misses = true;
+      double wait_time_ms;
       MultiThreadedReadWrite(&cache, max_start_offset, use_per_thread_filename,
-                             expect_misses);
+          expect_misses, &wait_time_ms);
+      LOG(INFO) << "AccessTraceSubsetPercentage, "
+                << FLAGS_data_cache_eviction_policy << ", "
+                << (FLAGS_data_cache_num_async_write_threads > 0 ? "Async" : "Sync")
+                << ", write_time_ms: " << wait_time_ms;
     }
 
     // Replay the trace and record the counts
@@ -732,7 +815,8 @@ TEST_P(DataCacheTest, RotationalDisk) {
   MockDisk("sda", /*is_rotational*/ true);
 
   const int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
   ASSERT_EQ(1, cache.partitions_[0]->data_cache_write_concurrency_);
 }
@@ -743,7 +827,8 @@ TEST_P(DataCacheTest, NonRotationalDisk) {
   MockDisk("nvme0n1", /*is_rotational*/ false);
 
   const int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
   ASSERT_EQ(8, cache.partitions_[0]->data_cache_write_concurrency_);
 }
@@ -754,11 +839,36 @@ TEST_P(DataCacheTest, InvalidDisk) {
   MockDisk("won't parse", /*is_rotational*/ false);
 
   const int64_t cache_size = DEFAULT_CACHE_SIZE;
-  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)));
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
   ASSERT_OK(cache.Init());
   ASSERT_EQ(1, cache.partitions_[0]->data_cache_write_concurrency_);
 }
 
+TEST_P(DataCacheTest, OutOfWriteBufferLimit) {
+  // This test is specific to asynchronous write
+  if (FLAGS_data_cache_num_async_write_threads == 0) return;
+
+  // Setting a lower buffer size limit will fail to insert data larger than this value at
+  // once.
+  FLAGS_data_cache_async_write_buffer_limit = Substitute("$0", TEMP_BUFFER_SIZE << 1);
+
+  const int64_t cache_size = DEFAULT_CACHE_SIZE;
+  DataCache cache(Substitute("$0:$1", data_cache_dirs()[0], std::to_string(cache_size)),
+      FLAGS_data_cache_num_async_write_threads);
+  ASSERT_OK(cache.Init());
+
+  // Continuously inserting a large amount of entries, there will be a certain number of
+  // store tasks fail due to the write buffer limit.
+  int32_t count = 0;
+  for (int64_t offset = 0; offset < NUM_CACHE_ENTRIES_NO_EVICT; ++offset) {
+    if (cache.Store(FNAME, MTIME, offset, test_buffer() + offset, TEMP_BUFFER_SIZE)) {
+      ++count;
+    }
+  }
+  EXPECT_LT(count, NUM_CACHE_ENTRIES_NO_EVICT);
+}
+
 } // namespace io
 } // namespace impala
 
diff --git a/be/src/runtime/io/data-cache-trace.cc b/be/src/runtime/io/data-cache-trace.cc
index 68bf31deb..f571c52e4 100644
--- a/be/src/runtime/io/data-cache-trace.cc
+++ b/be/src/runtime/io/data-cache-trace.cc
@@ -266,7 +266,8 @@ TraceReplayer::TraceReplayer(string trace_configuration)
 TraceReplayer::~TraceReplayer() {}
 
 Status TraceReplayer::Init() {
-  data_cache_.reset(new DataCache(trace_configuration_, /* trace_replay */ true));
+  data_cache_.reset(new DataCache(trace_configuration_, /* num_async_write_threads */ 0,
+      /* trace_replay */ true));
   RETURN_IF_ERROR(data_cache_->Init());
   initialized_ = true;
   return Status::OK();
diff --git a/be/src/runtime/io/data-cache.cc b/be/src/runtime/io/data-cache.cc
index 71ea2452a..2cf45041d 100644
--- a/be/src/runtime/io/data-cache.cc
+++ b/be/src/runtime/io/data-cache.cc
@@ -116,12 +116,20 @@ DEFINE_string(data_cache_eviction_policy, "LRU",
     "(Advanced) The cache eviction policy to use for the data cache. "
     "Either 'LRU' (default) or 'LIRS' (experimental)");
 
+DEFINE_string(data_cache_async_write_buffer_limit, "1GB",
+    "(Experimental) Limit of the total buffer size used by asynchronous store tasks.");
+
 namespace impala {
 namespace io {
 
 static const int64_t PAGE_SIZE = 1L << 12;
 const char* DataCache::Partition::CACHE_FILE_PREFIX = "impala-cache-file-";
 const int MAX_FILE_DELETER_QUEUE_SIZE = 500;
+
+// This large value for the queue size is harmless because the total size of the entries
+// on the queue are bound by --data_cache_async_write_bufer_limit.
+const int MAX_STORE_TASK_QUEUE_SIZE = 1 << 20;
+
 static const char* PARTITION_PATH_METRIC_KEY_TEMPLATE =
     "impala-server.io-mgr.remote-data-cache-partition-$0.path";
 static const char* PARTITION_READ_LATENCY_METRIC_KEY_TEMPLATE =
@@ -378,6 +386,32 @@ struct DataCache::CacheKey {
   faststring key_;
 };
 
+/// The class to abstract store behavior, holds a temporary buffer copied from the source
+/// buffer until store complete.
+class DataCache::StoreTask {
+ public:
+  explicit StoreTask(const std::string& filename, int64_t mtime, int64_t offset,
+      uint8_t* buffer, int64_t buffer_len, DataCache* cache)
+    : key_(filename, mtime, offset),
+      buffer_(buffer),
+      buffer_len_(buffer_len),
+      cache_(cache) { }
+
+  ~StoreTask() { cache_->CompleteStoreTask(*this); }
+
+  const CacheKey& key() const { return key_; }
+  const uint8_t* buffer() const { return buffer_.get(); }
+  int64_t buffer_len() const { return buffer_len_; }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(StoreTask);
+
+  CacheKey key_;
+  std::unique_ptr<uint8_t[]> buffer_;
+  int64_t buffer_len_;
+  DataCache* cache_;
+};
+
 static Cache::EvictionPolicy GetCacheEvictionPolicy(const std::string& policy_string) {
   Cache::EvictionPolicy policy = Cache::ParseEvictionPolicy(policy_string);
   if (policy != Cache::EvictionPolicy::LRU && policy != Cache::EvictionPolicy::LIRS) {
@@ -742,9 +776,6 @@ bool DataCache::Partition::Store(const CacheKey& cache_key, const uint8_t* buffe
     // Limit the write concurrency to avoid blocking the caller (which could be calling
     // from the critical path of an IO read) when the cache becomes IO bound due to either
     // limited memory for page cache or the cache is undersized which leads to eviction.
-    //
-    // TODO: defer the writes to another thread which writes asynchronously. Need to bound
-    // the extra memory consumption for holding the temporary buffer though.
     const bool exceed_concurrency =
         pending_insert_set_.size() >= data_cache_write_concurrency_;
     if (exceed_concurrency ||
@@ -840,6 +871,14 @@ bool DataCache::Partition::VerifyChecksum(const string& ops_name, const CacheEnt
   return true;
 }
 
+DataCache::DataCache(const std::string config, int32_t num_async_write_threads,
+    bool trace_replay)
+  : config_(config),
+    trace_replay_(trace_replay),
+    num_async_write_threads_(num_async_write_threads) { }
+
+DataCache::~DataCache() { ReleaseResources(); }
+
 Status DataCache::Init() {
   // Verifies all the configured flags are sane.
   if (FLAGS_data_cache_file_max_size_bytes < PAGE_SIZE) {
@@ -861,12 +900,12 @@ Status DataCache::Init() {
 
   // Parse the capacity string to make sure it's well-formed.
   bool is_percent;
-  int64_t capacity = ParseUtil::ParseMemSpec(all_cache_configs[1], &is_percent, 0);
+  per_partition_capacity_ = ParseUtil::ParseMemSpec(all_cache_configs[1], &is_percent, 0);
   if (is_percent) {
     return Status(Substitute("Malformed data cache capacity configuration $0",
         all_cache_configs[1]));
   }
-  if (capacity < PAGE_SIZE) {
+  if (per_partition_capacity_ < PAGE_SIZE) {
     return Status(Substitute("Configured data cache capacity $0 is too small",
         all_cache_configs[1]));
   }
@@ -882,9 +921,9 @@ Status DataCache::Init() {
   int32_t partition_idx = 0;
   for (const string& dir_path : cache_dirs) {
     LOG(INFO) << "Adding partition " << dir_path << " with capacity "
-              << PrettyPrinter::PrintBytes(capacity);
+              << PrettyPrinter::PrintBytes(per_partition_capacity_);
     std::unique_ptr<Partition> partition =
-        make_unique<Partition>(partition_idx, dir_path, capacity,
+        make_unique<Partition>(partition_idx, dir_path, per_partition_capacity_,
             max_opened_files_per_partition, trace_replay_);
     RETURN_IF_ERROR(partition->Init());
     partitions_.emplace_back(move(partition));
@@ -897,16 +936,46 @@ Status DataCache::Init() {
     // will enqueue a request (i.e. a partition index) when it notices the number of files
     // in a partition exceeds the per-partition limit. The files in a partition will be
     // closed in the order they are created until it's within the per-partition limit.
-    file_deleter_pool_.reset(new ThreadPool<int>("impala-server",
+    file_deleter_pool_.reset(new ThreadPool<int>("data-cache",
         "data-cache-file-deleter", 1, MAX_FILE_DELETER_QUEUE_SIZE,
         bind<void>(&DataCache::DeleteOldFiles, this, _1, _2)));
     RETURN_IF_ERROR(file_deleter_pool_->Init());
   }
 
+  /// If --data_cache_num_async_write_threads has been set above 0, the store behavior
+  /// will be asynchronous. A task queue and thread pool will be started for that.
+  if (num_async_write_threads_ > 0) {
+    if (UNLIKELY(trace_replay_)) {
+      return Status("Data cache does not support asynchronous writes when doing trace "
+          "replay. Please set 'data_cache_num_async_write_threads' to 0.");
+    }
+    // Parse the buffer limit string to make sure it's well-formed.
+    bool is_percent;
+    int64_t buffer_limit = ParseUtil::ParseMemSpec(
+        FLAGS_data_cache_async_write_buffer_limit, &is_percent, 0);
+    if (is_percent) {
+      return Status(Substitute("Malformed data cache write buffer limit configuration $0",
+          FLAGS_data_cache_async_write_buffer_limit));
+    }
+    if (!TestInfo::is_be_test() && buffer_limit < (1 << 23 /* 8MB */)) {
+      return Status(Substitute("Configured data cache write buffer limit $0 is too small "
+          "(less than 8MB)", FLAGS_data_cache_async_write_buffer_limit));
+    }
+    store_buffer_capacity_ = buffer_limit;
+    storer_pool_.reset(new ThreadPool<StoreTaskHandle>("data-cache", "data-cache-storer",
+        num_async_write_threads_, MAX_STORE_TASK_QUEUE_SIZE,
+        bind<void>(&DataCache::HandleStoreTask, this, _1, _2)));
+    RETURN_IF_ERROR(storer_pool_->Init());
+  }
+
   return Status::OK();
 }
 
 void DataCache::ReleaseResources() {
+  if (storer_pool_) {
+    storer_pool_->Shutdown();
+    storer_pool_->Join();
+  }
   if (file_deleter_pool_) file_deleter_pool_->Shutdown();
   for (auto& partition : partitions_) partition->ReleaseResources();
 }
@@ -945,19 +1014,14 @@ bool DataCache::Store(const string& filename, int64_t mtime, int64_t offset,
     return false;
   }
 
+  // If the storer thread pool is available, data will be stored asynchronously.
+  if (num_async_write_threads_ > 0) {
+    return SubmitStoreTask(filename, mtime, offset, buffer, buffer_len);
+  }
+
   // Construct a cache key. The cache key is also hashed to compute the partition index.
   const CacheKey key(filename, mtime, offset);
-  int idx = key.Hash() % partitions_.size();
-  bool start_reclaim;
-  bool stored = partitions_[idx]->Store(key, buffer, buffer_len, &start_reclaim);
-  if (VLOG_IS_ON(3)) {
-    stringstream ss;
-    ss << std::hex << reinterpret_cast<int64_t>(buffer);
-    LOG(INFO) << Substitute("Storing $0 mtime: $1 offset: $2 bytes_to_read: $3 "
-        "buffer: 0x$4 stored: $5", filename, mtime, offset, buffer_len, ss.str(), stored);
-  }
-  if (start_reclaim) file_deleter_pool_->Offer(idx);
-  return stored;
+  return StoreInternal(key, buffer, buffer_len);
 }
 
 Status DataCache::CloseFilesAndVerifySizes() {
@@ -972,6 +1036,72 @@ void DataCache::DeleteOldFiles(uint32_t thread_id, int partition_idx) {
   partitions_[partition_idx]->DeleteOldFiles();
 }
 
+bool DataCache::SubmitStoreTask(const std::string& filename, int64_t mtime,
+    int64_t offset, const uint8_t* buffer, int64_t buffer_len) {
+  const int64_t charge_len = BitUtil::RoundUp(buffer_len, PAGE_SIZE);
+  if (UNLIKELY(charge_len > per_partition_capacity_)) return false;
+
+  // Tries to increase the current_buffer_size_ by buffer_len before allocate buffer.
+  // If new size exceeds store_buffer_capacity_, return false and current_buffer_size_ is
+  // not changed.
+  while (true) {
+    int64_t current_size = current_buffer_size_.Load();
+    int64_t new_size = current_size + buffer_len;
+    if (UNLIKELY(new_size > store_buffer_capacity_)) {
+      VLOG(2) << Substitute("Failed to create store task due to buffer size limitation, "
+          "current buffer size: $0 size limitation: $1 require: $2",
+          current_size, store_buffer_capacity_, buffer_len);
+      ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_BYTES->
+          Increment(buffer_len);
+      ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_ENTRIES->Increment(1);
+      return false;
+    }
+    if (LIKELY(current_buffer_size_.CompareAndSwap(current_size, new_size))) {
+      ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_OUTSTANDING_BYTES->SetValue(
+          current_buffer_size_.Load());
+      break;
+    }
+  }
+
+  DCHECK(buffer != nullptr);
+  // TODO: Should we use buffer pool instead of piecemeal memory allocate?
+  uint8_t* task_buffer = new uint8_t[buffer_len];
+  memcpy(task_buffer, buffer, buffer_len);
+
+  const StoreTask* task =
+      new StoreTask(filename, mtime, offset, task_buffer, buffer_len, this);
+  storer_pool_->Offer(StoreTaskHandle(task));
+  ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_NUM_ASYNC_WRITES_SUBMITTED->Increment(1);
+  return true;
+}
+
+void DataCache::CompleteStoreTask(const StoreTask& task) {
+  current_buffer_size_.Add(-task.buffer_len());
+  DCHECK_GE(current_buffer_size_.Load(), 0);
+  ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_OUTSTANDING_BYTES->SetValue(
+      current_buffer_size_.Load());
+}
+
+void DataCache::HandleStoreTask(uint32_t thread_id, const StoreTaskHandle& task) {
+    StoreInternal(task->key(), task->buffer(), task->buffer_len());
+}
+
+bool DataCache::StoreInternal(const CacheKey& key, const uint8_t* buffer,
+    int64_t buffer_len) {
+  int idx = key.Hash() % partitions_.size();
+  bool start_reclaim;
+  bool stored = partitions_[idx]->Store(key, buffer, buffer_len, &start_reclaim);
+  if (VLOG_IS_ON(3)) {
+    stringstream ss;
+    ss << std::hex << reinterpret_cast<int64_t>(buffer);
+    LOG(INFO) << Substitute("Storing $0 mtime: $1 offset: $2 bytes_to_read: $3 "
+        "buffer: 0x$4 stored: $5", key.filename().ToString(), key.mtime(), key.offset(),
+        buffer_len, ss.str(), stored);
+  }
+  if (start_reclaim) file_deleter_pool_->Offer(idx);
+  return stored;
+}
+
 void DataCache::Partition::Trace(
     const trace::EventType& type, const DataCache::CacheKey& key,
     int64_t lookup_len, int64_t entry_len) {
diff --git a/be/src/runtime/io/data-cache.h b/be/src/runtime/io/data-cache.h
index f7a580e7a..1f5cd870b 100644
--- a/be/src/runtime/io/data-cache.h
+++ b/be/src/runtime/io/data-cache.h
@@ -123,10 +123,10 @@ class DataCache {
   /// an optimized mode that skips all file operations and only does the metadata
   /// operations. This is used to replay the access trace and compare different cache
   /// configurations. See data-cache-trace.h
-  explicit DataCache(const std::string config, bool trace_replay = false)
-    : config_(config), trace_replay_(trace_replay) { }
+  explicit DataCache(const std::string config, int32_t num_async_write_threads = 0,
+      bool trace_replay = false);
 
-  ~DataCache() { ReleaseResources(); }
+  ~DataCache();
 
   /// Parses the configuration string, initializes all partitions in the cache by
   /// checking for storage space available and creates a backing file for caching.
@@ -200,6 +200,7 @@ class DataCache {
   class CacheFile;
   struct CacheKey;
   class CacheEntry;
+  class StoreTask;
 
   /// An implementation of a cache partition. Each partition maintains its own set of
   /// cache keys in a LRU cache.
@@ -390,6 +391,9 @@ class DataCache {
   /// The configuration string for the data cache.
   const std::string config_;
 
+  /// The capacity in bytes of one partition.
+  int64_t per_partition_capacity_;
+
   /// Set to true if this is only doing trace replay. Trace replay does only metadata
   /// operations, and no filesystem operations are required.
   bool trace_replay_;
@@ -406,6 +410,35 @@ class DataCache {
   /// in partitions_[partition_idx].
   void DeleteOldFiles(uint32_t thread_id, int partition_idx);
 
+  /// Create a new store task and copy the data to a temporary buffer, then submit it to
+  /// the asynchronous write thread pool for handling. May abort due to buffer size limit.
+  /// Return true if success.
+  bool SubmitStoreTask(const std::string& filename, int64_t mtime, int64_t offset,
+      const uint8_t* buffer, int64_t buffer_len);
+
+  /// Called by StoreTask's d'tor, decrease the current_buffer_size_ by task's buffer_len.
+  void CompleteStoreTask(const StoreTask& task);
+
+  /// Thread pool for storing cache asynchronously, it is initialized only if
+  /// 'data_cache_num_async_write_threads' has been set above 0, and creates a
+  /// corresponding number of worker threads.
+  int32_t num_async_write_threads_;
+  using StoreTaskHandle = std::unique_ptr<const StoreTask>;
+  std::unique_ptr<ThreadPool<StoreTaskHandle>> storer_pool_;
+
+  /// Thread function called by threads in 'storer_pool_' for handling store task.
+  void HandleStoreTask(uint32_t thread_id, const StoreTaskHandle& task);
+
+  /// Limit of the total buffer size used by asynchronous store tasks, when the current
+  /// buffer size reaches the limit, the subsequent store task will be abandoned.
+  int64_t store_buffer_capacity_;
+
+  /// Total buffer size currently used by all asynchronous store tasks.
+  AtomicInt64 current_buffer_size_{0};
+
+  /// Call the corresponding cache partition for storing.
+  bool StoreInternal(const CacheKey& key, const uint8_t* buffer, int64_t buffer_len);
+
 };
 
 } // namespace io
diff --git a/be/src/runtime/io/disk-io-mgr.cc b/be/src/runtime/io/disk-io-mgr.cc
index 43b744f69..29babb3e0 100644
--- a/be/src/runtime/io/disk-io-mgr.cc
+++ b/be/src/runtime/io/disk-io-mgr.cc
@@ -74,6 +74,12 @@ DEFINE_string(data_cache, "", "The configuration string for IO data cache. "
     "a capacity quota per directory. For example /data/0,/data/1:1TB means the cache "
     "may use up to 2TB, with 1TB max in /data/0 and /data/1 respectively. Please note "
     "that each Impala daemon on a host must have a unique caching directory.");
+DEFINE_int32(data_cache_num_async_write_threads, 0,
+    "(Experimental) Number of data cache async write threads. Write threads will write "
+    "the cache asynchronously after IO thread read data, so IO thread will return more "
+    "quickly. The extra memory for temporary buffers is limited by "
+    "--data_cache_async_write_buffer_limit. If this's 0, then write will be "
+    "synchronous.");
 
 // Rotational disks should have 1 thread per disk to minimize seeks.  Non-rotational
 // don't have this penalty and benefit from multiple concurrent IO requests.
@@ -682,7 +688,8 @@ Status DiskIoMgr::Init() {
   DCHECK_EQ(ret, 0);
 
   if (!FLAGS_data_cache.empty()) {
-    remote_data_cache_.reset(new DataCache(FLAGS_data_cache));
+    remote_data_cache_.reset(
+        new DataCache(FLAGS_data_cache, FLAGS_data_cache_num_async_write_threads));
     RETURN_IF_ERROR(remote_data_cache_->Init());
   }
   return Status::OK();
diff --git a/be/src/util/impalad-metrics.cc b/be/src/util/impalad-metrics.cc
index cb88adca5..b5be78330 100644
--- a/be/src/util/impalad-metrics.cc
+++ b/be/src/util/impalad-metrics.cc
@@ -83,6 +83,14 @@ const char* ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_DROPPED_ENTRIES =
     "impala-server.io-mgr.remote-data-cache-dropped-entries";
 const char* ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_INSTANT_EVICTIONS =
     "impala-server.io-mgr.remote-data-cache-instant-evictions";
+const char* ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_OUTSTANDING_BYTES =
+    "impala-server.io-mgr.remote-data-cache-async-writes-outstanding-bytes";
+const char* ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_NUM_ASYNC_WRITES_SUBMITTED =
+    "impala-server.io-mgr.remote-data-cache-num-async-writes-submitted";
+const char* ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_BYTES =
+    "impala-server.io-mgr.remote-data-cache-async-writes-dropped-bytes";
+const char* ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_ENTRIES =
+    "impala-server.io-mgr.remote-data-cache-async-writes-dropped-entries";
 const char* ImpaladMetricKeys::IO_MGR_BYTES_WRITTEN =
     "impala-server.io-mgr.bytes-written";
 const char* ImpaladMetricKeys::IO_MGR_NUM_CACHED_FILE_HANDLES =
@@ -181,6 +189,12 @@ IntCounter* ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_NUM_WRITES = nullptr;
 IntCounter* ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_DROPPED_BYTES = nullptr;
 IntCounter* ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_DROPPED_ENTRIES = nullptr;
 IntCounter* ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_INSTANT_EVICTIONS = nullptr;
+IntCounter* ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_OUTSTANDING_BYTES =
+    nullptr;
+IntCounter* ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_NUM_ASYNC_WRITES_SUBMITTED = nullptr;
+IntCounter* ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_BYTES = nullptr;
+IntCounter* ImpaladMetrics::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_ENTRIES =
+    nullptr;
 IntCounter* ImpaladMetrics::IO_MGR_BYTES_WRITTEN = nullptr;
 IntCounter* ImpaladMetrics::IO_MGR_CACHED_FILE_HANDLES_REOPENED = nullptr;
 IntCounter* ImpaladMetrics::HEDGED_READ_OPS = nullptr;
@@ -376,6 +390,14 @@ void ImpaladMetrics::CreateMetrics(MetricGroup* m) {
       ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_DROPPED_ENTRIES, 0);
   IO_MGR_REMOTE_DATA_CACHE_INSTANT_EVICTIONS = IO_MGR_METRICS->AddCounter(
       ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_INSTANT_EVICTIONS, 0);
+  IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_OUTSTANDING_BYTES = IO_MGR_METRICS->AddCounter(
+      ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_OUTSTANDING_BYTES, 0);
+  IO_MGR_REMOTE_DATA_CACHE_NUM_ASYNC_WRITES_SUBMITTED = IO_MGR_METRICS->AddCounter(
+      ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_NUM_ASYNC_WRITES_SUBMITTED, 0);
+  IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_BYTES = IO_MGR_METRICS->AddCounter(
+      ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_BYTES, 0);
+  IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_ENTRIES = IO_MGR_METRICS->AddCounter(
+      ImpaladMetricKeys::IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_ENTRIES, 0);
 
   IO_MGR_CACHED_FILE_HANDLES_HIT_RATIO =
       StatsMetric<uint64_t, StatsType::MEAN>::CreateAndRegister(IO_MGR_METRICS,
diff --git a/be/src/util/impalad-metrics.h b/be/src/util/impalad-metrics.h
index 08131da30..3c1f0e17f 100644
--- a/be/src/util/impalad-metrics.h
+++ b/be/src/util/impalad-metrics.h
@@ -114,6 +114,20 @@ class ImpaladMetricKeys {
   /// Total number of entries evicted immediately from the remote data cache.
   static const char* IO_MGR_REMOTE_DATA_CACHE_INSTANT_EVICTIONS;
 
+  /// Total number of bytes async writes outstanding in the remote data cache.
+  static const char* IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_OUTSTANDING_BYTES;
+
+  /// Total number of async writes submitted in the remote data cache.
+  static const char* IO_MGR_REMOTE_DATA_CACHE_NUM_ASYNC_WRITES_SUBMITTED;
+
+  /// Total number of bytes not inserted in the remote data cache due to async writes
+  /// buffer size limit.
+  static const char* IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_BYTES;
+
+  /// Total number of entries not inserted in the remote data cache due to async writes
+  /// buffer size limit.
+  static const char* IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_ENTRIES;
+
   /// Total number of bytes written to disk by the io mgr (for spilling)
   static const char* IO_MGR_BYTES_WRITTEN;
 
@@ -274,6 +288,10 @@ class ImpaladMetrics {
   static IntCounter* IO_MGR_REMOTE_DATA_CACHE_DROPPED_BYTES;
   static IntCounter* IO_MGR_REMOTE_DATA_CACHE_DROPPED_ENTRIES;
   static IntCounter* IO_MGR_REMOTE_DATA_CACHE_INSTANT_EVICTIONS;
+  static IntCounter* IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_OUTSTANDING_BYTES;
+  static IntCounter* IO_MGR_REMOTE_DATA_CACHE_NUM_ASYNC_WRITES_SUBMITTED;
+  static IntCounter* IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_BYTES;
+  static IntCounter* IO_MGR_REMOTE_DATA_CACHE_ASYNC_WRITES_DROPPED_ENTRIES;
   static IntCounter* IO_MGR_SHORT_CIRCUIT_BYTES_READ;
   static IntCounter* IO_MGR_BYTES_WRITTEN;
   static IntCounter* IO_MGR_CACHED_FILE_HANDLES_REOPENED;
diff --git a/bin/run-all-tests.sh b/bin/run-all-tests.sh
index ca45f6a61..e4381293c 100755
--- a/bin/run-all-tests.sh
+++ b/bin/run-all-tests.sh
@@ -65,6 +65,8 @@ fi
 : ${DATA_CACHE_SIZE:=}
 # Data cache eviction policy
 : ${DATA_CACHE_EVICTION_POLICY:=}
+# Number of data cache async write threads.
+: ${DATA_CACHE_NUM_ASYNC_WRITE_THREADS:=}
 if [[ "${TARGET_FILESYSTEM}" == "local" ]]; then
   # TODO: Remove abort_on_config_error flag from here and create-load-data.sh once
   # checkConfiguration() accepts the local filesystem (see IMPALA-1850).
@@ -83,6 +85,10 @@ if [[ -n "${DATA_CACHE_DIR}" && -n "${DATA_CACHE_SIZE}" ]]; then
        TEST_START_CLUSTER_ARGS="${TEST_START_CLUSTER_ARGS} "`
            `"--data_cache_eviction_policy=${DATA_CACHE_EVICTION_POLICY}"
    fi
+   if [[ -n "${DATA_CACHE_NUM_ASYNC_WRITE_THREADS}" ]]; then
+       TEST_START_CLUSTER_ARGS="${TEST_START_CLUSTER_ARGS} "`
+           `"--data_cache_num_async_write_threads=${DATA_CACHE_NUM_ASYNC_WRITE_THREADS}"
+   fi
    # Force use of data cache for HDFS. Data cache is only enabled for remote reads.
    if [[ "${TARGET_FILESYSTEM}" == "hdfs" ]]; then
       TEST_START_CLUSTER_ARGS="${TEST_START_CLUSTER_ARGS} "`
diff --git a/bin/start-impala-cluster.py b/bin/start-impala-cluster.py
index 40ccd3d36..5f93f8170 100755
--- a/bin/start-impala-cluster.py
+++ b/bin/start-impala-cluster.py
@@ -132,6 +132,10 @@ parser.add_option("--data_cache_size", dest="data_cache_size", default=0,
 parser.add_option("--data_cache_eviction_policy", dest="data_cache_eviction_policy",
                   default="LRU", help="This specifies the cache eviction policy to use "
                   "for the data cache")
+parser.add_option("--data_cache_num_async_write_threads",
+                  dest="data_cache_num_async_write_threads", default=0,
+                  help="This specifies the number of asynchronous write threads for the "
+                  "data cache, with 0 set means synchronous writes.")
 parser.add_option("--data_cache_enable_tracing", dest="data_cache_enable_tracing",
                   action="store_true", default=False,
                   help="If the data cache is enabled, this enables tracing accesses.")
@@ -409,6 +413,10 @@ def build_impalad_arg_lists(cluster_size, num_coordinators, use_exclusive_coordi
       args = "-data_cache_eviction_policy={policy} {args}".format(
           policy=options.data_cache_eviction_policy, args=args)
 
+      # Add the number of async write threads.
+      args = "-data_cache_num_async_write_threads={num_threads} {args}".format(
+          num_threads=options.data_cache_num_async_write_threads, args=args)
+
       # Add access tracing arguments if requested
       if options.data_cache_enable_tracing:
         tracing_args = ""
diff --git a/common/thrift/metrics.json b/common/thrift/metrics.json
index 403724480..c24d49e7d 100644
--- a/common/thrift/metrics.json
+++ b/common/thrift/metrics.json
@@ -659,6 +659,46 @@
     "kind": "HISTOGRAM",
     "key": "impala-server.io-mgr.remote-data-cache-partition-$0.eviction-latency"
   },
+  {
+    "description": "Total number of bytes of async writes outstanding in the remote data cache.",
+    "contexts": [
+      "IMPALAD"
+    ],
+    "label": "Impala Server Io Mgr Remote Data Cache Async Writes Outstanding In Bytes",
+    "units": "BYTES",
+    "kind": "COUNTER",
+    "key": "impala-server.io-mgr.remote-data-cache-async-writes-outstanding-bytes"
+  },
+  {
+    "description": "Total number of async writes submitted in the remote data cache.",
+    "contexts": [
+      "IMPALAD"
+    ],
+    "label": "Impala Server Io Mgr Remote Data Cache Num Async Writes Submitted",
+    "units": "UNIT",
+    "kind": "COUNTER",
+    "key": "impala-server.io-mgr.remote-data-cache-num-async-writes-submitted"
+  },
+  {
+    "description": "Total number of bytes not inserted in the remote data cache due to async writes buffer size limit.",
+    "contexts": [
+      "IMPALAD"
+    ],
+    "label": "Impala Server Io Mgr Remote Data Cache Bytes Not Inserted Due To Async Writes Buffer Size Limit",
+    "units": "BYTES",
+    "kind": "COUNTER",
+    "key": "impala-server.io-mgr.remote-data-cache-async-writes-dropped-bytes"
+  },
+  {
+    "description": "Total number of entries not inserted in the remote data cache due to async writes buffer size limit.",
+    "contexts": [
+      "IMPALAD"
+    ],
+    "label": "Impala Server Io Mgr Remote Data Cache Entries Not Inserted Due To Async Writes Buffer Size Limit",
+    "units": "UNIT",
+    "kind": "COUNTER",
+    "key": "impala-server.io-mgr.remote-data-cache-async-writes-dropped-entries"
+  },
   {
     "description": "The number of allocated IO buffers. IO buffers are shared by all queries.",
     "contexts": [


[impala] 02/03: IMPALA-12017: Skip memory and cpu limit check if REQUEST_POOL is set

Posted by jo...@apache.org.
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 f2b01c1ddb7d5f002d07fdda12afe9300ac316e8
Author: Riza Suminto <ri...@cloudera.com>
AuthorDate: Wed Mar 22 11:50:35 2023 -0700

    IMPALA-12017: Skip memory and cpu limit check if REQUEST_POOL is set
    
    Memory and cpu limit checking in executor group
    selection (Frontend.java) should be skipped if REQUEST_POOL query option
    is set. Setting REQUEST_POOL means user is specifying pool to run the
    query regardless of memory and cpu limit.
    
    Testing:
    - Add test cases in test_query_cpu_count_divisor_default
    
    Change-Id: I14bf7fe71e2dda1099651b3edf62480e1fdbf845
    Reviewed-on: http://gerrit.cloudera.org:8080/19645
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Riza Suminto <ri...@cloudera.com>
---
 .../java/org/apache/impala/service/Frontend.java   | 21 ++++--
 tests/custom_cluster/test_executor_groups.py       | 74 +++++++++++++++++-----
 2 files changed, 75 insertions(+), 20 deletions(-)

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 28f097f3a..e59bf90bc 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -2059,6 +2059,7 @@ public class Frontend {
       }
 
       // Counters about this group set.
+      int available_cores = expectedTotalCores(group_set);
       String profileName = "Executor group " + (i + 1);
       if (group_set.isSetExec_group_name_prefix()
           && !group_set.getExec_group_name_prefix().isEmpty()) {
@@ -2067,6 +2068,7 @@ public class Frontend {
       TRuntimeProfileNode groupSetProfile = createTRuntimeProfileNode(profileName);
       addCounter(groupSetProfile,
           new TCounter(MEMORY_MAX, TUnit.BYTES, group_set.getMax_mem_limit()));
+      addCounter(groupSetProfile, new TCounter(CPU_MAX, TUnit.UNIT, available_cores));
       FrontendProfile.getCurrent().addChildrenProfile(groupSetProfile);
 
       // Find out the per host memory estimated from two possible sources.
@@ -2089,28 +2091,38 @@ public class Frontend {
 
       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()));
-        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 (memReqSatisfied && cpuReqSatisfied) {
+      boolean matchFound = false;
+      if (queryOptions.isSetRequest_pool()) {
+        if (!default_executor_group) {
+          Preconditions.checkState(group_set.getExec_group_name_prefix().endsWith(
+              queryOptions.getRequest_pool()));
+        }
+        reason = "query option REQUEST_POOL=" + queryOptions.getRequest_pool()
+            + " is set. Memory and cpu limit checking is skipped.";
+        addInfoString(groupSetProfile, VERDICT, reason);
+        matchFound = true;
+      } else 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");
+        matchFound = true;
+      }
 
+      if (matchFound) {
         // Set the group name prefix in both the returned query options and
         // the query context for non default group setup.
         if (!default_executor_group) {
@@ -2120,7 +2132,6 @@ public class Frontend {
             req.query_exec_request.query_ctx.setRequest_pool(namePrefix);
           }
         }
-
         break;
       }
 
diff --git a/tests/custom_cluster/test_executor_groups.py b/tests/custom_cluster/test_executor_groups.py
index 7df6c27bb..4ae476e81 100644
--- a/tests/custom_cluster/test_executor_groups.py
+++ b/tests/custom_cluster/test_executor_groups.py
@@ -22,6 +22,7 @@ from builtins import range
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.util.concurrent_workload import ConcurrentWorkload
 
+import copy
 import json
 import logging
 import os
@@ -33,9 +34,12 @@ 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.
+# 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 query option to use for testing CPU requirement.
+CPU_DOP_OPTIONS = {'MT_DOP': '2', 'COMPUTE_PROCESSING_COST': 'true'}
+
 DEFAULT_RESOURCE_POOL = "default-pool"
 
 
@@ -785,16 +789,16 @@ class TestExecutorGroups(CustomClusterTestSuite):
     result = self.execute_query_expect_success(self.client, LARGE_QUERY)
     assert "Executor Group: root.large-group" in str(result.runtime_profile)
 
-    # Force to run the large query on the small group should fail
+    # Force to run the large query on the small group.
+    # Query should run successfully since exec group memory limit is ignored.
     self.client.set_configuration({'request_pool': 'small'})
-    result = self.execute_query_expect_failure(self.client, LARGE_QUERY)
-    assert ("The query does not fit largest executor group sets. "
-        "Reason: not enough per-host memory") in str(result)
+    result = self.execute_query_expect_success(self.client, LARGE_QUERY)
+    assert ("Verdict: query option REQUEST_POOL=small is set. "
+        "Memory and cpu limit checking is skipped.") in str(result.runtime_profile)
 
     self.client.close()
 
-  def _run_with_compute_processing_cost(self, coordinator_test_args, TEST_QUERY,
-      expected_strings_in_profile):
+  def _setup_three_exec_group_cluster(self, coordinator_test_args):
     # The path to resources directory which contains the admission control config files.
     RESOURCES_DIR = os.path.join(os.environ['IMPALA_HOME'], "fe", "src", "test",
                                  "resources")
@@ -840,37 +844,77 @@ class TestExecutorGroups(CustomClusterTestSuite):
     assert self._get_num_executor_groups(only_healthy=True,
                                          exec_group_set_prefix="root.large") == 1
 
-    # assert that 'expected_profile' exist in query profile
-    self.execute_query_expect_success(self.client, 'SET MT_DOP=2;')
-    self.execute_query_expect_success(self.client, 'SET COMPUTE_PROCESSING_COST=1;')
-    result = self.execute_query_expect_success(self.client, TEST_QUERY)
+  def _run_query_and_verify_profile(self, query, query_options,
+      expected_strings_in_profile, not_expected_in_profile=[]):
+    """Run 'query' with given 'query_options'. Assert existence of
+    'expected_strings_in_profile' and nonexistence of 'not_expected_in_profile'
+    in query profile.
+    Caller is reponsible to close self.client at the end of test."""
+    for k, v in query_options.items():
+      self.execute_query_expect_success(self.client, "SET {}='{}';".format(k, v))
+    result = self.execute_query_expect_success(self.client, query)
     for expected_profile in expected_strings_in_profile:
       assert expected_profile in str(result.runtime_profile)
-    self.client.close()
+    for not_expected in not_expected_in_profile:
+      assert not_expected not in str(result.runtime_profile)
 
   @pytest.mark.execute_serially
   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, CPU_TEST_QUERY,
+    self._setup_three_exec_group_cluster(coordinator_test_args)
+    self._run_query_and_verify_profile(CPU_TEST_QUERY, CPU_DOP_OPTIONS,
         ["Executor Group: root.small-group", "EffectiveParallelism: 5",
          "ExecutorGroupsConsidered: 2"])
 
+    # Test disabling COMPUTE_PROCESING_COST and not setting REQUEST_POOL
+    options = copy.deepcopy(CPU_DOP_OPTIONS)
+    options['COMPUTE_PROCESSING_COST'] = 'false'
+    self._run_query_and_verify_profile(CPU_TEST_QUERY, options,
+        ["Executor Group: root.tiny-group", "ExecutorGroupsConsidered: 1",
+         "Verdict: Match"],
+        ["EffectiveParallelism:", "CpuAsk:"])
+
+    # Test that REQUEST_POOL will override executor group selection
+    options['COMPUTE_PROCESSING_COST'] = 'true'
+    options['REQUEST_POOL'] = 'root.large'
+    self._run_query_and_verify_profile(CPU_TEST_QUERY, options,
+        ["Executor Group: root.large-group",
+         ("Verdict: query option REQUEST_POOL=root.large is set. "
+          "Memory and cpu limit checking is skipped."),
+         "EffectiveParallelism: 7", "ExecutorGroupsConsidered: 1"])
+
+    # Test setting REQUEST_POOL and disabling COMPUTE_PROCESSING_COST
+    options['COMPUTE_PROCESSING_COST'] = 'false'
+    options['REQUEST_POOL'] = 'root.large'
+    self._run_query_and_verify_profile(CPU_TEST_QUERY, options,
+        ["Executor Group: root.large-group",
+         ("Verdict: query option REQUEST_POOL=root.large is set. "
+          "Memory and cpu limit checking is skipped."),
+         "ExecutorGroupsConsidered: 1"],
+        ["EffectiveParallelism:", "CpuAsk:"])
+
+    self.client.close()
+
   @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, CPU_TEST_QUERY,
+    self._setup_three_exec_group_cluster(coordinator_test_args)
+    self._run_query_and_verify_profile(CPU_TEST_QUERY, CPU_DOP_OPTIONS,
         ["Executor Group: root.tiny-group", "EffectiveParallelism: 3",
          "ExecutorGroupsConsidered: 1"])
+    self.client.close()
 
   @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, CPU_TEST_QUERY,
+    self._setup_three_exec_group_cluster(coordinator_test_args)
+    self._run_query_and_verify_profile(CPU_TEST_QUERY, CPU_DOP_OPTIONS,
         ["Executor Group: root.large-group", "EffectiveParallelism: 7",
          "ExecutorGroupsConsidered: 3"])
+    self.client.close()
 
   @pytest.mark.execute_serially
   def test_per_exec_group_set_metrics(self):


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

Posted by jo...@apache.org.
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):