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

[impala] branch master updated (a28da34a2 -> 148888e3e)

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

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


    from a28da34a2 IMPALA-9551: (Addendum) disable sorting if select list contains struct containing collection
     new 7a3b8e365 IMPALA-11983: Update stale comments in sorter.h referencing Init()
     new 29ad046d0 IMPALA-11604 (part 1): Model ProcessingCost for PlanNodes & DataSink
     new dafc0fb7a IMPALA-11604 (part 2): Compute Effective Parallelism of Query
     new 27e238c8c IMPALA-11906: [DOCS] Document the support for non-unique primary key
     new afe59f7f0 IMPALA-11854: ImpalaStringWritable's underlying array can't be changed in UDFs
     new c09578a48 IMPALA-11946: Add Thrift HTTP support for external frontend
     new 148888e3e IMPALA-11822: Optimize the Refresh/Invalidate event processing by skipping unnecessary events

The 7 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/sorter.h                            |    4 +-
 be/src/scheduling/scheduler.cc                     |   51 +-
 be/src/scheduling/scheduler.h                      |   28 +-
 be/src/service/impala-server.cc                    |   26 +-
 be/src/service/query-options.cc                    |   16 +
 be/src/service/query-options.h                     |    8 +-
 be/src/util/backend-gflag-util.cc                  |   53 +
 common/thrift/BackendGflags.thrift                 |    6 +
 common/thrift/ImpalaService.thrift                 |   10 +-
 common/thrift/Planner.thrift                       |    7 +
 common/thrift/Query.thrift                         |   11 +
 docs/topics/impala_kudu.xml                        |  147 +-
 .../impala/compat/HiveEsriGeospatialBuiltins.java  |   37 +-
 .../org/apache/impala/analysis/AggregateInfo.java  |    9 +
 .../java/org/apache/impala/analysis/SortInfo.java  |    8 +
 .../impala/catalog/CatalogServiceCatalog.java      |   21 +
 .../org/apache/impala/catalog/HdfsPartition.java   |   21 +-
 .../java/org/apache/impala/catalog/HdfsTable.java  |    8 +
 .../main/java/org/apache/impala/catalog/Table.java |   18 +
 .../org/apache/impala/catalog/TableLoader.java     |    1 +
 .../impala/catalog/events/MetastoreEvents.java     |   19 +-
 .../impala/hive/executor/HiveUdfExecutor.java      |    5 +-
 .../hive/executor/HiveUdfExecutorGeneric.java      |   64 +-
 .../hive/executor/HiveUdfExecutorLegacy.java       |   11 +-
 .../impala/hive/executor/ImpalaBytesWritable.java  |   34 +-
 .../impala/hive/executor/ImpalaStringWritable.java |  117 -
 .../impala/hive/executor/ImpalaTextWritable.java   |   22 +-
 .../impala/hive/executor/JavaUdfDataType.java      |  315 +-
 .../apache/impala/hive/executor/Reloadable.java    |    6 +-
 .../org/apache/impala/planner/AggregationNode.java |   11 +
 .../apache/impala/planner/AnalyticEvalNode.java    |   12 +
 .../apache/impala/planner/BaseProcessingCost.java  |   69 +
 .../impala/planner/BroadcastProcessingCost.java    |   74 +
 .../impala/planner/CardinalityCheckNode.java       |    5 +
 .../java/org/apache/impala/planner/CoreCount.java  |  110 +
 .../org/apache/impala/planner/CostingSegment.java  |  235 +
 .../java/org/apache/impala/planner/DataSink.java   |   33 +-
 .../apache/impala/planner/DataSourceScanNode.java  |    5 +
 .../org/apache/impala/planner/DataStreamSink.java  |   12 +-
 .../org/apache/impala/planner/EmptySetNode.java    |   10 +
 .../org/apache/impala/planner/ExchangeNode.java    |   57 +-
 .../org/apache/impala/planner/HBaseScanNode.java   |    6 +-
 .../org/apache/impala/planner/HBaseTableSink.java  |    6 +
 .../org/apache/impala/planner/HashJoinNode.java    |   27 +
 .../org/apache/impala/planner/HdfsScanNode.java    |    5 +
 .../org/apache/impala/planner/HdfsTableSink.java   |   14 +-
 .../org/apache/impala/planner/JoinBuildSink.java   |   16 +
 .../java/org/apache/impala/planner/JoinNode.java   |   21 +
 .../org/apache/impala/planner/KuduScanNode.java    |    5 +
 .../org/apache/impala/planner/KuduTableSink.java   |    7 +-
 .../apache/impala/planner/NestedLoopJoinNode.java  |   47 +
 .../org/apache/impala/planner/PlanFragment.java    |  440 +-
 .../java/org/apache/impala/planner/PlanNode.java   |   80 +-
 .../org/apache/impala/planner/PlanRootSink.java    |   20 +
 .../java/org/apache/impala/planner/Planner.java    |  104 +-
 .../org/apache/impala/planner/ProcessingCost.java  |  304 +
 .../impala/planner/ScaledProcessingCost.java       |   65 +
 .../java/org/apache/impala/planner/ScanNode.java   |   19 +
 .../java/org/apache/impala/planner/SelectNode.java |    5 +
 .../apache/impala/planner/SingularRowSrcNode.java  |    8 +
 .../java/org/apache/impala/planner/SortNode.java   |    6 +
 .../org/apache/impala/planner/SubplanNode.java     |    5 +
 .../apache/impala/planner/SumProcessingCost.java   |   61 +
 .../java/org/apache/impala/planner/TableSink.java  |    8 +-
 .../java/org/apache/impala/planner/UnionNode.java  |   21 +
 .../java/org/apache/impala/planner/UnnestNode.java |    7 +-
 .../org/apache/impala/service/BackendConfig.java   |   10 +
 .../apache/impala/service/CatalogOpExecutor.java   |   26 +-
 .../java/org/apache/impala/service/Frontend.java   |   54 +-
 .../main/java/org/apache/impala/util/ExprUtil.java |   31 +
 .../impala/customcluster/CustomClusterRunner.java  |    6 +
 .../impala/customcluster/ExternalFrontendTest.java |  195 +
 .../impala/hive/executor/UdfExecutorTest.java      |   45 +-
 .../org/apache/impala/planner/PlannerTest.java     |   19 +-
 ...er-2-groups.xml => fair-scheduler-3-groups.xml} |    2 +
 ...a-site-2-groups.xml => llama-site-3-groups.xml} |   22 +
 .../java/org/apache/impala/BufferAlteringUdf.java  |   85 +
 .../apache/impala/GenericBufferAlteringUdf.java    |  105 +
 .../queries/PlannerTest/tpcds-processing-cost.test | 8739 ++++++++++++++++++++
 .../queries/QueryTest/generic-java-udf.test        |   28 +
 .../queries/QueryTest/java-udf.test                |   35 +
 .../queries/QueryTest/load-generic-java-udfs.test  |    8 +
 .../queries/QueryTest/load-java-udfs.test          |   12 +
 tests/custom_cluster/test_events_custom_configs.py |   34 +-
 tests/custom_cluster/test_executor_groups.py       |   85 +-
 tests/query_test/test_tpcds_queries.py             |   17 +
 86 files changed, 12060 insertions(+), 521 deletions(-)
 delete mode 100644 fe/src/main/java/org/apache/impala/hive/executor/ImpalaStringWritable.java
 copy be/src/util/test-info.cc => fe/src/main/java/org/apache/impala/hive/executor/Reloadable.java (88%)
 create mode 100644 fe/src/main/java/org/apache/impala/planner/BaseProcessingCost.java
 create mode 100644 fe/src/main/java/org/apache/impala/planner/BroadcastProcessingCost.java
 create mode 100644 fe/src/main/java/org/apache/impala/planner/CoreCount.java
 create mode 100644 fe/src/main/java/org/apache/impala/planner/CostingSegment.java
 create mode 100644 fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
 create mode 100644 fe/src/main/java/org/apache/impala/planner/ScaledProcessingCost.java
 create mode 100644 fe/src/main/java/org/apache/impala/planner/SumProcessingCost.java
 create mode 100644 fe/src/test/java/org/apache/impala/customcluster/ExternalFrontendTest.java
 rename fe/src/test/resources/{fair-scheduler-2-groups.xml => fair-scheduler-3-groups.xml} (88%)
 rename fe/src/test/resources/{llama-site-2-groups.xml => llama-site-3-groups.xml} (70%)
 create mode 100644 java/test-hive-udfs/src/main/java/org/apache/impala/BufferAlteringUdf.java
 create mode 100644 java/test-hive-udfs/src/main/java/org/apache/impala/GenericBufferAlteringUdf.java
 create mode 100644 testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test


[impala] 04/07: IMPALA-11906: [DOCS] Document the support for non-unique primary key

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 27e238c8cf31e5c28ad0fe63a1aeabf2a6e05414
Author: Shajini Thayasingh <st...@cloudera.com>
AuthorDate: Mon Mar 6 10:49:16 2023 -0800

    IMPALA-11906: [DOCS] Document the support for non-unique primary key
    
    Incorporated the comments received.
    Added a new sub-section.
    Change-Id: I7b5a452f2199d097077150c012497aa4a3ecf7d9
    Reviewed-on: http://gerrit.cloudera.org:8080/19587
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Abhishek Chennaka <ac...@cloudera.com>
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
---
 docs/topics/impala_kudu.xml | 147 +++++++++++++++++++++++++++++++++++++-------
 1 file changed, 126 insertions(+), 21 deletions(-)

diff --git a/docs/topics/impala_kudu.xml b/docs/topics/impala_kudu.xml
index 0ae80625f..0bc781b3b 100644
--- a/docs/topics/impala_kudu.xml
+++ b/docs/topics/impala_kudu.xml
@@ -203,6 +203,97 @@ under the License.
       <p outputclass="toc inpage"/>
 
     </conbody>
+    <concept id="non_unique_primary_key">
+      <title>Non-unique Primary Keys for Kudu Tables</title>
+      <conbody>
+        <p>Kudu now allows a user to create a non-unique primary key for a table when creating a
+          table. The data engine handles this by appending a system generated auto-incrementing
+          column to the non-unique primary key columns. This is done to guarantee the uniqueness of
+          the primary key. This auto-incrementing column is named as 'auto_incrementing_id' with
+          bigint type. The assignment to it during insertion is automatic.</p>
+      </conbody>
+    </concept>
+    <concept id="create">
+      <title>Create a Kudu Table with a non-unique PRIMARY KEY</title>
+      <conbody>
+        <p>The following example shows creating a table with a non-unique PRIMARY KEY.</p>
+<codeblock>
+CREATE TABLE kudu_tbl1
+(
+ id INT NON UNIQUE PRIMARY KEY,
+ name STRING
+)
+PARTITION BY HASH (id) PARTITIONS 3 STORED as KUDU;</codeblock>
+        <p>The effective PRIMARY KEY in the above case will be {id, auto_increment_id}</p>
+        <note>"auto_incrementing_id" column cannot be added, removed or renamed with ALTER TABLE
+          statements.</note>
+      </conbody>
+    </concept>
+    <concept id="verify">
+      <title>Verify the PRIMARY KEY is non-unique</title>
+      <conbody>
+        <p>You can now check the PRIMARY KEY created is non-unique by running the following DESCRIBE
+          command. A new property "key_unique" shows if the primary key is unique. System generated
+          column "auto_incrementing_id" is shown in the output for the table as a non-unique primary
+          key.</p>
+<codeblock>
+  describe kudu_tbl1
+  +----------------------+--------+---------+-------------+------------+----------+---------------+---------------+---------------------+------------+
+  | name                 | type   | comment | primary_key | key_unique | nullable | default_value | encoding      | compression         | block_size |
+  +----------------------+--------+---------+-------------+------------+----------+---------------+---------------+---------------------+------------+
+  | id                   | int    |         | true        | false      | false    |               | AUTO_ENCODING | DEFAULT_COMPRESSION | 0          |
+  | auto_incrementing_id | bigint |         | true        | false      | false    |               | AUTO_ENCODING | DEFAULT_COMPRESSION | 0          |
+  | name                 | string |         | false       |            | true     |               | AUTO_ENCODING | DEFAULT_COMPRESSION | 0          |
+  +----------------------+--------+---------+-------------+------------+----------+---------------+---------------+---------------------+------------+
+  Fetched 3 row(s) in 4.72s
+</codeblock>
+      </conbody>
+    </concept>
+    <concept id="auto_incrementing_col">
+      <title>Query Auto Incrementing Column</title>
+      <conbody>
+        <p>When you query a table using the SELECT statement, it will not display the system
+          generated auto incrementing column unless the column is explicitly specified in the select
+          list.</p>
+      </conbody>
+    </concept>
+    <concept id="no_primary_key">
+      <title>Create a Kudu table without a PRIMARY KEY attribute</title>
+      <conbody>
+        <p>You can create a Kudu table without specifying a PRIMARY KEY or a PARTITION KEY since
+          they are optional, however you cannot create a Kudu table without specifying both PRIMARY
+          KEY and PARTITION KEY. If you do not specify the primary key attribute, the partition key
+          columns can be promoted as a non-unique primary key. This is possible only if those
+          columns are the beginning columns of the table.</p>
+        <p>In the following example, 'a' and 'b' will be promoted as a non-unique primary key,
+          'auto_incrementing_id' column will be added by Kudu engine. 'a', 'b' and
+          'auto_incrementing_id' form the effective unique composite primary key.</p>
+        <example>
+<codeblock>
+CREATE TABLE auto_table
+(
+ a BIGINT,
+ b STRING,
+)
+PARTITION BY HASH(a, b) PARTITIONS 2 STORED AS KUDU;
+</codeblock>
+          <p>The effective primary key in this case would be {a, b, auto_incrementing_id}</p>
+        </example>
+      </conbody>
+    </concept>
+    <concept id="limitations">
+      <title>Limitations</title>
+      <conbody>
+        <ul>
+          <li>UPSERT operation is not supported for Kudu tables with non-unique primary key. If you
+            run an UPSERT statement for a Kudu table with a non-unique primary key it will fail with
+            an error.</li>
+          <li>Since the auto generated key for each row will be assigned after the row’s data is
+            generated and after the row lands in the tablet, you cannot use this column in the
+            partition key.</li>
+        </ul>
+      </conbody>
+    </concept>
 
     <concept id="kudu_primary_key">
 
@@ -210,14 +301,13 @@ under the License.
 
       <conbody>
 
-        <p>
-          Kudu tables introduce the notion of primary keys to Impala for the first time. The
+        <p> Kudu tables introduce the notion of primary keys to Impala for the first time. The
           primary key is made up of one or more columns, whose values are combined and used as a
-          lookup key during queries. The tuple represented by these columns must be unique and cannot contain any
-          <codeph>NULL</codeph> values, and can never be updated once inserted. For a
-          Kudu table, all the partition key columns must come from the set of
-          primary key columns.
-        </p>
+          lookup key during queries. The primary key can be non-unique. The uniqueness of the
+          primary key is guaranteed by appending a system-generated auto-incrementing column to the
+          non-unique primary key columns. The tuple represented by these columns cannot contain any
+          NULL values, and can never be updated once inserted. For a Kudu table, all the partition
+          key columns must come from the set of primary key columns. </p>
 
         <p>
           The primary key has both physical and logical aspects:
@@ -232,14 +322,13 @@ under the License.
             </p>
           </li>
           <li>
-            <p>
-              On the logical side, the uniqueness constraint allows you to avoid duplicate data in a table.
-              For example, if an <codeph>INSERT</codeph> operation fails partway through, only some of the
-              new rows might be present in the table. You can re-run the same <codeph>INSERT</codeph>, and
-              only the missing rows will be added. Or if data in the table is stale, you can run an
-              <codeph>UPSERT</codeph> statement that brings the data up to date, without the possibility
-              of creating duplicate copies of existing rows.
-            </p>
+            <p> You can insert non-unique data using an INSERT statement but the data saved in Kudu
+              table for each row which will be turned to unique by the system generated
+              auto-incrementing column. If the primary key is non-unique, the uniqueness will not
+              cause insertion failure. However, if the primary key is set as non-unique and if an
+              INSERT operation fails part way through, all rows except the rows with writing errors
+              will be added into the table. The duplicated rows will be added with different values
+              for auto-incrementing columns. </p>
           </li>
         </ul>
 
@@ -273,7 +362,7 @@ under the License.
         </p>
 
 <codeblock>
-  PRIMARY KEY
+[NON UNIQUE] PRIMARY KEY
 | [NOT] NULL
 | ENCODING <varname>codec</varname>
 | COMPRESSION <varname>algorithm</varname>
@@ -300,7 +389,9 @@ under the License.
             combination of values for the columns.
           </p>
 
-          <p conref="../shared/impala_common.xml#common/pk_implies_not_null"/>
+          <p>Because all of the primary key columns must have non-null values, specifying a column
+            in the PRIMARY KEY or NON-UNIQUE PRIMARY KEY clause implicitly adds the NOT NULL
+            attribute to that column.</p>
 
           <p>
             The primary key columns must be the first ones specified in the <codeph>CREATE
@@ -331,6 +422,21 @@ CREATE TABLE pk_at_end
   col3 BOOLEAN,
   PRIMARY KEY (col1)
 ) PARTITION BY HASH(col1) PARTITIONS 2 STORED AS KUDU;
+
+CREATE TABLE pk_inline
+(
+col1 BIGINT [NON UNIQUE] PRIMARY KEY,
+col2 STRING,
+col3 BOOLEAN
+) PARTITION BY HASH(col1) PARTITIONS 2 STORED AS KUDU;
+
+CREATE TABLE pk_at_end
+(
+col1 BIGINT,
+col2 STRING,
+col3 BOOLEAN,
+[NON UNIQUE] PRIMARY KEY (col1)
+) PARTITION BY HASH(col1) PARTITIONS 2 STORED AS KUDU;
 </codeblock>
 
           <p>
@@ -373,11 +479,10 @@ SHOW CREATE TABLE inline_pk_rewritten;
 +------------------------------------------------------------------------------+
 </codeblock>
 
-          <p>
-            The notion of primary key only applies to Kudu tables. Every Kudu table requires a
+          <p> The notion of primary key only applies to Kudu tables. Every Kudu table requires a
             primary key. The primary key consists of one or more columns. You must specify any
-            primary key columns first in the column list.
-          </p>
+            primary key columns first in the column list or specify partition key with the beginning
+            columns of the table. </p>
 
           <p>
             The contents of the primary key columns cannot be changed by an


[impala] 06/07: IMPALA-11946: Add Thrift HTTP support for external frontend

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c09578a4840715854100b72b68b813107a3e25d4
Author: John Sherman <jf...@cloudera.com>
AuthorDate: Wed Feb 1 14:25:37 2023 -0800

    IMPALA-11946: Add Thrift HTTP support for external frontend
    
    - Add enable_external_fe_http flag that defaults to false
      - When true the external frontend service (external_fe_port) will
      expect clients to use http transport.
      - When false the external frontend service will expect binary
      transport.
    
    - Add tests for basic external frontend functionality
    - Add test to ensure the non-external frontend services do not expose
      the ExecutePlannedStatement interface.
    
    Change-Id: I2ad400b1df471e3d61b62d8c0360b27396c26050
    Reviewed-on: http://gerrit.cloudera.org:8080/19537
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Aman Sinha <am...@cloudera.com>
---
 be/src/service/impala-server.cc                    |  26 ++-
 .../impala/customcluster/CustomClusterRunner.java  |   6 +
 .../impala/customcluster/ExternalFrontendTest.java | 195 +++++++++++++++++++++
 3 files changed, 220 insertions(+), 7 deletions(-)

diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index c0add43c2..2e1046c1e 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -163,6 +163,9 @@ DEFINE_int32(external_fe_port, 0, "port on which External Frontend requests are
     "If 0 or less, the External Frontend server is not started. Careful consideration "
     "must be taken when enabling due to the fact that this port is currently always "
     "unauthenticated.");
+DEFINE_bool(enable_external_fe_http, false,
+    "if true enables http transport for external_fe_port otherwise binary transport is "
+    "used");
 
 DEFINE_int32(fe_service_threads, 64,
     "number of threads available to serve client requests");
@@ -3031,17 +3034,26 @@ Status ImpalaServer::Start(int32_t beeswax_port, int32_t hs2_port,
           new RpcEventHandler("external_frontend", exec_env_->metrics()));
       external_fe_processor->setEventHandler(event_handler);
 
+      ThriftServer::TransportType external_fe_port_transport =
+          ThriftServer::TransportType::BINARY;
+      if (FLAGS_enable_external_fe_http) {
+        LOG(INFO) << "External FE endpoint is using HTTP for transport";
+        external_fe_port_transport = ThriftServer::TransportType::HTTP;
+      }
+
       ThriftServerBuilder builder(EXTERNAL_FRONTEND_SERVER_NAME, external_fe_processor,
           external_fe_port);
       ThriftServer* server;
       RETURN_IF_ERROR(
-          builder.auth_provider(
-              AuthManager::GetInstance()->GetExternalFrontendAuthProvider())
-          .metrics(exec_env_->metrics())
-          .max_concurrent_connections(FLAGS_fe_service_threads)
-          .queue_timeout_ms(FLAGS_accepted_client_cnxn_timeout)
-          .idle_poll_period_ms(FLAGS_idle_client_poll_period_s * MILLIS_PER_SEC)
-          .Build(&server));
+          builder
+              .auth_provider(
+                  AuthManager::GetInstance()->GetExternalFrontendAuthProvider())
+              .transport_type(external_fe_port_transport)
+              .metrics(exec_env_->metrics())
+              .max_concurrent_connections(FLAGS_fe_service_threads)
+              .queue_timeout_ms(FLAGS_accepted_client_cnxn_timeout)
+              .idle_poll_period_ms(FLAGS_idle_client_poll_period_s * MILLIS_PER_SEC)
+              .Build(&server));
       external_fe_server_.reset(server);
       external_fe_server_->SetConnectionHandler(this);
     }
diff --git a/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java b/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
index a2547a8a1..ecc1924f2 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
@@ -63,6 +63,12 @@ class CustomClusterRunner {
         impaladArgs, catalogdArgs, statestoredArgs, new HashMap<String, String>(), "");
   }
 
+  public static int StartImpalaCluster(String impaladArgs, String catalogdArgs,
+      String statestoredArgs, String startArgs) throws IOException, InterruptedException {
+    return StartImpalaCluster(impaladArgs, catalogdArgs, statestoredArgs,
+        new HashMap<String, String>(), startArgs);
+  }
+
   /**
    * Starts Impala, setting environment variables in 'env', and passing 'impalad_args',
    * 'catalogd_args', 'statestored_args', and 'startArgs' to start-impala-cluster.py.
diff --git a/fe/src/test/java/org/apache/impala/customcluster/ExternalFrontendTest.java b/fe/src/test/java/org/apache/impala/customcluster/ExternalFrontendTest.java
new file mode 100644
index 000000000..dc9866649
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/customcluster/ExternalFrontendTest.java
@@ -0,0 +1,195 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.customcluster;
+
+import java.util.List;
+
+import org.apache.hive.service.rpc.thrift.TCloseSessionReq;
+import org.apache.hive.service.rpc.thrift.TCloseSessionResp;
+import org.apache.hive.service.rpc.thrift.TColumn;
+import org.apache.hive.service.rpc.thrift.TExecuteStatementReq;
+import org.apache.hive.service.rpc.thrift.TExecuteStatementResp;
+import org.apache.hive.service.rpc.thrift.TFetchOrientation;
+import org.apache.hive.service.rpc.thrift.TFetchResultsReq;
+import org.apache.hive.service.rpc.thrift.TFetchResultsResp;
+import org.apache.hive.service.rpc.thrift.TOpenSessionReq;
+import org.apache.hive.service.rpc.thrift.TOpenSessionResp;
+import org.apache.hive.service.rpc.thrift.TStatus;
+import org.apache.hive.service.rpc.thrift.TStatusCode;
+import org.apache.impala.catalog.Catalog;
+import org.apache.impala.common.FrontendFixture;
+import org.apache.impala.common.ImpalaException;
+import org.apache.impala.service.Frontend.PlanCtx;
+import org.apache.impala.service.Frontend;
+import org.apache.impala.testutil.TestUtils;
+import org.apache.impala.thrift.ImpalaHiveServer2Service;
+import org.apache.impala.thrift.TExecRequest;
+import org.apache.impala.thrift.TExecutePlannedStatementReq;
+import org.apache.impala.thrift.TQueryCtx;
+import org.apache.impala.thrift.TQueryOptions;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.transport.THttpClient;
+import org.apache.thrift.transport.TSocket;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ExternalFrontendTest {
+  private final FrontendFixture feFixture_ = FrontendFixture.instance();
+  private final Frontend frontend_ = feFixture_.frontend();
+  private final int externalFePort = 21159;
+  private final int hs2BinaryPort = 21050;
+  private final int hs2HttpPort = 28000;
+
+  void setup(int port, boolean isHttp) throws Exception {
+    String impaladFlags = "--external_fe_port=" + port;
+    if (isHttp) {
+      impaladFlags += " --enable_external_fe_http";
+    }
+    // Start the impala cluster with the first impalad configured with external frontend
+    // arguments
+    int ret = CustomClusterRunner.StartImpalaCluster(
+        "", "", "", "--per_impalad_args=" + impaladFlags);
+    Assert.assertEquals(
+        "custom cluster failed to start with args: " + impaladFlags, ret, 0);
+  }
+
+  void setupExternalFe() throws Exception { setup(externalFePort, false); }
+
+  void setupExternalFeHttp() throws Exception { setup(externalFePort, true); }
+
+  ImpalaHiveServer2Service.Client createBinaryClient(int port) throws Exception {
+    // Create a binary connection against the hs2 port
+    TSocket sock = new TSocket("localhost", port);
+    sock.open();
+    return new ImpalaHiveServer2Service.Client(new TBinaryProtocol(sock));
+  }
+
+  ImpalaHiveServer2Service.Client createHttpClient(int port) throws Exception {
+    String host_url = "http://localhost:" + port + "/cliservice";
+    THttpClient client = new THttpClient(host_url);
+    return new ImpalaHiveServer2Service.Client(new TBinaryProtocol(client));
+  }
+
+  static TStatus verifySuccess(TStatus status) throws Exception {
+    if (status.getStatusCode() == TStatusCode.SUCCESS_STATUS
+        || status.getStatusCode() == TStatusCode.SUCCESS_WITH_INFO_STATUS) {
+      return status;
+    }
+    throw new Exception(status.toString());
+  }
+
+  void executeTestQuery(ImpalaHiveServer2Service.Client client) throws Exception {
+    executeTestQuery(client, false);
+  }
+
+  TStatus executeTestQueryExpectFailure(ImpalaHiveServer2Service.Client client)
+      throws Exception {
+    return executeTestQuery(client, true);
+  }
+
+  TStatus executeTestQuery(ImpalaHiveServer2Service.Client client,
+      boolean shouldFailExecute) throws Exception {
+    String testStmt = "SELECT 'this is a test, this is only a test'";
+    String expectedValue = "this is a test, this is only a test";
+
+    // Create the TExecRequest
+    TQueryOptions options = new TQueryOptions();
+    options.setExec_single_node_rows_threshold(0);
+
+    TQueryCtx queryCtx =
+        TestUtils.createQueryContext(Catalog.DEFAULT_DB, System.getProperty("user.name"));
+    queryCtx.client_request.setStmt(testStmt);
+    queryCtx.client_request.query_options = options;
+
+    TExecRequest request = null;
+    try {
+      request = frontend_.createExecRequest(new PlanCtx(queryCtx));
+    } catch (ImpalaException e) {
+      Assert.fail(
+          "Failed to create exec request for '" + testStmt + "': " + e.getMessage());
+    }
+
+    // Open Session
+    TOpenSessionResp openResp = client.OpenSession(new TOpenSessionReq());
+    verifySuccess(openResp.getStatus());
+
+    // Create TExecutePlannedStatementReq
+    TExecuteStatementReq executeReq = new TExecuteStatementReq();
+    executeReq.setSessionHandle(openResp.getSessionHandle());
+    executeReq.setStatement(testStmt);
+    TExecutePlannedStatementReq executePlannedReq = new TExecutePlannedStatementReq();
+    executePlannedReq.setStatementReq(executeReq);
+    executePlannedReq.setPlan(request);
+
+    // Execute and Fetch
+    TExecuteStatementResp execResp = client.ExecutePlannedStatement(executePlannedReq);
+    if (shouldFailExecute) {
+      return execResp.getStatus();
+    }
+    verifySuccess(execResp.getStatus());
+
+    TFetchResultsReq fetchReq = new TFetchResultsReq(
+        execResp.getOperationHandle(), TFetchOrientation.FETCH_NEXT, 1000);
+    TFetchResultsResp fetchResp = client.FetchResults(fetchReq);
+    verifySuccess(fetchResp.getStatus());
+
+    // Verify Results
+    List<TColumn> columns = fetchResp.getResults().getColumns();
+    Assert.assertEquals(1, columns.size());
+    Assert.assertEquals(expectedValue, columns.get(0).getStringVal().getValues().get(0));
+
+    // Close Session
+    TCloseSessionResp closeResp =
+        client.CloseSession(new TCloseSessionReq(openResp.getSessionHandle()));
+    return verifySuccess(closeResp.getStatus());
+  }
+
+  @Test
+  public void testExternalFrontendBinary() throws Exception {
+    setupExternalFe();
+    executeTestQuery(createBinaryClient(externalFePort));
+  }
+
+  @Test
+  public void testExternalFrontendHttp() throws Exception {
+    setupExternalFeHttp();
+    executeTestQuery(createHttpClient(externalFePort));
+  }
+
+  @Test
+  public void testExecutePlannedStatementDisallowedNonExternalFe() throws Exception {
+    setupExternalFe();
+    // Try to execute a planned query against the hs2 service (it should fail)
+    TStatus status = executeTestQueryExpectFailure(createBinaryClient(hs2BinaryPort));
+    Assert.assertEquals(status.getStatusCode(), TStatusCode.ERROR_STATUS);
+    Assert.assertTrue(status.toString().contains("Unsupported operation"));
+
+    // Now try to execute against the hs2 http service (it should also fail)
+    status = executeTestQueryExpectFailure(createHttpClient(hs2HttpPort));
+    Assert.assertEquals(status.getStatusCode(), TStatusCode.ERROR_STATUS);
+    Assert.assertTrue(status.toString().contains("Unsupported operation"));
+  }
+
+  @After
+  public void cleanUp() throws Exception {
+    // Restore cluster to state before the test
+    CustomClusterRunner.StartImpalaCluster();
+  }
+}


[impala] 02/07: IMPALA-11604 (part 1): Model ProcessingCost for PlanNodes & DataSink

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 29ad046d05869bed7489bc487636e0f64b3328aa
Author: Qifan Chen <qc...@cloudera.com>
AuthorDate: Thu Sep 22 16:36:34 2022 -0400

    IMPALA-11604 (part 1): Model ProcessingCost for PlanNodes & DataSink
    
    This patch augments IMPALA-10992 by establishing a model to allow the
    weighted total amount of data to process to be used as a new factor in
    the definition and selection of an executor group. We call this model
    ProcessingCost.
    
    ProcessingCost of a PlanNode/DataSink is a weighted amount of data
    processed by that node/sink. The basic ProcessingCost is computed with a
    general formula as follows.
    
      ProcessingCost is a pair: PC(D, N), where D = I * (C + M)
    
      where D is the weighted amount of data processed
            I is the input cardinality
            C is the expression evaluation cost per row.
              Set to total weight of expression evaluation in node/sink.
            M is a materialization cost per row.
              Only used by scan and exchange node. Otherwise, 0.
            N is the number of instances.
              Default to D / 10000000.
    
    In this patch, the weight of each expression evaluation is set to a
    constant of 1. A description of the computation for each kind of
    PlanNode/DataSink is given below.
    
    01. AggregationNode:
        Each AggregateInfo has its C as a sum of grouping expression and
        aggregate expression and then assigned a single ProcessingCost
        individually. These ProcessingCosts then summed to be the Aggregation
        node's ProcessingCost;
    
    02. AnalyticEvalNode:
        C is the sum of the evaluation costs for analytic functions;
    
    03. CardinalityCheckNode:
        Use the general formula, I = 1;
    
    04. DataSourceScanNode:
        Follow the formula from the superclass ScanNode;
    
    05. EmptySetNode:
          I = 0;
    
    06. ExchangeNode:
          M = (average serialized row size) / 1024
    
        A modification of the general formula when in broadcast mode:
          D = D * number of receivers;
    
    07. HashJoinNode:
          probe cost = PC(I0 * C(equiJoin predicate),  N)  +
                       PC(output cardinality * C(otherJoin predicate), N)
          build cost = PC(I1 * C(equi-join predicate), N)
    
        With I0 and I1 as input cardinality of the probe and build side
        accordingly. If the plan node does not have a separate build, ProcessingCost
        is the sum of probe cost and build cost. Otherwise, ProcessingCost is
        equal to probeCost.
    
    08. HbaseScanNode, HdfsScanNode, and KuduScanNode:
        Follow the formula from the superclass ScanNode;
    
    09. Nested loop join node:
        When the right child is not a SingularRowSrcNode:
    
          probe cost = PC(I0 * C(equiJoin predicate), N)  +
                       PC(output cardinality * C(otherJoin predicate), N)
          build cost = PC(I1 * C(equiJoin predicate), N)
    
        When the right child is a SingularRowSrcNode:
    
          probe cost = PC(I0, N)
          build cost = PC(I0 * I1, N)
    
        With I0 and I1 as input cardinality of the probe and build side
        accordingly. If the plan node does not have a separate build, ProcessingCost
        is the sum of probe cost and build cost. Otherwise, ProcessingCost is
        equal to probeCost.
    
    10. ScanNode:
          M = (average row size) / 1024;
    
    11. SelectNode:
        Use the general formula;
    
    12. SingularRowSrcNode:
        Since the node is involved once per input in nested loop join, the
        contribution of this node is computed in nested loop join;
    
    13. SortNode:
        C is the evaluation cost for the sort expression;
    
    14. SubplanNode:
        C is 1. I is the multiplication of the cardinality of the left and
        the right child;
    
    15. Union node:
        C is the cost of result expression evaluation from all non-pass-through
        children;
    
    16. Unnest node:
        I is the cardinality of the containing SubplanNode and C is 1.
    
    17. DataStreamSink:
          M = 1 / num rows per batch.
    
    18. JoinBuildSink:
        ProcessingCost is the build cost of its associated JoinNode.
    
    19. PlanRootSink:
        If result spooling is enabled, C is the cost of output expression
        evaluation. Otherwise. ProcessingCost is zero.
    
    20. TableSink:
        C is the cost of output expression evaluation.
        TableSink subclasses (including HBaseTableSink, HdfsTableSink, and
        KuduTableSink) follows the same formula;
    
    Part 2 of IMPALA-11604 will implement an algorithm that tries to adjust
    the number of instances for each fragment by considering their
    production-consumption ratio, and then finally returns a number
    representing an ideal CPU core count required for a query to run
    efficiently.
    
    Testing:
    - Pass FE tests.
    
    Co-authored-by: Riza Suminto <ri...@cloudera.com>
    
    Change-Id: If32dc770dfffcdd0be2b5555a789a7720952c68a
    Reviewed-on: http://gerrit.cloudera.org:8080/19033
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
    Reviewed-by: Kurt Deschler <kd...@cloudera.com>
    Reviewed-by: Riza Suminto <ri...@cloudera.com>
    Tested-by: Riza Suminto <ri...@cloudera.com>
---
 .../org/apache/impala/analysis/AggregateInfo.java  |   9 +
 .../java/org/apache/impala/analysis/SortInfo.java  |   8 +
 .../org/apache/impala/planner/AggregationNode.java |  11 +
 .../apache/impala/planner/AnalyticEvalNode.java    |  12 +
 .../apache/impala/planner/BaseProcessingCost.java  |  69 +++++
 .../impala/planner/BroadcastProcessingCost.java    |  74 +++++
 .../impala/planner/CardinalityCheckNode.java       |   5 +
 .../java/org/apache/impala/planner/DataSink.java   |  33 ++-
 .../apache/impala/planner/DataSourceScanNode.java  |   5 +
 .../org/apache/impala/planner/DataStreamSink.java  |  12 +-
 .../org/apache/impala/planner/EmptySetNode.java    |  10 +
 .../org/apache/impala/planner/ExchangeNode.java    |  54 +++-
 .../org/apache/impala/planner/HBaseScanNode.java   |   6 +-
 .../org/apache/impala/planner/HBaseTableSink.java  |   6 +
 .../org/apache/impala/planner/HashJoinNode.java    |  27 ++
 .../org/apache/impala/planner/HdfsScanNode.java    |   5 +
 .../org/apache/impala/planner/HdfsTableSink.java   |   8 +-
 .../org/apache/impala/planner/JoinBuildSink.java   |   8 +
 .../java/org/apache/impala/planner/JoinNode.java   |  21 ++
 .../org/apache/impala/planner/KuduScanNode.java    |   5 +
 .../org/apache/impala/planner/KuduTableSink.java   |   7 +-
 .../apache/impala/planner/NestedLoopJoinNode.java  |  47 ++++
 .../java/org/apache/impala/planner/PlanNode.java   |  71 ++++-
 .../org/apache/impala/planner/PlanRootSink.java    |  14 +
 .../org/apache/impala/planner/ProcessingCost.java  | 306 +++++++++++++++++++++
 .../impala/planner/ScaledProcessingCost.java       |  65 +++++
 .../java/org/apache/impala/planner/ScanNode.java   |  19 ++
 .../java/org/apache/impala/planner/SelectNode.java |   5 +
 .../apache/impala/planner/SingularRowSrcNode.java  |   8 +
 .../java/org/apache/impala/planner/SortNode.java   |   6 +
 .../org/apache/impala/planner/SubplanNode.java     |   5 +
 .../apache/impala/planner/SumProcessingCost.java   |  61 ++++
 .../java/org/apache/impala/planner/TableSink.java  |   8 +-
 .../java/org/apache/impala/planner/UnionNode.java  |  21 ++
 .../java/org/apache/impala/planner/UnnestNode.java |   7 +-
 .../main/java/org/apache/impala/util/ExprUtil.java |  19 ++
 36 files changed, 1033 insertions(+), 24 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java b/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java
index 865d1d15b..324d64829 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java
@@ -25,6 +25,8 @@ import org.apache.impala.catalog.AggregateFunction;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.InternalException;
+import org.apache.impala.planner.ProcessingCost;
+import org.apache.impala.util.ExprUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -717,4 +719,11 @@ public class AggregateInfo extends AggregateInfoBase {
 
   @Override
   public AggregateInfo clone() { return new AggregateInfo(this); }
+
+  public ProcessingCost computeProcessingCost(String label, long inputCardinality) {
+    float weight = ExprUtil.computeExprsTotalCost(getGroupingExprs())
+        + ExprUtil.computeExprsTotalCost(getAggregateExprs());
+
+    return ProcessingCost.basicCost(label, inputCardinality, weight);
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/SortInfo.java b/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
index 995c686eb..e6d802ede 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
@@ -26,7 +26,9 @@ import java.util.Set;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.TreeNode;
 import org.apache.impala.planner.PlanNode;
+import org.apache.impala.planner.ProcessingCost;
 import org.apache.impala.thrift.TSortingOrder;
+import org.apache.impala.util.ExprUtil;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
@@ -313,4 +315,10 @@ public class SortInfo {
     }
     return result;
   }
+
+  public ProcessingCost computeProcessingCost(String label, long inputCardinality) {
+    float weight = ExprUtil.computeExprsTotalCost(getSortExprs());
+
+    return ProcessingCost.basicCost(label, inputCardinality, weight);
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
index b9741ff7b..e4eea634c 100644
--- a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
@@ -77,6 +77,7 @@ public class AggregationNode extends PlanNode {
   private boolean useStreamingPreagg_ = false;
 
   // Resource profiles for each aggregation class.
+  // Set in computeNodeResourceProfile().
   private List<ResourceProfile> resourceProfiles_;
 
   // Conservative minimum size of hash table for low-cardinality aggregations.
@@ -505,6 +506,16 @@ public class AggregationNode extends PlanNode {
     return output;
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = ProcessingCost.zero();
+    for (AggregateInfo aggInfo : aggInfos_) {
+      ProcessingCost aggCost =
+          aggInfo.computeProcessingCost(getDisplayLabel(), getChild(0).getCardinality());
+      processingCost_ = ProcessingCost.sumCost(processingCost_, aggCost);
+    }
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     resourceProfiles_ = Lists.newArrayListWithCapacity(aggInfos_.size());
diff --git a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
index d4fb6abf4..2ab23a643 100644
--- a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
@@ -45,6 +45,7 @@ import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
 import org.apache.impala.thrift.TQueryOptions;
+import org.apache.impala.util.ExprUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -357,6 +358,17 @@ public class AnalyticEvalNode extends PlanNode {
     return output.toString();
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    // The total cost per row is the sum of the evaluation costs for analytic functions,
+    // partition by equal and order by equal predicate. 'partitionByEq_' and 'orderByEq_'
+    // are excluded since the input data stream is already partitioned and sorted within
+    // each partition (see notes on class AnalyticEvalNode in analytic-eval-node.h).
+    float totalCostToEvalOneRow = ExprUtil.computeExprsTotalCost(analyticFnCalls_);
+    processingCost_ = ProcessingCost.basicCost(
+        getDisplayLabel(), getCardinality(), totalCostToEvalOneRow);
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     Preconditions.checkNotNull(
diff --git a/fe/src/main/java/org/apache/impala/planner/BaseProcessingCost.java b/fe/src/main/java/org/apache/impala/planner/BaseProcessingCost.java
new file mode 100644
index 000000000..3f6de5ffd
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/BaseProcessingCost.java
@@ -0,0 +1,69 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.planner;
+
+/**
+ * A basic implementation of {@link ProcessingCost} that takes account expression cost
+ * and average row size as per-row costing weight.
+ */
+public class BaseProcessingCost extends ProcessingCost {
+  private final long cardinality_;
+  private final float exprsCost_;
+  private final float materializationCost_;
+
+  public BaseProcessingCost(
+      long cardinality, float exprsCost, float materializationCost) {
+    // TODO: materializationCost accommodate ProcessingCost where row width should be
+    // factor in. Currently, ProcessingCost of ScanNode, ExchangeNode, and DataStreamSink
+    // has row width factored in through materialization parameter here. Investigate if
+    // other operator need to have its row width factored in as well and whether we should
+    // have specific 'rowWidth' parameter here.
+    cardinality_ = cardinality;
+    exprsCost_ = exprsCost;
+    materializationCost_ = materializationCost;
+  }
+
+  private float costFactor() { return exprsCost_ + materializationCost_; }
+
+  @Override
+  public long getTotalCost() {
+    // Total cost must be non-negative.
+    return (long) Math.ceil(Math.max(cardinality_, 0) * costFactor());
+  }
+
+  @Override
+  public boolean isValid() {
+    return cardinality_ >= 0;
+  }
+
+  @Override
+  public ProcessingCost clone() {
+    return new BaseProcessingCost(cardinality_, exprsCost_, materializationCost_);
+  }
+
+  @Override
+  public String getDetails() {
+    StringBuilder output = new StringBuilder();
+    output.append(super.getDetails());
+    output.append(" cardinality=")
+        .append(cardinality_)
+        .append(" cost-factor=")
+        .append(costFactor());
+    return output.toString();
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/BroadcastProcessingCost.java b/fe/src/main/java/org/apache/impala/planner/BroadcastProcessingCost.java
new file mode 100644
index 000000000..d031ecea6
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/BroadcastProcessingCost.java
@@ -0,0 +1,74 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.planner;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.impala.util.MathUtil;
+
+import java.util.function.Supplier;
+
+/**
+ * Similar as {@link ScaledProcessingCost}, but the multiple (countSupplier) represent
+ * fragment instance count associated with this ProcessingCost and may change after the
+ * object construction.
+ * <p>
+ * countSupplier must always return positive value.
+ */
+public class BroadcastProcessingCost extends ProcessingCost {
+  private final ProcessingCost childProcessingCost_;
+
+  protected BroadcastProcessingCost(
+      ProcessingCost cost, Supplier<Integer> countSupplier) {
+    Preconditions.checkArgument(
+        cost.isValid(), "BroadcastProcessingCost: cost is invalid!");
+    childProcessingCost_ = cost;
+    setNumInstanceExpected(countSupplier);
+  }
+
+  @Override
+  public long getTotalCost() {
+    return MathUtil.saturatingMultiply(
+        childProcessingCost_.getTotalCost(), getNumInstancesExpected());
+  }
+
+  @Override
+  public boolean isValid() {
+    return getNumInstancesExpected() > 0;
+  }
+
+  @Override
+  public ProcessingCost clone() {
+    return new BroadcastProcessingCost(childProcessingCost_, numInstanceSupplier_);
+  }
+
+  @Override
+  public String getExplainString(String detailPrefix, boolean fullExplain) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(detailPrefix);
+    sb.append("BroadcastCost(");
+    sb.append(getNumInstancesExpected());
+    sb.append("): ");
+    sb.append(getDetails());
+    if (fullExplain) {
+      sb.append("\n");
+      sb.append(childProcessingCost_.getExplainString(detailPrefix + "  ", true));
+    }
+    return sb.toString();
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/CardinalityCheckNode.java b/fe/src/main/java/org/apache/impala/planner/CardinalityCheckNode.java
index 00cd67fdd..78d32d8ac 100644
--- a/fe/src/main/java/org/apache/impala/planner/CardinalityCheckNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/CardinalityCheckNode.java
@@ -78,6 +78,11 @@ public class CardinalityCheckNode extends PlanNode {
     msg.setCardinality_check_node(cardinalityCheckNode);
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = computeDefaultProcessingCost();
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     nodeResourceProfile_ = ResourceProfile.noReservation(0);
diff --git a/fe/src/main/java/org/apache/impala/planner/DataSink.java b/fe/src/main/java/org/apache/impala/planner/DataSink.java
index 2cacb4d3d..4b665b4c4 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSink.java
@@ -45,6 +45,10 @@ public abstract class DataSink {
   // set in computeResourceProfile()
   protected ResourceProfile resourceProfile_ = ResourceProfile.invalid();
 
+  // A total processing cost across all instances of this plan node.
+  // Set in computeProcessingCost() for a meaningful value.
+  protected ProcessingCost processingCost_ = ProcessingCost.invalid();
+
   /**
    * Return an explain string for the DataSink. Each line of the explain will be prefixed
    * by "prefix".
@@ -56,6 +60,19 @@ public abstract class DataSink {
     if (explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
       output.append(detailPrefix);
       output.append(resourceProfile_.getExplainString());
+      if (ProcessingCost.isComputeCost(queryOptions)) {
+        // Show processing cost total.
+        output.append(" cost=");
+        if (processingCost_.isValid()) {
+          output.append(processingCost_.getTotalCost());
+          if (explainLevel.ordinal() >= TExplainLevel.VERBOSE.ordinal()) {
+            output.append("\n");
+            output.append(processingCost_.getExplainString(detailPrefix, false));
+          }
+        } else {
+          output.append("<invalid>");
+        }
+      }
       output.append("\n");
     }
     return output.toString();
@@ -107,15 +124,29 @@ public abstract class DataSink {
   public void setFragment(PlanFragment fragment) { fragment_ = fragment; }
   public PlanFragment getFragment() { return fragment_; }
   public ResourceProfile getResourceProfile() { return resourceProfile_; }
+  public ProcessingCost getProcessingCost() { return processingCost_; }
+
+  public abstract void computeProcessingCost(TQueryOptions queryOptions);
 
   /**
    * Compute the resource profile for an instance of this DataSink.
    */
   public abstract void computeResourceProfile(TQueryOptions queryOptions);
 
+  /**
+   * Set number of rows consumed and produced data fields in processing cost.
+   */
+  public void computeRowConsumptionAndProductionToCost() {
+    Preconditions.checkState(processingCost_.isValid(),
+        "Processing cost of DataSink " + fragment_.getId() + ":" + getLabel()
+            + " is invalid!");
+    long inputOutputCardinality = fragment_.getPlanRoot().getCardinality();
+    processingCost_.setNumRowToConsume(inputOutputCardinality);
+    processingCost_.setNumRowToProduce(inputOutputCardinality);
+  }
+
   /**
    * Collect all expressions evaluated by this data sink.
    */
   public abstract void collectExprs(List<Expr> exprs);
-
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
index c89b2760f..7cafa1840 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
@@ -332,6 +332,11 @@ public class DataSourceScanNode extends ScanNode {
         new TScanRange(), Lists.newArrayList(new TScanRangeLocation(hostIndex))));
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = computeDefaultProcessingCost();
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // This node fetches a thrift representation of the rows from the data
diff --git a/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java b/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
index 06b8f31b0..fcf1f266e 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
@@ -71,9 +71,7 @@ public class DataStreamSink extends DataSink {
   private long estimateOutboundRowBatchBuffers(TQueryOptions queryOptions) {
     int numChannels =
         outputPartition_.isPartitioned() ? exchNode_.getFragment().getNumInstances() : 1;
-    long rowBatchSize = queryOptions.isSetBatch_size() && queryOptions.batch_size > 0 ?
-        queryOptions.batch_size :
-        PlanNode.DEFAULT_ROWBATCH_SIZE;
+    long rowBatchSize = PlanNode.getRowBatchSize(queryOptions);
     long avgOutboundRowBatchSize = Math.min(
         (long) Math.ceil(rowBatchSize * exchNode_.getAvgSerializedRowSize(exchNode_)),
         PlanNode.ROWBATCH_MAX_MEM_USAGE);
@@ -86,6 +84,14 @@ public class DataStreamSink extends DataSink {
     return bufferSize;
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    // The sending part of the processing cost for the exchange node.
+    processingCost_ =
+        ProcessingCost.basicCost(getLabel() + "(" + exchNode_.getDisplayLabel() + ")",
+            exchNode_.getCardinality(), 0, exchNode_.estimateSerializationCostPerRow());
+  }
+
   @Override
   public void computeResourceProfile(TQueryOptions queryOptions) {
     long estimatedMem = estimateOutboundRowBatchBuffers(queryOptions);
diff --git a/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java b/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
index b39a2ae15..02bdcff1f 100644
--- a/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
@@ -59,6 +59,11 @@ public class EmptySetNode extends PlanNode {
     computeStats(analyzer);
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = ProcessingCost.zero();
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // TODO: add an estimate
@@ -78,4 +83,9 @@ public class EmptySetNode extends PlanNode {
 
   @Override
   protected boolean displayCardinality(TExplainLevel detailLevel) { return false; }
+
+  @Override
+  protected boolean isLeafNode() {
+    return true;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
index c3856956f..44232b53d 100644
--- a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
@@ -29,6 +29,7 @@ import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
 import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TSortInfo;
+import org.apache.impala.util.ExprUtil;
 
 import com.google.common.base.Preconditions;
 
@@ -66,7 +67,7 @@ public class ExchangeNode extends PlanNode {
     return mergeInfo_ != null;
   }
 
-  private boolean isBroadcastExchange() {
+  protected boolean isBroadcastExchange() {
     // If the output of the sink is not partitioned but the target fragment is
     // partitioned, then the data exchange is broadcast.
     Preconditions.checkState(!children_.isEmpty());
@@ -178,6 +179,39 @@ public class ExchangeNode extends PlanNode {
         (exchInput.getTupleIds().size() * PER_TUPLE_SERIALIZATION_OVERHEAD);
   }
 
+  // Return the number of sending instances of this exchange.
+  public int getNumSenders() {
+    Preconditions.checkState(!children_.isEmpty());
+    Preconditions.checkNotNull(children_.get(0).getFragment());
+    return children_.get(0).getFragment().getNumInstances();
+  }
+
+  // Return the number of receiving instances of this exchange.
+  public int getNumReceivers() {
+    DataSink sink = fragment_.getSink();
+    if (sink == null) return 1;
+    return sink.getFragment().getNumInstances();
+  }
+
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    // The computation for the processing cost for exchange splits into two parts:
+    //   1. The sending processing cost which is computed in the DataStreamSink of the
+    //      bottom sending fragment;
+    //   2. The receiving processing cost in the top receiving fragment which is computed
+    //      here.
+    // Assume serialization and deserialization costs per row are equal.
+    float conjunctsCost = ExprUtil.computeExprsTotalCost(conjuncts_);
+    float materializationCost = estimateSerializationCostPerRow();
+    processingCost_ = ProcessingCost.basicCost(getDisplayLabel() + "(receiving)",
+        getChild(0).getCardinality(), conjunctsCost, materializationCost);
+
+    if (isBroadcastExchange()) {
+      processingCost_ = ProcessingCost.broadcastCost(processingCost_,
+          () -> getNumReceivers());
+    }
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // For non-merging exchanges, one row batch queue is maintained for row
@@ -193,9 +227,7 @@ public class ExchangeNode extends PlanNode {
     // the system load. This makes it difficult to accurately estimate the
     // memory usage at runtime. The following estimates assume that memory usage will
     // lean towards the soft limits.
-    Preconditions.checkState(!children_.isEmpty());
-    Preconditions.checkNotNull(children_.get(0).getFragment());
-    int numSenders = children_.get(0).getFragment().getNumInstances();
+    int numSenders = getNumSenders();
     long estimatedTotalQueueByteSize = estimateTotalQueueByteSize(numSenders);
     long estimatedDeferredRPCQueueSize = estimateDeferredRPCQueueSize(queryOptions,
         numSenders);
@@ -205,12 +237,18 @@ public class ExchangeNode extends PlanNode {
     nodeResourceProfile_ = ResourceProfile.noReservation(estimatedMem);
   }
 
+  /**
+   * Estimate per-row serialization/deserialization cost as 1 per 1KB.
+   */
+  protected float estimateSerializationCostPerRow() {
+    return (float) getAvgSerializedRowSize(this) / 1024;
+  }
+
   // Returns the estimated size of the deferred batch queue (in bytes) by
   // assuming that at least one row batch rpc payload per sender is queued.
   private long estimateDeferredRPCQueueSize(TQueryOptions queryOptions,
       int numSenders) {
-    long rowBatchSize = queryOptions.isSetBatch_size() && queryOptions.batch_size > 0
-        ? queryOptions.batch_size : DEFAULT_ROWBATCH_SIZE;
+    long rowBatchSize = getRowBatchSize(queryOptions);
     // Set an upper limit based on estimated cardinality.
     if (getCardinality() > 0) rowBatchSize = Math.min(rowBatchSize, getCardinality());
     long avgRowBatchByteSize = Math.min(
@@ -251,6 +289,10 @@ public class ExchangeNode extends PlanNode {
 
   @Override
   protected void toThrift(TPlanNode msg) {
+    if (processingCost_.isValid() && processingCost_ instanceof BroadcastProcessingCost) {
+      Preconditions.checkState(
+          getNumReceivers() == processingCost_.getNumInstancesExpected());
+    }
     msg.node_type = TPlanNodeType.EXCHANGE_NODE;
     msg.exchange_node = new TExchangeNode();
     for (TupleId tid: tupleIds_) {
diff --git a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
index 2b764e602..96460b14b 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
@@ -59,7 +59,6 @@ import org.apache.impala.thrift.TScanRangeLocation;
 import org.apache.impala.thrift.TScanRangeLocationList;
 import org.apache.impala.thrift.TScanRangeSpec;
 import org.apache.impala.util.BitUtil;
-import org.apache.impala.util.ExecutorMembershipSnapshot;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -655,6 +654,11 @@ public class HBaseScanNode extends ScanNode {
     }
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = computeScanProcessingCost(queryOptions);
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     FeHBaseTable tbl = (FeHBaseTable) desc_.getTable();
diff --git a/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java b/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java
index cfbb335e7..abeb27f09 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java
@@ -54,6 +54,12 @@ public class HBaseTableSink extends TableSink {
     return "HBASE WRITER";
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    // The processing cost to export rows.
+    processingCost_ = computeDefaultProcessingCost();
+  }
+
   @Override
   public void computeResourceProfile(TQueryOptions queryOptions) {
     resourceProfile_ = ResourceProfile.noReservation(0);
diff --git a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
index 0fe91eed3..7aeee7433 100644
--- a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
@@ -37,6 +37,7 @@ import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
 import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.util.BitUtil;
+import org.apache.impala.util.ExprUtil;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.MoreObjects;
@@ -314,4 +315,30 @@ public class HashJoinNode extends JoinNode {
         .setMaxRowBufferBytes(maxRowBufferSize).build();
     return Pair.create(probeProfile, buildProfile);
   }
+
+  @Override
+  public Pair<ProcessingCost, ProcessingCost> computeJoinProcessingCost() {
+    // TODO: The cost should consider conjuncts_ as well.
+    // Assume 'eqJoinConjuncts_' will be applied to all rows from lhs and rhs side,
+    // and 'otherJoinConjuncts_' to the resultant rows.
+    float eqJoinPredicateEvalCost = ExprUtil.computeExprsTotalCost(eqJoinConjuncts_);
+    float otherJoinPredicateEvalCost =
+        ExprUtil.computeExprsTotalCost(otherJoinConjuncts_);
+
+    // Compute the processing cost for lhs.
+    ProcessingCost probeProcessingCost =
+        ProcessingCost.basicCost(getDisplayLabel() + " Probe side (eqJoinConjuncts_)",
+            getChild(0).getCardinality(), eqJoinPredicateEvalCost);
+    if (otherJoinPredicateEvalCost > 0) {
+      probeProcessingCost = ProcessingCost.sumCost(probeProcessingCost,
+          ProcessingCost.basicCost(getDisplayLabel() + " Probe side(otherJoinConjuncts_)",
+              getCardinality(), otherJoinPredicateEvalCost));
+    }
+
+    // Compute the processing cost for rhs.
+    ProcessingCost buildProcessingCost =
+        ProcessingCost.basicCost(getDisplayLabel() + " Build side",
+            getChild(1).getCardinality(), eqJoinPredicateEvalCost);
+    return Pair.create(probeProcessingCost, buildProcessingCost);
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
index 9931d5849..6ba0b3c31 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -2108,6 +2108,11 @@ public class HdfsScanNode extends ScanNode {
     return output.toString();
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = computeScanProcessingCost(queryOptions);
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // Update 'useMtScanNode_' before any return cases. It's used in BE.
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
index 76b68a1ff..399e45829 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
@@ -41,10 +41,8 @@ import org.apache.impala.thrift.TTableSinkType;
 import org.apache.impala.util.BitUtil;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -142,6 +140,12 @@ public class HdfsTableSink extends TableSink {
     externalOutputPartitionDepth_ = partitionDepth;
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    // The processing cost to export rows.
+    processingCost_ = computeDefaultProcessingCost();
+  }
+
   @Override
   public void computeResourceProfile(TQueryOptions queryOptions) {
     PlanNode inputNode = fragment_.getPlanRoot();
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
index c8cc056d0..feed83381 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
@@ -141,11 +141,19 @@ public class JoinBuildSink extends DataSink {
                                        joinNode_.getFragment().getNumInstances();
   }
 
+  public boolean isShared() { return joinNode_.canShareBuild(); }
+
   @Override
   protected String getLabel() {
     return "JOIN BUILD";
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    // The processing cost to export rows.
+    processingCost_ = joinNode_.computeJoinProcessingCost().second;
+  }
+
   @Override
   public void computeResourceProfile(TQueryOptions queryOptions) {
     resourceProfile_ = joinNode_.computeJoinResourceProfile(queryOptions).second;
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinNode.java b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
index f31045235..02d39e991 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
@@ -909,6 +909,19 @@ public abstract class JoinNode extends PlanNode {
     return result;
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    Pair<ProcessingCost, ProcessingCost> probeBuildCost = computeJoinProcessingCost();
+    if (hasSeparateBuild()) {
+      // All build resource consumption is accounted for in the separate builder.
+      processingCost_ = probeBuildCost.first;
+    } else {
+      // Both build and profile resources are accounted for in the node.
+      processingCost_ =
+          ProcessingCost.sumCost(probeBuildCost.first, probeBuildCost.second);
+    }
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     Pair<ResourceProfile, ResourceProfile> profiles =
@@ -931,6 +944,14 @@ public abstract class JoinNode extends PlanNode {
   public abstract Pair<ResourceProfile, ResourceProfile> computeJoinResourceProfile(
       TQueryOptions queryOptions);
 
+  /**
+   * Helper method to compute the processing cost for the join that can be
+   * called from the builder or the join node. Returns a pair of the probe
+   * processing cost and the build processing cost.
+   * Does not modify the state of this node.
+   */
+  public abstract Pair<ProcessingCost, ProcessingCost> computeJoinProcessingCost();
+
   /* Helper to return all predicates as a string. */
   public String getAllPredicatesAsString(TExplainLevel level) {
     return "Conjuncts=" + Expr.getExplainString(getConjuncts(), level)
diff --git a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
index b65388ac6..6ec5255cb 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -395,6 +395,11 @@ public class KuduScanNode extends ScanNode {
     }
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = computeScanProcessingCost(queryOptions);
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // The bulk of memory used by Kudu scan node is generally utilized by the
diff --git a/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java b/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java
index eba08b5ce..52fa82d65 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java
@@ -18,7 +18,6 @@
 
 package org.apache.impala.planner;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.impala.analysis.DescriptorTable;
@@ -94,6 +93,12 @@ public class KuduTableSink extends TableSink {
     return "KUDU WRITER";
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    // The processing cost to export rows.
+    processingCost_ = computeDefaultProcessingCost();
+  }
+
   @Override
   public void computeResourceProfile(TQueryOptions queryOptions) {
     // The major chunk of memory used by this node is untracked. Part of which
diff --git a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
index b36cb9fc5..ecd5c03ae 100644
--- a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
@@ -31,6 +31,7 @@ import org.apache.impala.thrift.TNestedLoopJoinNode;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
 import org.apache.impala.thrift.TQueryOptions;
+import org.apache.impala.util.ExprUtil;
 
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
@@ -78,6 +79,7 @@ public class NestedLoopJoinNode extends JoinNode {
   @Override
   public Pair<ResourceProfile, ResourceProfile> computeJoinResourceProfile(
       TQueryOptions queryOptions) {
+    // TODO: This seems a bug below that the total data is not divided by numInstances_.
     long perInstanceMemEstimate;
     if (getChild(1).getCardinality() == -1 || getChild(1).getAvgRowSize() == -1
         || numNodes_ == 0) {
@@ -92,6 +94,51 @@ public class NestedLoopJoinNode extends JoinNode {
     return Pair.create(ResourceProfile.noReservation(0), buildProfile);
   }
 
+  @Override
+  public Pair<ProcessingCost, ProcessingCost> computeJoinProcessingCost() {
+    // TODO: Make this general regardless of SingularRowSrcNode exist or not.
+    // TODO: The cost should consider conjuncts_ as well.
+    ProcessingCost probeProcessingCost = ProcessingCost.zero();
+    ProcessingCost buildProcessingCost = ProcessingCost.zero();
+    if (getChild(1) instanceof SingularRowSrcNode) {
+      // Compute the processing cost for lhs.
+      probeProcessingCost =
+          ProcessingCost.basicCost(getDisplayLabel() + "(c0, singularRowSrc) Probe side",
+              getChild(0).getCardinality(), 0);
+
+      // Compute the processing cost for rhs.
+      buildProcessingCost = ProcessingCost.basicCost(
+          getDisplayLabel() + "(c0, singularRowSrc) Build side per probe",
+          getChild(1).getCardinality(), 0);
+      // Multiply by the number of probes
+      buildProcessingCost = ProcessingCost.scaleCost(
+          buildProcessingCost, Math.max(0, getChild(0).getCardinality()));
+    } else {
+      // Assume 'eqJoinConjuncts_' will be applied to all rows from lhs side,
+      // and 'otherJoinConjuncts_' to the resultant rows.
+      float eqJoinPredicateEvalCost = ExprUtil.computeExprsTotalCost(eqJoinConjuncts_);
+      float otherJoinPredicateEvalCost =
+          ExprUtil.computeExprsTotalCost(otherJoinConjuncts_);
+
+      // Compute the processing cost for lhs.
+      probeProcessingCost = ProcessingCost.basicCost(
+          getDisplayLabel() + "(c0, non-singularRowSrc, eqJoinConjuncts_) Probe side",
+          getChild(0).getCardinality(), eqJoinPredicateEvalCost);
+
+      probeProcessingCost = ProcessingCost.sumCost(probeProcessingCost,
+          ProcessingCost.basicCost(getDisplayLabel()
+                  + "(c0, non-singularRowSrc, otherJoinConjuncts_) Probe side",
+              getCardinality(), otherJoinPredicateEvalCost));
+
+      // Compute the processing cost for rhs, assuming 'eqJoinConjuncts_' will be applied
+      // to all rows from rhs side.
+      buildProcessingCost = ProcessingCost.basicCost(
+          getDisplayLabel() + "(c0, non-singularRowSrc) Build side",
+          getChild(1).getCardinality(), eqJoinPredicateEvalCost);
+    }
+    return Pair.create(probeProcessingCost, buildProcessingCost);
+  }
+
   @Override
   protected String getNodeExplainString(String prefix, String detailPrefix,
       TExplainLevel detailLevel) {
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanNode.java b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
index ae19ce278..6dc9709c5 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -28,7 +28,6 @@ import java.util.Set;
 
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.BinaryPredicate;
-import org.apache.impala.analysis.CollectionTableRef;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.ExprId;
 import org.apache.impala.analysis.ExprSubstitutionMap;
@@ -49,6 +48,7 @@ import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TSortingOrder;
 import org.apache.impala.util.BitUtil;
+import org.apache.impala.util.ExprUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -154,6 +154,10 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   // Runtime filters assigned to this node.
   protected List<RuntimeFilter> runtimeFilters_ = new ArrayList<>();
 
+  // A total processing cost across all instances of this plan node.
+  // Gets set correctly in computeProcessingCost().
+  protected ProcessingCost processingCost_ = ProcessingCost.invalid();
+
   protected PlanNode(PlanNodeId id, List<TupleId> tupleIds, String displayName) {
     this(id, displayName);
     tupleIds_.addAll(tupleIds);
@@ -235,6 +239,7 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   public void setAssignedConjuncts(Set<ExprId> conjuncts) {
     assignedConjuncts_ = conjuncts;
   }
+  public ProcessingCost getProcessingCost() { return processingCost_; }
 
   /**
    * Set the limit_ to the given limit_ only if the limit_ hasn't been set, or the new limit_
@@ -342,6 +347,13 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
       expBuilder.append(nodeResourceProfile_.getExplainString());
       expBuilder.append("\n");
 
+      if (ProcessingCost.isComputeCost(queryOptions) && processingCost_.isValid()
+          && detailLevel.ordinal() >= TExplainLevel.VERBOSE.ordinal()) {
+        // Print processing cost.
+        expBuilder.append(processingCost_.getExplainString(detailPrefix, false));
+        expBuilder.append("\n");
+      }
+
       // Print tuple ids, row size and cardinality.
       expBuilder.append(detailPrefix + "tuple-ids=");
       for (int i = 0; i < tupleIds_.size(); ++i) {
@@ -358,10 +370,19 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     if (displayCardinality) {
       if (detailLevel == TExplainLevel.STANDARD) expBuilder.append(detailPrefix);
       expBuilder.append("row-size=")
-        .append(PrintUtils.printBytes(Math.round(avgRowSize_)))
-        .append(" cardinality=")
-        .append(PrintUtils.printEstCardinality(cardinality_))
-        .append("\n");
+          .append(PrintUtils.printBytes(Math.round(avgRowSize_)))
+          .append(" cardinality=")
+          .append(PrintUtils.printEstCardinality(cardinality_));
+      if (ProcessingCost.isComputeCost(queryOptions)) {
+        // Show processing cost total.
+        expBuilder.append(" cost=");
+        if (processingCost_.isValid()) {
+          expBuilder.append(processingCost_.getTotalCost());
+        } else {
+          expBuilder.append("<invalid>");
+        }
+      }
+      expBuilder.append("\n");
     }
 
     if (detailLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
@@ -653,6 +674,13 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     return cardinality;
   }
 
+  // Default implementation of computing the total data processed in bytes.
+  protected ProcessingCost computeDefaultProcessingCost() {
+    Preconditions.checkState(hasValidStats());
+    return ProcessingCost.basicCost(getDisplayLabel(), getInputCardinality(),
+        ExprUtil.computeExprsTotalCost(getConjuncts()));
+  }
+
   public static long capCardinalityAtLimit(long cardinality, long limit) {
     return cardinality == -1 ? limit : Math.min(cardinality, limit);
   }
@@ -882,14 +910,41 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     }
   }
 
+  public abstract void computeProcessingCost(TQueryOptions queryOptions);
+
   /**
    * Compute peak resources consumed when executing this PlanNode, initializing
-   * 'nodeResourceProfile_'. May only be called after this PlanNode has been placed in
-   * a PlanFragment because the cost computation is dependent on the enclosing fragment's
-   * data partition.
+   * 'nodeResourceProfile_' and 'processingCost_'. May only be called after this PlanNode
+   * has been placed in a PlanFragment because the cost computation is dependent on the
+   * enclosing fragment's data partition.
    */
   public abstract void computeNodeResourceProfile(TQueryOptions queryOptions);
 
+  /**
+   * Determine whether a PlanNode is a leaf node within the plan tree.
+   * @return true if a PlanNode is a leaf node within the plan tree.
+   */
+  protected boolean isLeafNode() { return false; }
+
+  /**
+   * Set number of rows consumed and produced data fields in processing cost.
+   */
+  public void computeRowConsumptionAndProductionToCost() {
+    Preconditions.checkState(processingCost_.isValid(),
+        "Processing cost of PlanNode " + getDisplayLabel() + " is invalid!");
+    processingCost_.setNumRowToConsume(getInputCardinality());
+    processingCost_.setNumRowToProduce(getCardinality());
+  }
+
+  /**
+   * Get row batch size after considering 'batch_size' query option.
+   */
+  protected static long getRowBatchSize(TQueryOptions queryOptions) {
+    return (queryOptions.isSetBatch_size() && queryOptions.batch_size > 0) ?
+        queryOptions.batch_size :
+        PlanNode.DEFAULT_ROWBATCH_SIZE;
+  }
+
   /**
    * Wrapper class to represent resource profiles during different phases of execution.
    */
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
index 53887b738..133aec186 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
@@ -26,6 +26,7 @@ import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanRootSink;
 import org.apache.impala.thrift.TQueryOptions;
+import org.apache.impala.util.ExprUtil;
 import org.apache.log4j.Logger;
 
 import com.google.common.base.Preconditions;
@@ -67,6 +68,19 @@ public class PlanRootSink extends DataSink {
     return "ROOT";
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    if (queryOptions.isSpool_query_results() && queryOptions.getScratch_limit() != 0
+        && !BackendConfig.INSTANCE.getScratchDirs().isEmpty()) {
+      // The processing cost to buffer these many rows in root.
+      processingCost_ =
+          ProcessingCost.basicCost(getLabel(), fragment_.getPlanRoot().getCardinality(),
+              ExprUtil.computeExprsTotalCost(outputExprs_));
+    } else {
+      processingCost_ = ProcessingCost.zero();
+    }
+  }
+
   /**
    * Computes and sets the {@link ResourceProfile} for this PlanRootSink. If result
    * spooling is disabled, a ResourceProfile is returned with no reservation or buffer
diff --git a/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java b/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
new file mode 100644
index 000000000..df695bddc
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
@@ -0,0 +1,306 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.planner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.math.LongMath;
+
+import org.apache.impala.thrift.TQueryOptions;
+
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.function.Supplier;
+
+/**
+ * A base class that encapsulate processing cost which models a total cost or amount
+ * of work shared across all instances of specific {@link PlanNode}, {@link DataSink}, or
+ * {@link PlanFragment}.
+ */
+public abstract class ProcessingCost implements Cloneable {
+  public static ProcessingCost invalid() { return new BaseProcessingCost(-1, 1, 0); }
+  public static ProcessingCost zero() { return new BaseProcessingCost(0, 1, 0); }
+
+  public static ProcessingCost maxCost(ProcessingCost a, ProcessingCost b) {
+    return (a.getTotalCost() >= b.getTotalCost()) ? a : b;
+  }
+
+  public static ProcessingCost sumCost(ProcessingCost a, ProcessingCost b) {
+    return new SumProcessingCost(a, b);
+  }
+
+  public static ProcessingCost scaleCost(ProcessingCost cost, long factor) {
+    return new ScaledProcessingCost(cost, factor);
+  }
+
+  public static ProcessingCost broadcastCost(
+      ProcessingCost cost, Supplier<Integer> numInstanceSupplier) {
+    return new BroadcastProcessingCost(cost, numInstanceSupplier);
+  }
+
+  protected static void tryAdjustConsumerParallelism(int nodeStepCount,
+      int minParallelism, int maxParallelism, ProcessingCost producer,
+      ProcessingCost consumer) {
+    Preconditions.checkState(consumer.getNumInstancesExpected() > 0);
+    Preconditions.checkState(producer.getNumInstancesExpected() > 0);
+    if (producer.getCostPerRowProduced() > 0
+        && (consumer.canReducedBy(nodeStepCount, minParallelism, producer)
+            || (consumer.canIncreaseBy(nodeStepCount, maxParallelism, producer)))) {
+      // Adjust consumer's concurrency following producer's parallelism and their
+      // produce-consume rate ratio.
+      float consProdRatio = consumer.consumerProducerRatio(producer);
+      int adjustedCount = (int) Math.ceil(consProdRatio
+                              * producer.getNumInstancesExpected() / nodeStepCount)
+          * nodeStepCount;
+      final int finalCount =
+          Math.max(minParallelism, Math.min(maxParallelism, adjustedCount));
+      consumer.setNumInstanceExpected(() -> finalCount);
+    } else if (maxParallelism < consumer.getNumInstancesExpected()) {
+      consumer.setNumInstanceExpected(() -> maxParallelism);
+    }
+  }
+
+  private static ProcessingCost computeValidBaseCost(
+      long cardinality, float exprsCost, float materializationCost) {
+    return new BaseProcessingCost(
+        Math.max(0, cardinality), exprsCost, materializationCost);
+  }
+
+  public static ProcessingCost basicCost(
+      String label, long cardinality, float exprsCost, float materializationCost) {
+    ProcessingCost processingCost =
+        computeValidBaseCost(cardinality, exprsCost, materializationCost);
+    processingCost.setLabel(label);
+    return processingCost;
+  }
+
+  public static ProcessingCost basicCost(
+      String label, long cardinality, float exprsCost) {
+    ProcessingCost processingCost = computeValidBaseCost(cardinality, exprsCost, 0);
+    processingCost.setLabel(label);
+    return processingCost;
+  }
+
+  public static boolean isComputeCost(TQueryOptions queryOptions) {
+    // TODO: Replace with proper check in IMPALA-11604 part 2.
+    return false;
+  }
+
+  /**
+   * Merge multiple ProcessingCost into a single new ProcessingCost.
+   * <p>
+   * The resulting ProcessingCost will have the total cost, number of rows produced,
+   * and number of rows consumed as a sum of respective properties of all ProcessingCost
+   * in the given list. Meanwhile, the number of instances expected is the maximum among
+   * all ProcessingCost is the list.
+   *
+   * @param costs list of all ProcessingCost to merge.
+   * @return A new combined ProcessingCost.
+   */
+  protected static ProcessingCost fullMergeCosts(List<ProcessingCost> costs) {
+    Preconditions.checkNotNull(costs);
+    Preconditions.checkArgument(!costs.isEmpty());
+
+    ProcessingCost resultingCost = ProcessingCost.zero();
+    long inputCardinality = 0;
+    long outputCardinality = 0;
+    int maxProducerParallelism = 1;
+    for (ProcessingCost cost : costs) {
+      resultingCost = ProcessingCost.sumCost(resultingCost, cost);
+      inputCardinality += cost.getNumRowToConsume();
+      outputCardinality += cost.getNumRowToProduce();
+      maxProducerParallelism =
+          Math.max(maxProducerParallelism, cost.getNumInstancesExpected());
+    }
+    resultingCost.setNumRowToConsume(inputCardinality);
+    resultingCost.setNumRowToProduce(outputCardinality);
+    final int finalProducerParallelism = maxProducerParallelism;
+    resultingCost.setNumInstanceExpected(() -> finalProducerParallelism);
+    return resultingCost;
+  }
+
+  protected Supplier<Integer> numInstanceSupplier_ = null;
+  private long numRowToProduce_ = 0;
+  private long numRowToConsume_ = 0;
+  private String label_ = null;
+  private boolean isSetNumRowToProduce_ = false;
+  private boolean isSetNumRowToConsume_ = false;
+
+  public abstract long getTotalCost();
+
+  public abstract boolean isValid();
+
+  public abstract ProcessingCost clone();
+
+  public String getDetails() {
+    StringBuilder output = new StringBuilder();
+    output.append("cost-total=")
+        .append(getTotalCost())
+        .append(" max-instances=")
+        .append(getNumInstanceMax());
+    if (hasAdjustedInstanceCount()) {
+      output.append(" adj-instances=").append(getNumInstancesExpected());
+    }
+    output.append(" cost/inst=")
+        .append(getPerInstanceCost())
+        .append(" #cons:#prod=")
+        .append(numRowToConsume_)
+        .append(":")
+        .append(numRowToProduce_);
+    if (isSetNumRowToConsume_ && isSetNumRowToProduce_) {
+      output.append(" reduction=").append(getReduction());
+    }
+    if (isSetNumRowToConsume_) {
+      output.append(" cost/cons=").append(getCostPerRowConsumed());
+    }
+    if (isSetNumRowToProduce_) {
+      output.append(" cost/prod=").append(getCostPerRowProduced());
+    }
+    return output.toString();
+  }
+
+  public String debugString() {
+    StringBuilder output = new StringBuilder();
+    if (label_ != null) {
+      output.append(label_);
+      output.append("=");
+    }
+    output.append(this);
+    return output.toString();
+  }
+
+  @Override
+  public String toString() {
+    return "{" + getDetails() + "}";
+  }
+
+  public String getExplainString(String detailPrefix, boolean fullExplain) {
+    return detailPrefix + getDetails();
+  }
+
+  public void setNumInstanceExpected(Supplier<Integer> countSupplier) {
+    Preconditions.checkArgument(
+        countSupplier.get() > 0, "Number of instance must be greater than 0!");
+    numInstanceSupplier_ = countSupplier;
+  }
+
+  public int getNumInstancesExpected() {
+    return hasAdjustedInstanceCount() ? numInstanceSupplier_.get() : getNumInstanceMax();
+  }
+
+  private boolean hasAdjustedInstanceCount() {
+    return numInstanceSupplier_ != null && numInstanceSupplier_.get() > 0;
+  }
+
+  private int getNumInstanceMax() {
+    // TODO: replace minProcessingCostPerThread with backend flag.
+    long minProcessingCostPerThread = 10000000L;
+    long maxInstance = LongMath.divide(getTotalCost(),
+        minProcessingCostPerThread, RoundingMode.CEILING);
+    if (maxInstance > 0) {
+      return maxInstance < Integer.MAX_VALUE ? (int) maxInstance : Integer.MAX_VALUE;
+    } else {
+      return 1;
+    }
+  }
+
+  /**
+   * Set num rows to produce.
+   *
+   * @param numRowToProduce Number of rows to produce by plan node or data sink associated
+   *     with this cost. Assume 0 rows if negative value is given.
+   */
+  public void setNumRowToProduce(long numRowToProduce) {
+    numRowToProduce_ = Math.max(0, numRowToProduce);
+    isSetNumRowToProduce_ = true;
+  }
+
+  /**
+   * Set num rows to consume.
+   *
+   * @param numRowToConsume Number of rows to consume by plan node or data sink associated
+   *     with this cost. Assume 0 rows if negative value is given.
+   */
+  protected void setNumRowToConsume(long numRowToConsume) {
+    numRowToConsume_ = Math.max(0, numRowToConsume);
+    isSetNumRowToConsume_ = true;
+  }
+
+  public void setLabel(String label) { label_ = label; }
+  public long getNumRowToConsume() { return numRowToConsume_; }
+  public long getNumRowToProduce() { return numRowToProduce_; }
+
+  private int getPerInstanceCost() {
+    Preconditions.checkState(getNumInstancesExpected() > 0);
+    return (int) Math.ceil((float) getTotalCost() / getNumInstancesExpected());
+  }
+
+  private float getReduction() {
+    return (float) numRowToConsume_ / Math.max(1, numRowToProduce_);
+  }
+
+  private float getCostPerRowProduced() {
+    return (float) getTotalCost() / Math.max(1, numRowToProduce_);
+  }
+
+  private float getCostPerRowConsumed() {
+    return (float) getTotalCost() / Math.max(1, numRowToConsume_);
+  }
+
+  private float instanceRatio(ProcessingCost other) {
+    Preconditions.checkState(getNumInstancesExpected() > 0);
+    return (float) getNumInstancesExpected() / other.getNumInstancesExpected();
+  }
+
+  private float consumerProducerRatio(ProcessingCost other) {
+    return getCostPerRowConsumed() / Math.max(1, other.getCostPerRowProduced());
+  }
+
+  private boolean isAtLowestInstanceRatio(
+      int nodeStepCount, int minParallelism, ProcessingCost other) {
+    if (getNumInstancesExpected() - nodeStepCount < minParallelism) {
+      return true;
+    } else {
+      float lowerRatio = (float) (getNumInstancesExpected() - nodeStepCount)
+          / other.getNumInstancesExpected();
+      return lowerRatio < consumerProducerRatio(other);
+    }
+  }
+
+  private boolean isAtHighestInstanceRatio(
+      int nodeStepCount, int maxInstance, ProcessingCost other) {
+    if (getNumInstancesExpected() + nodeStepCount > maxInstance) {
+      return true;
+    } else {
+      float higherRatio = (float) (getNumInstancesExpected() + nodeStepCount)
+          / other.getNumInstancesExpected();
+      return higherRatio > consumerProducerRatio(other);
+    }
+  }
+
+  private boolean canReducedBy(
+      int nodeStepCount, int minParallelism, ProcessingCost other) {
+    return !isAtLowestInstanceRatio(nodeStepCount, minParallelism, other)
+        && consumerProducerRatio(other) < instanceRatio(other);
+  }
+
+  private boolean canIncreaseBy(
+      int nodeStepCount, int maxInstance, ProcessingCost other) {
+    return !isAtHighestInstanceRatio(nodeStepCount, maxInstance, other)
+        && consumerProducerRatio(other) > instanceRatio(other);
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/ScaledProcessingCost.java b/fe/src/main/java/org/apache/impala/planner/ScaledProcessingCost.java
new file mode 100644
index 000000000..421ee9e64
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/ScaledProcessingCost.java
@@ -0,0 +1,65 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.planner;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.impala.util.MathUtil;
+
+public class ScaledProcessingCost extends ProcessingCost {
+  private final ProcessingCost cost_;
+  private final long multiplier_;
+
+  protected ScaledProcessingCost(ProcessingCost cost, long multiplier) {
+    Preconditions.checkArgument(cost.isValid(), "ScaledProcessingCost: cost is invalid!");
+    Preconditions.checkArgument(
+        multiplier >= 0, "ScaledProcessingCost: multiplier must be non-negative!");
+    cost_ = cost;
+    multiplier_ = multiplier;
+  }
+
+  @Override
+  public long getTotalCost() {
+    return MathUtil.saturatingMultiply(cost_.getTotalCost(), multiplier_);
+  }
+
+  @Override
+  public boolean isValid() {
+    return true;
+  }
+
+  @Override
+  public ProcessingCost clone() {
+    return new ScaledProcessingCost(cost_, multiplier_);
+  }
+
+  @Override
+  public String getExplainString(String detailPrefix, boolean fullExplain) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(detailPrefix);
+    sb.append("ScaledCost(");
+    sb.append(multiplier_);
+    sb.append("): ");
+    sb.append(getDetails());
+    if (fullExplain) {
+      sb.append("\n");
+      sb.append(cost_.getExplainString(detailPrefix + "  ", true));
+    }
+    return sb.toString();
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/ScanNode.java b/fe/src/main/java/org/apache/impala/planner/ScanNode.java
index d250eb8db..be12b293c 100644
--- a/fe/src/main/java/org/apache/impala/planner/ScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/ScanNode.java
@@ -40,6 +40,7 @@ import org.apache.impala.thrift.TNetworkAddress;
 import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TScanRangeSpec;
 import org.apache.impala.thrift.TTableStats;
+import org.apache.impala.util.ExprUtil;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.MoreObjects;
@@ -343,6 +344,24 @@ abstract public class ScanNode extends PlanNode {
     }
     return maxScannerThreads;
   }
+
+  protected ProcessingCost computeScanProcessingCost(TQueryOptions queryOptions) {
+    return ProcessingCost.basicCost(getDisplayLabel(), getInputCardinality(),
+        ExprUtil.computeExprsTotalCost(conjuncts_), rowMaterializationCost());
+  }
+
+  /**
+   * Estimate per-row cost as 1 per 1KB row size.
+   * <p>
+   * This reflect deserialization/copy cost per row.
+   */
+  private float rowMaterializationCost() { return getAvgRowSize() / 1024; }
+
+  @Override
+  protected boolean isLeafNode() {
+    return true;
+  }
+
   /**
    * Returns true if this node has conjuncts to be evaluated by Impala against the scan
    * tuple.
diff --git a/fe/src/main/java/org/apache/impala/planner/SelectNode.java b/fe/src/main/java/org/apache/impala/planner/SelectNode.java
index fa888f47f..1bde0d41a 100644
--- a/fe/src/main/java/org/apache/impala/planner/SelectNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SelectNode.java
@@ -117,6 +117,11 @@ public class SelectNode extends PlanNode {
 
   public void setSelectivity(double value) { selectivity_ = value; }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = computeDefaultProcessingCost();
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // The select node initializes a single row-batch which it recycles on every
diff --git a/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java b/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
index e9c498417..f37b67c11 100644
--- a/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
@@ -23,6 +23,7 @@ import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
 import org.apache.impala.thrift.TQueryOptions;
+import org.apache.impala.util.ExprUtil;
 
 import com.google.common.base.Preconditions;
 
@@ -66,6 +67,13 @@ public class SingularRowSrcNode extends PlanNode {
     numInstances_ = containingSubplanNode_.getChild(0).getNumInstances();
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = ProcessingCost.basicCost(getDisplayLabel(),
+        containingSubplanNode_.getChild(0).getCardinality(),
+        ExprUtil.computeExprsTotalCost(getConjuncts()));
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // TODO: add an estimate
diff --git a/fe/src/main/java/org/apache/impala/planner/SortNode.java b/fe/src/main/java/org/apache/impala/planner/SortNode.java
index 5f12dee45..6d8ad8b03 100644
--- a/fe/src/main/java/org/apache/impala/planner/SortNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SortNode.java
@@ -453,6 +453,12 @@ public class SortNode extends PlanNode {
     return offset_ != 0 ? prefix + "offset: " + Long.toString(offset_) + "\n" : "";
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ =
+        getSortInfo().computeProcessingCost(getDisplayLabel(), getCardinality());
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     Preconditions.checkState(hasValidStats());
diff --git a/fe/src/main/java/org/apache/impala/planner/SubplanNode.java b/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
index eb57df628..09a6ac7d8 100644
--- a/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
@@ -96,6 +96,11 @@ public class SubplanNode extends PlanNode {
     cardinality_ = capCardinalityAtLimit(cardinality_);
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = ProcessingCost.basicCost(getDisplayLabel(), getCardinality(), 0);
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // TODO: add an estimate
diff --git a/fe/src/main/java/org/apache/impala/planner/SumProcessingCost.java b/fe/src/main/java/org/apache/impala/planner/SumProcessingCost.java
new file mode 100644
index 000000000..fcbd6dfab
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/SumProcessingCost.java
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.planner;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.impala.util.MathUtil;
+
+public class SumProcessingCost extends ProcessingCost {
+  private final ProcessingCost cost1_;
+  private final ProcessingCost cost2_;
+
+  protected SumProcessingCost(ProcessingCost cost1, ProcessingCost cost2) {
+    Preconditions.checkArgument(cost1.isValid(), "SumProcessingCost: cost1 is invalid!");
+    Preconditions.checkArgument(cost2.isValid(), "SumProcessingCost: cost2 is invalid!");
+    cost1_ = cost1;
+    cost2_ = cost2;
+  }
+
+  @Override
+  public long getTotalCost() {
+    return MathUtil.saturatingAdd(cost1_.getTotalCost(), cost2_.getTotalCost());
+  }
+
+  @Override
+  public boolean isValid() {
+    return true;
+  }
+
+  @Override
+  public ProcessingCost clone() {
+    return new SumProcessingCost(cost1_, cost2_);
+  }
+
+  @Override
+  public String getExplainString(String detailPrefix, boolean fullExplain) {
+    StringBuilder output = new StringBuilder();
+    output.append(detailPrefix).append("SumCost: ").append(getDetails());
+    if (fullExplain) {
+      String nextPrefix = detailPrefix + "  ";
+      output.append("\n").append(cost1_.getExplainString(nextPrefix, true));
+      output.append("\n").append(cost2_.getExplainString(nextPrefix, true));
+    }
+    return output.toString();
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/TableSink.java b/fe/src/main/java/org/apache/impala/planner/TableSink.java
index cc12bd995..0f34ddb20 100644
--- a/fe/src/main/java/org/apache/impala/planner/TableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/TableSink.java
@@ -17,7 +17,6 @@
 
 package org.apache.impala.planner;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.impala.analysis.Expr;
@@ -28,6 +27,7 @@ import org.apache.impala.catalog.FeTable;
 import org.apache.impala.common.Pair;
 import org.apache.impala.thrift.TSinkAction;
 import org.apache.impala.thrift.TSortingOrder;
+import org.apache.impala.util.ExprUtil;
 
 import com.google.common.base.Preconditions;
 
@@ -160,4 +160,10 @@ public abstract class TableSink extends DataSink {
           "Cannot create data sink into table of type: " + table.getClass().getName());
     }
   }
+
+  protected ProcessingCost computeDefaultProcessingCost() {
+    // TODO: consider including materialization cost into the returned cost.
+    return ProcessingCost.basicCost(getLabel(), fragment_.getPlanRoot().getCardinality(),
+        ExprUtil.computeExprsTotalCost(outputExprs_));
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/UnionNode.java b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
index b1dbe7212..c8f45dd7b 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnionNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
@@ -33,6 +33,7 @@ import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
 import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TUnionNode;
+import org.apache.impala.util.ExprUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -148,6 +149,19 @@ public class UnionNode extends PlanNode {
     }
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    // Compute the cost for materializing child rows and use that to figure out
+    // the total data processed. Assume the costs of processing pass-through rows are 0.
+    float totalMaterializeCost = 0;
+    for (int i = firstMaterializedChildIdx_; i < resultExprLists_.size(); i++) {
+      totalMaterializeCost += ExprUtil.computeExprsTotalCost(resultExprLists_.get(i));
+    }
+
+    processingCost_ =
+        ProcessingCost.basicCost(getDisplayLabel(), cardinality_, totalMaterializeCost);
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // The union node directly returns the rows for children marked as pass
@@ -375,4 +389,11 @@ public class UnionNode extends PlanNode {
       }
     }
   }
+
+  @Override
+  protected boolean isLeafNode() {
+    // Union node is being scheduled the same as scan node.
+    // See Scheduler::CreateCollocatedAndScanInstances() in scheduler.cc.
+    return true;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
index dd6eb33f5..919d849e3 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
@@ -17,7 +17,6 @@
 
 package org.apache.impala.planner;
 
-import java.util.Comparator;
 import java.util.List;
 
 import org.apache.impala.analysis.Analyzer;
@@ -96,6 +95,12 @@ public class UnnestNode extends PlanNode {
     cardinality_ = capCardinalityAtLimit(cardinality_);
   }
 
+  @Override
+  public void computeProcessingCost(TQueryOptions queryOptions) {
+    processingCost_ = ProcessingCost.basicCost(
+        getDisplayLabel(), containingSubplanNode_.getChild(0).getCardinality(), 0);
+  }
+
   @Override
   public void computeNodeResourceProfile(TQueryOptions queryOptions) {
     // TODO: add an estimate
diff --git a/fe/src/main/java/org/apache/impala/util/ExprUtil.java b/fe/src/main/java/org/apache/impala/util/ExprUtil.java
index 934fabd0b..2b34835da 100644
--- a/fe/src/main/java/org/apache/impala/util/ExprUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/ExprUtil.java
@@ -27,9 +27,12 @@ import org.apache.impala.analysis.StringLiteral;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.InternalException;
+import org.apache.impala.service.BackendConfig;
 import org.apache.impala.service.FeSupport;
 import org.apache.impala.thrift.TColumnValue;
 
+import java.util.List;
+
 public class ExprUtil {
   /**
    * Converts a UTC timestamp to UNIX microseconds.
@@ -102,4 +105,20 @@ public class ExprUtil {
     toUtcTimestamp.analyze(analyzer);
     return toUtcTimestamp;
   }
+
+  // Compute total cost for a list of expressions. Return 0 for a null list.
+  public static float computeExprsTotalCost(List<? extends Expr> exprs) {
+    // TODO: Implement the cost for conjunts once the implemetation for
+    // 'Expr' is in place.
+    if (exprs == null) return 0;
+    return exprs.size();
+  }
+
+  public static float computeExprCost(Expr e) {
+    if (e == null) return 0;
+    return 1;
+    // TODO Implement a function that can take into consideration of data types,
+    // expressions and potentially LLVM translation in BE. The function must also
+    // run fast.
+  }
 }


[impala] 07/07: IMPALA-11822: Optimize the Refresh/Invalidate event processing by skipping unnecessary events

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 148888e3ed4f97292499b2e6ee8d5a756dc648d9
Author: Sai Hemanth Gantasala <sa...@cloudera.com>
AuthorDate: Wed Feb 8 12:01:41 2023 -0800

    IMPALA-11822: Optimize the Refresh/Invalidate event processing by skipping unnecessary events
    
    Added a new variable 'lastRefreshEventId' in the
    catalogD's table/partition object to store the
    latest event id before loading the table/partition.
    This will be updated frequently based on refresh
    or invalidate commands. This variable can be used
    in the event processor to decide whether to process
    or skip the reload event by comparing it with the
    current event id. It is enough to store the refresh
    event's event id, invalidate event anyway flushes
    out the object from cache.
    
    Note: Need to enable two configs for this
    optimization to work:
    1) enable_reload_events=true
    2) enable_sync_to_latest_event_on_ddls=true
    
    Testing: Added a test to fire few reload events via
    HMS API and then verify in the event processor that
    some older events are skipped.
    
    Change-Id: I905957683a96c3ea01ab4bf043d6658ce37b7574
    Reviewed-on: http://gerrit.cloudera.org:8080/19484
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../impala/catalog/CatalogServiceCatalog.java      | 21 +++++++++++++
 .../org/apache/impala/catalog/HdfsPartition.java   | 21 +++++++++++--
 .../java/org/apache/impala/catalog/HdfsTable.java  |  8 +++++
 .../main/java/org/apache/impala/catalog/Table.java | 18 ++++++++++++
 .../org/apache/impala/catalog/TableLoader.java     |  1 +
 .../impala/catalog/events/MetastoreEvents.java     | 19 +++++++++++-
 .../apache/impala/service/CatalogOpExecutor.java   | 26 ++++++++++++-----
 tests/custom_cluster/test_events_custom_configs.py | 34 ++++++++++++++++++++--
 8 files changed, 135 insertions(+), 13 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index c87cda9a2..53f423f4c 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -2567,6 +2567,13 @@ public class CatalogServiceCatalog extends Catalog {
       }
       tbl.setCatalogVersion(newCatalogVersion);
       LOG.info(String.format("Refreshed table metadata: %s", tbl.getFullName()));
+      // Set the last refresh event id as current HMS event id since all the metadata
+      // until the current HMS event id is refreshed at this point.
+      if (currentHmsEventId > eventId) {
+        tbl.setLastRefreshEventId(currentHmsEventId);
+      } else {
+        tbl.setLastRefreshEventId(eventId);
+      }
       return tbl.toTCatalogObject(resultType);
     } finally {
       context.stop();
@@ -3837,4 +3844,18 @@ public class CatalogServiceCatalog extends Catalog {
     return syncToLatestEventFactory_;
   }
 
+  public boolean isPartitionLoadedAfterEvent(String dbName, String tableName,
+      Partition msPartition, long eventId) {
+    try {
+      HdfsPartition hdfsPartition = getHdfsPartition(dbName, tableName, msPartition);
+      if (hdfsPartition.getLastRefreshEventId() > eventId) {
+        return true;
+      }
+    } catch (CatalogException ex) {
+      LOG.warn("Encountered an exception while the partition's last refresh event id: "
+          + dbName + "." + tableName + ". Ignoring further processing and try to " +
+          "reload the partition.", ex);
+    }
+    return false;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsPartition.java b/fe/src/main/java/org/apache/impala/catalog/HdfsPartition.java
index 034790fd9..bbd4bb129 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsPartition.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsPartition.java
@@ -746,6 +746,10 @@ public class HdfsPartition extends CatalogObjectImpl
   // -1 means there is no previous compaction event or compaction is not supported
   private final long lastCompactionId_;
 
+  // The last refresh event id of the partition
+  // -1 means there is no previous refresh event happened
+  private final long lastRefreshEventId_;
+
   /**
    * Constructor.  Needed for third party extensions that want to use their own builder
    * to construct the object.
@@ -765,7 +769,8 @@ public class HdfsPartition extends CatalogObjectImpl
         encodedInsertFileDescriptors, encodedDeleteFileDescriptors, location,
         isMarkedCached, accessLevel, hmsParameters, cachedMsPartitionDescriptor,
         partitionStats, hasIncrementalStats, numRows, writeId,
-        inFlightEvents, /*createEventId=*/-1L, /*lastCompactionId*/-1L);
+        inFlightEvents, /*createEventId=*/-1L, /*lastCompactionId*/-1L,
+        /*lastRefreshEventId*/-1L);
   }
 
   protected HdfsPartition(HdfsTable table, long id, long prevId, String partName,
@@ -777,7 +782,8 @@ public class HdfsPartition extends CatalogObjectImpl
       boolean isMarkedCached, TAccessLevel accessLevel, Map<String, String> hmsParameters,
       CachedHmsPartitionDescriptor cachedMsPartitionDescriptor,
       byte[] partitionStats, boolean hasIncrementalStats, long numRows, long writeId,
-      InFlightEvents inFlightEvents, long createEventId, long lastCompactionId) {
+      InFlightEvents inFlightEvents, long createEventId, long lastCompactionId,
+      long lastRefreshEventId) {
     table_ = table;
     id_ = id;
     prevId_ = prevId;
@@ -798,6 +804,7 @@ public class HdfsPartition extends CatalogObjectImpl
     inFlightEvents_ = inFlightEvents;
     createEventId_ = createEventId;
     lastCompactionId_ = lastCompactionId;
+    lastRefreshEventId_ = lastRefreshEventId;
     if (partName == null && id_ != CatalogObjectsConstants.PROTOTYPE_PARTITION_ID) {
       partName_ = FeCatalogUtils.getPartitionName(this);
     } else {
@@ -807,6 +814,8 @@ public class HdfsPartition extends CatalogObjectImpl
 
   public long getCreateEventId() { return createEventId_; }
 
+  public long getLastRefreshEventId() { return lastRefreshEventId_; }
+
   @Override // FeFsPartition
   public HdfsStorageDescriptor getInputFormatDescriptor() {
     return fileFormatDescriptor_;
@@ -1251,6 +1260,7 @@ public class HdfsPartition extends CatalogObjectImpl
     // is not active.
     private long createEventId_ = -1L;
     private long lastCompactionId_ = -1L;
+    private long lastRefreshEventId_ = -1L;
     private InFlightEvents inFlightEvents_ = new InFlightEvents();
 
     @Nullable
@@ -1318,7 +1328,7 @@ public class HdfsPartition extends CatalogObjectImpl
           encodedDeleteFileDescriptors_, location_, isMarkedCached_, accessLevel_,
           hmsParameters_, cachedMsPartitionDescriptor_, partitionStats_,
           hasIncrementalStats_, numRows_, writeId_, inFlightEvents_, createEventId_,
-          lastCompactionId_);
+          lastCompactionId_, lastRefreshEventId_);
     }
 
     public Builder setId(long id) {
@@ -1331,6 +1341,11 @@ public class HdfsPartition extends CatalogObjectImpl
       return this;
     }
 
+    public Builder setLastRefreshEventId(long eventId) {
+      lastRefreshEventId_ = eventId;
+      return this;
+    }
+
     public Builder setPrevId(long prevId) {
       prevId_ = prevId;
       return this;
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index 538e813c6..da83a47ac 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -2876,6 +2876,13 @@ public class HdfsTable extends Table implements FeFsTable {
     FsPermissionCache permissionCache = new FsPermissionCache();
     Map<HdfsPartition.Builder, HdfsPartition> partBuilderToPartitions = new HashMap<>();
     Set<HdfsPartition.Builder> partBuildersFileMetadataRefresh = new HashSet<>();
+    long latestEventId = -1L;
+    try {
+      latestEventId = client.getCurrentNotificationEventId().getEventId();
+    } catch (TException exception) {
+      LOG.warn(String.format("Unable to fetch latest event id from HMS: %s",
+          exception.getMessage()));
+    }
     for (Map.Entry<Partition, HdfsPartition> entry : hmsPartsToHdfsParts.entrySet()) {
       Partition hmsPartition = entry.getKey();
       HdfsPartition oldPartition = entry.getValue();
@@ -2887,6 +2894,7 @@ public class HdfsTable extends Table implements FeFsTable {
       if (oldPartition != null) {
         partBuilder.setFileDescriptors(oldPartition);
       }
+      partBuilder.setLastRefreshEventId(latestEventId);
       switch (fileMetadataLoadOpts) {
         case FORCE_LOAD:
           partBuildersFileMetadataRefresh.add(partBuilder);
diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java
index aaad451df..21b766ce8 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Table.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Table.java
@@ -197,6 +197,8 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
   // not by reading this flag and without acquiring read lock on table object
   protected volatile long lastSyncedEventId_ = -1;
 
+  protected volatile long lastRefreshEventId_ = -1L;
+
   protected Table(org.apache.hadoop.hive.metastore.api.Table msTable, Db db,
       String name, String owner) {
     msTable_ = msTable;
@@ -1028,4 +1030,20 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
    * Clears the in-progress modifications in case of failures.
    */
   public void resetInProgressModification() { }
+
+  public long getLastRefreshEventId() { return lastRefreshEventId_; }
+
+  public void setLastRefreshEventId(long eventId) {
+    if (eventId > lastRefreshEventId_) {
+      lastRefreshEventId_ = eventId;
+    }
+    LOG.debug("last refreshed event id for table: {} set to: {}", getFullName(),
+        lastRefreshEventId_);
+    // TODO: Should we reset lastSyncedEvent Id if it is less than event Id?
+    // If we don't reset it - we may start syncing table from an event id which
+    // is less than refresh event id
+    if (lastSyncedEventId_ < eventId) {
+      setLastSyncedEventId(eventId);
+    }
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/TableLoader.java b/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
index bdd97a3f6..6ccae1fe1 100644
--- a/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
+++ b/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
@@ -153,6 +153,7 @@ public class TableLoader {
         MetastoreEventsProcessor.syncToLatestEventId(catalog_, table,
             catalog_.getEventFactoryForSyncToLatestEvent(), metrics_);
       }
+      table.setLastRefreshEventId(latestEventId);
     } catch (TableLoadingException e) {
       table = IncompleteTable.createFailedMetadataLoadTable(db, tblName, e);
     } catch (NoSuchObjectException e) {
diff --git a/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java b/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java
index 78895d2b4..805f58093 100644
--- a/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java
+++ b/fe/src/main/java/org/apache/impala/catalog/events/MetastoreEvents.java
@@ -2456,6 +2456,9 @@ public class MetastoreEvents {
 
     // if isRefresh_ is set to true then it is refresh query, else it is invalidate query
     private boolean isRefresh_;
+
+    private org.apache.impala.catalog.Table tbl_;
+
     /**
      * Prevent instantiation from outside should use MetastoreEventFactory instead
      */
@@ -2471,6 +2474,7 @@ public class MetastoreEvents {
             updatedFields.get("table"));
         reloadPartition_ = (Partition)updatedFields.get("partition");
         isRefresh_ = (boolean)updatedFields.get("isRefresh");
+        tbl_ = catalog_.getTable(dbName_, tblName_);
       } catch (Exception e) {
         throw new MetastoreNotificationException(debugString("Unable to "
                 + "parse reload message"), e);
@@ -2494,7 +2498,7 @@ public class MetastoreEvents {
 
     @Override
     public void process() throws MetastoreNotificationException {
-      if (isSelfEvent()) {
+      if (isSelfEvent() || isOlderEvent()) {
         metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
             .inc(getNumberOfEvents());
         infoLog("Incremented events skipped counter to {}",
@@ -2514,6 +2518,19 @@ public class MetastoreEvents {
       }
     }
 
+    private boolean isOlderEvent() {
+      if (tbl_ instanceof IncompleteTable) {
+        return false;
+      }
+      // Always check the lastRefreshEventId on the table first for table level refresh
+      if (tbl_.getLastRefreshEventId() > getEventId() || (reloadPartition_ != null &&
+          catalog_.isPartitionLoadedAfterEvent(dbName_, tblName_, reloadPartition_,
+              getEventId()))) {
+        return true;
+      }
+      return false;
+    }
+
     /**
      * Process partition reload
      */
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index bc8f1af1a..3830ffa7e 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -6435,8 +6435,8 @@ public class CatalogOpExecutor {
       }
 
       if (BackendConfig.INSTANCE.enableReloadEvents()) {
-        // fire event for refresh event
-        fireReloadEventHelper(req, updatedThriftTable, tblName, tbl);
+        // fire event for refresh event and update the last refresh event id
+        fireReloadEventAndUpdateRefreshEventId(req, updatedThriftTable, tblName, tbl);
       }
 
       // Return the TCatalogObject in the result to indicate this request can be
@@ -6480,12 +6480,13 @@ public class CatalogOpExecutor {
   /**
    * Helper class for refresh event.
    * This class invokes metastore shim's fireReloadEvent to fire event to HMS
+   * and update the last refresh event id in the cache
    * @param req - request object for TResetMetadataRequest.
    * @param updatedThriftTable - updated thrift table after refresh query
    * @param tblName
    * @param tbl
    */
-  private void fireReloadEventHelper(TResetMetadataRequest req,
+  private void fireReloadEventAndUpdateRefreshEventId(TResetMetadataRequest req,
       TCatalogObject updatedThriftTable, TableName tblName, Table tbl) {
     List<String> partVals = null;
     if (req.isSetPartition_spec()) {
@@ -6500,19 +6501,30 @@ public class CatalogOpExecutor {
           catalog_.getCatalogServiceId());
       tableParams.put(MetastoreEventPropertyKey.CATALOG_VERSION.getKey(),
           String.valueOf(newCatalogVersion));
-      MetastoreShim.fireReloadEventHelper(catalog_.getMetaStoreClient(),
-          req.isIs_refresh(), partVals, tblName.getDb(), tblName.getTbl(),
-          tableParams);
+      List<Long> eventIds = MetastoreShim.fireReloadEventHelper(
+          catalog_.getMetaStoreClient(), req.isIs_refresh(), partVals, tblName.getDb(),
+          tblName.getTbl(), tableParams);
       if (req.isIs_refresh()) {
         if (catalog_.tryLock(tbl, true, 600000)) {
           catalog_.addVersionsForInflightEvents(false, tbl, newCatalogVersion);
+          if (!eventIds.isEmpty()) {
+            if (req.isSetPartition_spec()) {
+              HdfsPartition partition = ((HdfsTable) tbl)
+                  .getPartitionFromThriftPartitionSpec(req.getPartition_spec());
+              HdfsPartition.Builder partBuilder = new HdfsPartition.Builder(partition);
+              partBuilder.setLastRefreshEventId(eventIds.get(0));
+              ((HdfsTable) tbl).updatePartition(partBuilder);
+            } else {
+              tbl.setLastRefreshEventId(eventIds.get(0));
+            }
+          }
         } else {
           LOG.warn(String.format("Couldn't obtain a version lock for the table: %s. " +
               "Self events may go undetected in that case",
               tbl.getName()));
         }
       }
-    } catch (TException e) {
+    } catch (TException | CatalogException e) {
       LOG.error(String.format(HMS_RPC_ERROR_FORMAT_STR,
           "fireReloadEvent") + e.getMessage());
     } finally {
diff --git a/tests/custom_cluster/test_events_custom_configs.py b/tests/custom_cluster/test_events_custom_configs.py
index 63385a325..95fb83e6c 100644
--- a/tests/custom_cluster/test_events_custom_configs.py
+++ b/tests/custom_cluster/test_events_custom_configs.py
@@ -18,6 +18,9 @@ from __future__ import print_function
 import logging
 import pytest
 
+
+from hive_metastore.ttypes import FireEventRequest
+from hive_metastore.ttypes import FireEventRequestData
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfFS
@@ -227,8 +230,10 @@ class TestEventProcessingCustomConfigs(CustomClusterTestSuite):
     self.__run_self_events_test(unique_database, True)
     self.__run_self_events_test(unique_database, False)
 
-  @CustomClusterTestSuite.with_args(catalogd_args="--hms_event_polling_interval_s=1"
-                                                  " --enable_reload_events=true")
+  @CustomClusterTestSuite.with_args(
+      catalogd_args="--hms_event_polling_interval_s=5"
+                    " --enable_reload_events=true"
+                    " --enable_sync_to_latest_event_on_ddls=true")
   def test_refresh_invalidate_events(self, unique_database):
     """Test is to verify Impala-11808, refresh/invalidate commands should generate a
     Reload event in HMS and CatalogD's event processor should process this event.
@@ -265,6 +270,31 @@ class TestEventProcessingCustomConfigs(CustomClusterTestSuite):
     check_self_events("refresh {}.{} partition(year=2022)"
         .format(unique_database, test_reload_table))
     check_self_events("refresh {}.{}".format(unique_database, test_reload_table))
+    EventProcessorUtils.wait_for_event_processing(self)
+
+    # Test to verify if older events are being skipped in event processor
+    data = FireEventRequestData()
+    data.refreshEvent = True
+    req = FireEventRequest(True, data)
+    req.dbName = unique_database
+    req.tableName = test_reload_table
+    # table level reload events
+    tbl_events_skipped_before = EventProcessorUtils.get_num_skipped_events()
+    for i in range(10):
+      self.hive_client.fire_listener_event(req)
+    EventProcessorUtils.wait_for_event_processing(self)
+    tbl_events_skipped_after = EventProcessorUtils.get_num_skipped_events()
+    assert tbl_events_skipped_after > tbl_events_skipped_before
+    # partition level reload events
+    EventProcessorUtils.wait_for_event_processing(self)
+    part_events_skipped_before = EventProcessorUtils.get_num_skipped_events()
+    req.partitionVals = ["2022"]
+    for i in range(10):
+      self.hive_client.fire_listener_event(req)
+    EventProcessorUtils.wait_for_event_processing(self)
+    part_events_skipped_after = EventProcessorUtils.get_num_skipped_events()
+    assert part_events_skipped_after > part_events_skipped_before
+
 
   @CustomClusterTestSuite.with_args(catalogd_args="--hms_event_polling_interval_s=1")
   def test_commit_compaction_events(self, unique_database):


[impala] 01/07: IMPALA-11983: Update stale comments in sorter.h referencing Init()

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 7a3b8e365fe65651c67a7b8892ad3513f317cd2e
Author: Daniel Becker <da...@cloudera.com>
AuthorDate: Tue Mar 7 16:05:07 2023 +0100

    IMPALA-11983: Update stale comments in sorter.h referencing Init()
    
    IMPALA-5446 replaced Sorter::Init() with Sorter::Prepare() and
    Sorter::Open(), but comments in sorter.h still refer to Init().
    
    This change updates these comments so they now refer to Prepare()
    instead.
    
    Change-Id: Ic3de8062d25cc226d5fea0cfb13a332e2e7f4aa4
    Reviewed-on: http://gerrit.cloudera.org:8080/19599
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Daniel Becker <da...@cloudera.com>
    Tested-by: Daniel Becker <da...@cloudera.com>
---
 be/src/runtime/sorter.h | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/be/src/runtime/sorter.h b/be/src/runtime/sorter.h
index 6b9785959..c83bf81cc 100644
--- a/be/src/runtime/sorter.h
+++ b/be/src/runtime/sorter.h
@@ -79,7 +79,7 @@ class RowBatch;
 /// input run, and one batch is created to hold deep copied rows (i.e. ptrs + data) from
 /// the output of the merge.
 //
-/// Note that Init() must be called right after the constructor.
+/// Note that Prepare() must be called right after the constructor.
 //
 /// During a merge, one row batch is created for each input run, and one batch is created
 /// for the output of the merge (if is not the final merge). It is assumed that the memory
@@ -151,7 +151,7 @@ class Sorter {
   Status GetNext(RowBatch* batch, bool* eos) WARN_UNUSED_RESULT;
 
   /// Resets all internal state like ExecNode::Reset().
-  /// Init() must have been called, AddBatch()/GetNext()/InputDone()
+  /// Prepare() must have been called, AddBatch()/GetNext()/InputDone()
   /// may or may not have been called.
   void Reset();
 


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

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

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

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

diff --git a/be/src/scheduling/scheduler.cc b/be/src/scheduling/scheduler.cc
index 947f234f2..09204ff4a 100644
--- a/be/src/scheduling/scheduler.cc
+++ b/be/src/scheduling/scheduler.cc
@@ -264,6 +264,25 @@ void Scheduler::ComputeFragmentExecParams(
   }
 }
 
+void Scheduler::CheckEffectiveInstanceCount(
+    const FragmentScheduleState* fragment_state, const ScheduleState* state) {
+  int effective_instance_count = fragment_state->fragment.effective_instance_count;
+  if (effective_instance_count > 0) {
+    if (ContainsUnionNode(fragment_state->fragment.plan)
+        || ContainsScanNode(fragment_state->fragment.plan)
+        || IsExceedMaxFsWriters(fragment_state, fragment_state, state)) {
+      // TODO: Fragment with UnionNode or ScanNode or one where IsExceedMaxFsWriters
+      // equals true is not checked for now since it require further modification of the
+      // scan-range based scheduling in CreateCollocatedAndScanInstances().
+      return;
+    }
+
+    DCHECK_GE(effective_instance_count, fragment_state->instance_states.size())
+        << fragment_state->fragment.display_name
+        << " scheduled higher than the effective count.";
+  }
+}
+
 void Scheduler::ComputeFragmentExecParams(const ExecutorConfig& executor_config,
     const TPlanExecInfo& plan_exec_info, FragmentScheduleState* fragment_state,
     ScheduleState* state) {
@@ -338,8 +357,7 @@ void Scheduler::ComputeFragmentExecParams(const ExecutorConfig& executor_config,
         instance_state.AddScanRanges(entry.first, entry.second);
       }
     }
-  } else if (ContainsNode(fragment.plan, TPlanNodeType::UNION_NODE)
-      || ContainsScanNode(fragment.plan)) {
+  } else if (ContainsUnionNode(fragment.plan) || ContainsScanNode(fragment.plan)) {
     VLOG(3) << "Computing exec params for scan and/or union fragment.";
     // case 2: leaf fragment (i.e. no input fragments) with a single scan node.
     // case 3: union fragment, which may have scan nodes and may have input fragments.
@@ -348,9 +366,11 @@ void Scheduler::ComputeFragmentExecParams(const ExecutorConfig& executor_config,
     VLOG(3) << "Computing exec params for interior fragment.";
     // case 4: interior (non-leaf) fragment without a scan or union.
     // We assign the same hosts as those of our leftmost input fragment (so that a
-    // merge aggregation fragment runs on the hosts that provide the input data).
+    // merge aggregation fragment runs on the hosts that provide the input data) OR
+    // follow the effective_instance_count specified by planner.
     CreateInputCollocatedInstances(fragment_state, state);
   }
+  CheckEffectiveInstanceCount(fragment_state, state);
 }
 
 // Maybe the easiest way to understand the objective of this algorithm is as a
@@ -381,7 +401,7 @@ void Scheduler::ComputeFragmentExecParams(const ExecutorConfig& executor_config,
 void Scheduler::CreateCollocatedAndScanInstances(const ExecutorConfig& executor_config,
     FragmentScheduleState* fragment_state, ScheduleState* state) {
   const TPlanFragment& fragment = fragment_state->fragment;
-  bool has_union = ContainsNode(fragment.plan, TPlanNodeType::UNION_NODE);
+  bool has_union = ContainsUnionNode(fragment.plan);
   DCHECK(has_union || ContainsScanNode(fragment.plan));
   // Build a map of hosts to the num instances this fragment should have, before we take
   // into account scan ranges. If this fragment has input fragments, we always run with
@@ -499,10 +519,7 @@ void Scheduler::CreateCollocatedAndScanInstances(const ExecutorConfig& executor_
       }
     }
   }
-  if (fragment.output_sink.__isset.table_sink
-      && fragment.output_sink.table_sink.__isset.hdfs_table_sink
-      && state->query_options().max_fs_writers > 0
-      && fragment_state->instance_states.size() > state->query_options().max_fs_writers) {
+  if (IsExceedMaxFsWriters(fragment_state, fragment_state, state)) {
     LOG(WARNING) << "Extra table sink instances scheduled, probably due to mismatch of "
                     "cluster state during planning vs scheduling. Expected: "
                  << state->query_options().max_fs_writers
@@ -536,11 +553,14 @@ void Scheduler::CreateInputCollocatedInstances(
       *state->GetFragmentScheduleState(fragment_state->exchange_input_fragments[0]);
   int per_fragment_instance_idx = 0;
 
-  if (fragment.output_sink.__isset.table_sink
-      && fragment.output_sink.table_sink.__isset.hdfs_table_sink
-      && state->query_options().max_fs_writers > 0
-      && input_fragment_state.instance_states.size()
-          > state->query_options().max_fs_writers) {
+  int max_instances = input_fragment_state.instance_states.size();
+  if (IsExceedMaxFsWriters(fragment_state, &input_fragment_state, state)) {
+    max_instances = state->query_options().max_fs_writers;
+  } else if (fragment.effective_instance_count > 0) {
+    max_instances = fragment.effective_instance_count;
+  }
+
+  if (max_instances != input_fragment_state.instance_states.size()) {
     std::unordered_set<std::pair<NetworkAddressPB, NetworkAddressPB>> all_hosts;
     for (const FInstanceScheduleState& input_instance_state :
         input_fragment_state.instance_states) {
@@ -550,7 +570,6 @@ void Scheduler::CreateInputCollocatedInstances(
     // across hosts and ensuring that instances on the same host get consecutive instance
     // indexes.
     int num_hosts = all_hosts.size();
-    int max_instances = state->query_options().max_fs_writers;
     int instances_per_host = max_instances / num_hosts;
     int remainder = max_instances % num_hosts;
     auto host_itr = all_hosts.begin();
@@ -787,6 +806,10 @@ bool Scheduler::ContainsScanNode(const TPlan& plan) {
   return ContainsNode(plan, SCAN_NODE_TYPES);
 }
 
+bool Scheduler::ContainsUnionNode(const TPlan& plan) {
+  return ContainsNode(plan, TPlanNodeType::UNION_NODE);
+}
+
 std::vector<TPlanNodeId> Scheduler::FindNodes(
     const TPlan& plan, const vector<TPlanNodeType::type>& types) {
   vector<TPlanNodeId> results;
diff --git a/be/src/scheduling/scheduler.h b/be/src/scheduling/scheduler.h
index f75bace41..084d9cb34 100644
--- a/be/src/scheduling/scheduler.h
+++ b/be/src/scheduling/scheduler.h
@@ -420,13 +420,17 @@ class Scheduler {
       std::vector<NetworkAddressPB>* scan_hosts);
 
   /// Return true if 'plan' contains a node of the given type.
-  bool ContainsNode(const TPlan& plan, TPlanNodeType::type type);
+  static bool ContainsNode(const TPlan& plan, TPlanNodeType::type type);
 
   /// Return true if 'plan' contains a node of one of the given types.
-  bool ContainsNode(const TPlan& plan, const std::vector<TPlanNodeType::type>& types);
+  static bool ContainsNode(
+      const TPlan& plan, const std::vector<TPlanNodeType::type>& types);
 
   /// Return true if 'plan' contains a scan node.
-  bool ContainsScanNode(const TPlan& plan);
+  static bool ContainsScanNode(const TPlan& plan);
+
+  /// Return true if 'plan' contains a union node.
+  static bool ContainsUnionNode(const TPlan& plan);
 
   /// Return all ids of nodes in 'plan' of any of the given types.
   std::vector<TPlanNodeId> FindNodes(
@@ -435,6 +439,24 @@ class Scheduler {
   /// Return all ids of all scan nodes in 'plan'.
   std::vector<TPlanNodeId> FindScanNodes(const TPlan& plan);
 
+  /// If TPlanFragment.effective_instance_count is positive, verify that resulting
+  /// instance_states size match with effective_instance_count. Fragment with UnionNode or
+  /// ScanNode or one where IsExceedMaxFsWriters equals true is not checked.
+  static void CheckEffectiveInstanceCount(
+      const FragmentScheduleState* fragment_state, const ScheduleState* state);
+
+  /// Check if sink_fragment_state has hdfs_table_sink AND ref_fragment_state scheduled
+  /// to exceed max_fs_writers query option.
+  static inline bool IsExceedMaxFsWriters(
+      const FragmentScheduleState* sink_fragment_state,
+      const FragmentScheduleState* ref_fragment_state, const ScheduleState* state) {
+    return (sink_fragment_state->fragment.output_sink.__isset.table_sink
+        && sink_fragment_state->fragment.output_sink.table_sink.__isset.hdfs_table_sink
+        && state->query_options().max_fs_writers > 0
+        && ref_fragment_state->instance_states.size()
+            > state->query_options().max_fs_writers);
+  }
+
   friend class impala::test::SchedulerWrapper;
   FRIEND_TEST(SimpleAssignmentTest, ComputeAssignmentDeterministicNonCached);
   FRIEND_TEST(SimpleAssignmentTest, ComputeAssignmentRandomNonCached);
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index e9c72c4ed..3e98dc619 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -1060,6 +1060,22 @@ Status impala::SetQueryOption(const string& key, const string& value,
         query_options->__set_stringify_map_keys(IsTrue(value));
         break;
       }
+      case TImpalaQueryOptions::COMPUTE_PROCESSING_COST: {
+        query_options->__set_compute_processing_cost(IsTrue(value));
+        break;
+      }
+      case TImpalaQueryOptions::PROCESSING_COST_MIN_THREADS: {
+        StringParser::ParseResult result;
+        const int32_t min_num =
+            StringParser::StringToInt<int32_t>(value.c_str(), value.length(), &result);
+        if (result != StringParser::PARSE_SUCCESS || min_num < 1 || min_num > 128) {
+          return Status(Substitute("$0 is not valid for processing_cost_min_threads. "
+                                   "Valid values are in [1, 128].",
+              value));
+        }
+        query_options->__set_processing_cost_min_threads(min_num);
+        break;
+      }
       default:
         if (IsRemovedQueryOption(key)) {
           LOG(WARNING) << "Ignoring attempt to set removed query option '" << key << "'";
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index 402582bf8..741888f84 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -50,7 +50,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
 // time we add or remove a query option to/from the enum TImpalaQueryOptions.
 #define QUERY_OPTS_TABLE                                                                 \
   DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),                                 \
-      TImpalaQueryOptions::ENABLE_TRIVIAL_QUERY_FOR_ADMISSION + 1);                      \
+      TImpalaQueryOptions::PROCESSING_COST_MIN_THREADS + 1);                             \
   REMOVED_QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED) \
   QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR, TQueryOptionLevel::REGULAR)               \
   REMOVED_QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)             \
@@ -283,7 +283,11 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
   QUERY_OPT_FN(codegen_cache_mode, CODEGEN_CACHE_MODE, TQueryOptionLevel::DEVELOPMENT)   \
   QUERY_OPT_FN(stringify_map_keys, STRINGIFY_MAP_KEYS, TQueryOptionLevel::ADVANCED)      \
   QUERY_OPT_FN(enable_trivial_query_for_admission, ENABLE_TRIVIAL_QUERY_FOR_ADMISSION,   \
-      TQueryOptionLevel::REGULAR);
+      TQueryOptionLevel::REGULAR)                                                        \
+  QUERY_OPT_FN(                                                                          \
+      compute_processing_cost, COMPUTE_PROCESSING_COST, TQueryOptionLevel::ADVANCED)     \
+  QUERY_OPT_FN(processing_cost_min_threads, PROCESSING_COST_MIN_THREADS,                 \
+      TQueryOptionLevel::ADVANCED);
 
 /// Enforce practical limits on some query options to avoid undesired query state.
 static const int64_t SPILLABLE_BUFFER_LIMIT = 1LL << 40; // 1 TB
diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc
index 1e8b9683f..d39c5c73b 100644
--- a/be/src/util/backend-gflag-util.cc
+++ b/be/src/util/backend-gflag-util.cc
@@ -197,8 +197,57 @@ DEFINE_string(ignored_dir_prefix_list, ".,_tmp.,_spark_metadata",
     "Comma separated list to specify the prefix for tmp/staging dirs that catalogd should"
     " skip in loading file metadata.");
 
+DEFINE_double_hidden(query_cpu_count_divisor, 1.0,
+    "(Advance) Divide the CPU requirement of a query to fit the total available CPU in "
+    "the executor group. For example, setting value 2 will fit the query with CPU "
+    "requirement 2X to an executor group with total available CPU X. Note that setting "
+    "with a fractional value less than 1 effectively multiplies the query CPU "
+    "requirement. A valid value is > 0.0. The default value is 1.");
+
+// TODO: Tune the individual expression cost from IMPALA-2805.
+DEFINE_bool_hidden(processing_cost_use_equal_expr_weight, true,
+    "(Advance) If true, all expression evaluations are weighted equally to 1 during the "
+    "plan node's processing cost calculation. If false, expression cost from IMPALA-2805 "
+    "will be used. Default to false.");
+
+// TODO: Benchmark and tune this config with an optimal value.
+DEFINE_int64_hidden(min_processing_per_thread, 10000000,
+    "(Advance) Minimum processing load (in processing cost unit) that a fragment "
+    "instance need to work on before planner consider increasing instance count. Used to "
+    "adjust fragment instance count based on estimated workload rather than the MT_DOP "
+    "setting. Setting this to high number will reduce parallelism of a fragment (more "
+    "workload per fragment), while setting to low number will increase parallelism (less "
+    "workload per fragment). Actual parallelism might still be constrained by the total "
+    "number of cores in selected executor group, MT_DOP, or PROCESSING_COST_MIN_THREAD "
+    "query option. Must be a positive integer. Default to 10M.");
+
+using strings::Substitute;
+
 namespace impala {
 
+// Flag validation
+// ------------------------------------------------------------
+static bool ValidateCpuCountDivisor(const char* flagname, double value) {
+  if (0.0 < value) {
+    return true;
+  }
+  LOG(ERROR) << Substitute(
+      "$0 must be greater than 0.0, value $1 is invalid", flagname, value);
+  return false;
+}
+
+static bool ValidateMinProcessingPerThread(const char* flagname, int64_t value) {
+  if (0 < value) {
+    return true;
+  }
+  LOG(ERROR) << Substitute(
+      "$0 must be a positive integer, value $1 is invalid", flagname, value);
+  return false;
+}
+
+DEFINE_validator(query_cpu_count_divisor, &ValidateCpuCountDivisor);
+DEFINE_validator(min_processing_per_thread, &ValidateMinProcessingPerThread);
+
 Status GetConfigFromCommand(const string& flag_cmd, string& result) {
   result.clear();
   if (flag_cmd.empty()) return Status::OK();
@@ -344,6 +393,10 @@ Status PopulateThriftBackendGflags(TBackendGflags& cfg) {
     DCHECK_EQ(FLAGS_geospatial_library, to_string(TGeospatialLibrary::HIVE_ESRI));
     cfg.__set_geospatial_library(TGeospatialLibrary::HIVE_ESRI);
   }
+  cfg.__set_query_cpu_count_divisor(FLAGS_query_cpu_count_divisor);
+  cfg.__set_processing_cost_use_equal_expr_weight(
+      FLAGS_processing_cost_use_equal_expr_weight);
+  cfg.__set_min_processing_per_thread(FLAGS_min_processing_per_thread);
   return Status::OK();
 }
 
diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift
index 809c4f88a..dec1f44ad 100644
--- a/common/thrift/BackendGflags.thrift
+++ b/common/thrift/BackendGflags.thrift
@@ -240,4 +240,10 @@ struct TBackendGflags {
   104: required bool enable_reload_events
 
   105: required TGeospatialLibrary geospatial_library
+
+  106: required double query_cpu_count_divisor
+
+  107: required bool processing_cost_use_equal_expr_weight
+
+  108: required i64 min_processing_per_thread
 }
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index add37998b..208bb9926 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -738,7 +738,7 @@ enum TImpalaQueryOptions {
   ORC_SCHEMA_RESOLUTION = 146;
 
   // Expands complex types in star queries
-  EXPAND_COMPLEX_TYPES = 147
+  EXPAND_COMPLEX_TYPES = 147;
 
   // Specify the database name which stores global udf
   FALLBACK_DB_FOR_FUNCTIONS = 148;
@@ -760,6 +760,14 @@ enum TImpalaQueryOptions {
 
   // Enable immediate admission for trivial queries.
   ENABLE_TRIVIAL_QUERY_FOR_ADMISSION = 152
+
+  // Control whether to consider CPU processing cost during query planning.
+  COMPUTE_PROCESSING_COST = 153;
+
+  // Minimum number of threads of a query fragment per host in processing
+  // cost algorithm. It is recommend to not set it with value more than number of
+  // physical cores in executor node. Valid values are in [1, 128]. Default to 1.
+  PROCESSING_COST_MIN_THREADS = 154;
 }
 
 // The summary of a DML statement.
diff --git a/common/thrift/Planner.thrift b/common/thrift/Planner.thrift
index ba206cb39..5eb11c452 100644
--- a/common/thrift/Planner.thrift
+++ b/common/thrift/Planner.thrift
@@ -83,6 +83,13 @@ struct TPlanFragment {
   // Maximum number of required threads that will be executing concurrently for this plan
   // fragment, i.e. the number of threads that this query needs to execute successfully.
   10: optional i64 thread_reservation
+
+  // The effective number of parallelism for this fragment that dictated by the frontend
+  // planner. If the frontend planner set this to a positive number, the backend scheduler
+  // must make sure that it schedules no more than this many instance fragments. Currently
+  // not enforced on fragments having scan nodes or union nodes or when fragment need to
+  // exceed max_fs_writers query option (see IMPALA-8125).
+  14: optional i32 effective_instance_count
 }
 
 // location information for a single scan range
diff --git a/common/thrift/Query.thrift b/common/thrift/Query.thrift
index 49d34b40d..b190f2f11 100644
--- a/common/thrift/Query.thrift
+++ b/common/thrift/Query.thrift
@@ -618,6 +618,12 @@ struct TQueryOptions {
 
   // See comment in ImpalaService.thrift
   153: optional bool enable_trivial_query_for_admission = true;
+
+  // See comment in ImpalaService.thrift
+  154: optional bool compute_processing_cost = false;
+
+  // See comment in ImpalaService.thrift
+  155: optional i32 processing_cost_min_threads = 1;
 }
 
 // Impala currently has three types of sessions: Beeswax, HiveServer2 and external
@@ -898,5 +904,10 @@ struct TQueryExecRequest {
 
   // Indicate whether the request is a trivial query. Used by admission control.
   13: optional bool is_trivial_query
+
+  // CPU core count required to run the query. Used by admission control to decide which
+  // executor group to run the query. Non-positive value means no specific CPU core count
+  // is required.
+  14: optional i32 cores_required;
 }
 
diff --git a/fe/src/main/java/org/apache/impala/planner/CoreCount.java b/fe/src/main/java/org/apache/impala/planner/CoreCount.java
new file mode 100644
index 000000000..e8fc87101
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/CoreCount.java
@@ -0,0 +1,110 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.planner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.impala.common.Id;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * A container class that represent CPU cores, computed from the CPU cost, of certain
+ * subtree of a query or the query itself.
+ */
+public class CoreCount {
+  // List of Id (either PlanFragmentId or PlanNodeId) that describe the origin of counts_
+  // element.
+  private final ImmutableList<Id> ids_;
+
+  // List of CPU core count contributing to this CoreCount.
+  private final ImmutableList<Integer> counts_;
+
+  // Sum of all elements in count_.
+  // Cached after the first call of total().
+  private int total_ = -1;
+
+  public CoreCount(Id id, int count) {
+    Preconditions.checkArgument(count >= 0, "Core count must be a non-negative number");
+    ids_ = ImmutableList.of(id);
+    counts_ = ImmutableList.of(count);
+  }
+
+  private CoreCount(ImmutableList<Id> ids, ImmutableList<Integer> counts) {
+    Preconditions.checkArgument(
+        ids.size() == counts.size(), "ids and counts must have same size!");
+    ids_ = ids;
+    counts_ = counts;
+  }
+
+  public int total() {
+    if (total_ < 0) {
+      total_ = counts_.stream().mapToInt(v -> v).sum();
+    }
+    return total_;
+  }
+
+  @Override
+  public String toString() {
+    if (ids_.isEmpty()) {
+      return "<empty>";
+    } else {
+      StringBuilder sb = new StringBuilder();
+      sb.append("{total=");
+      sb.append(total());
+      sb.append(" trace=");
+      sb.append(IntStream.range(0, ids_.size())
+                    .mapToObj(i
+                        -> ((ids_.get(i) instanceof PlanNodeId) ? "N" : "") + ids_.get(i)
+                            + ":" + counts_.get(i))
+                    .collect(Collectors.joining("+")));
+      sb.append("}");
+      return sb.toString();
+    }
+  }
+
+  protected static CoreCount sum(List<CoreCount> cores) {
+    ImmutableList.Builder<Id> idBuilder = new ImmutableList.Builder<Id>();
+    ImmutableList.Builder<Integer> countBuilder = new ImmutableList.Builder<Integer>();
+    for (CoreCount coreRequirement : cores) {
+      idBuilder.addAll(coreRequirement.ids_);
+      countBuilder.addAll(coreRequirement.counts_);
+    }
+    return new CoreCount(idBuilder.build(), countBuilder.build());
+  }
+
+  protected static CoreCount sum(CoreCount core1, CoreCount core2) {
+    ImmutableList.Builder<Id> idBuilder = new ImmutableList.Builder<Id>();
+    ImmutableList.Builder<Integer> countBuilder = new ImmutableList.Builder<Integer>();
+
+    idBuilder.addAll(core1.ids_);
+    idBuilder.addAll(core2.ids_);
+    countBuilder.addAll(core1.counts_);
+    countBuilder.addAll(core2.counts_);
+
+    return new CoreCount(idBuilder.build(), countBuilder.build());
+  }
+
+  protected static CoreCount max(CoreCount core1, CoreCount core2) {
+    return (core1.total() < core2.total()) ? core2 : core1;
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/CostingSegment.java b/fe/src/main/java/org/apache/impala/planner/CostingSegment.java
new file mode 100644
index 000000000..a3c5ed775
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/CostingSegment.java
@@ -0,0 +1,235 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.planner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.apache.impala.common.Id;
+import org.apache.impala.common.Pair;
+import org.apache.impala.common.TreeNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * A grouping of adjacent PlanNode and DataSink of a fragment for CPU costing purpose.
+ * <p>
+ * Each fragment segment is a subtree of PlanNodes/DataSink in the fragment with a
+ * DataSink or blocking PlanNode as root. All other nodes in the segment are non-blocking.
+ * In most cases, every segment is a blocking segment, that is the root is either
+ * a blocking operator or blocking DataSink (ie., JoinBuildSink). A segment is not a
+ * blocking segment if its root is a non-blocking DataSink (ie., DataStreamSink).
+ * <p>
+ * PlanNodes or DataSink that belong to the same segment will have their ProcessingCost
+ * summed. Analyses done through this class might recurse around the CostingSegment tree,
+ * but should not traverse into different fragment.
+ */
+public class CostingSegment extends TreeNode<CostingSegment> {
+  private final static Logger LOG = LoggerFactory.getLogger(CostingSegment.class);
+
+  // List of PlanNode belong to this segment.
+  private List<PlanNode> nodes_ = Lists.newArrayList();
+
+  // The ProcessingCost of this fragment segment, which is the sum of the processing cost
+  // of all nodes in nodes_ and sink_ (if set).
+  private ProcessingCost cost_ = ProcessingCost.zero();
+
+  // DataSink associated with this segment.
+  // Must not be null for output segment.
+  private DataSink sink_ = null;
+
+  public CostingSegment(DataSink sink) { setSink(sink); }
+
+  public CostingSegment(PlanNode node) { appendNode(node); }
+
+  private CostingSegment() {}
+
+  public ProcessingCost getProcessingCost() { return cost_; }
+  public boolean isOutputSegment() { return sink_ != null; }
+
+  private Id getRootId() {
+    if (isOutputSegment()) {
+      return sink_.getFragment().getId();
+    } else {
+      Preconditions.checkState(!nodes_.isEmpty());
+      return nodes_.get(nodes_.size() - 1).getId();
+    }
+  }
+
+  private void appendCost(ProcessingCost additionalCost) {
+    Preconditions.checkArgument(additionalCost.isValid());
+    ProcessingCost newTotalCost = ProcessingCost.sumCost(additionalCost, cost_);
+    newTotalCost.setNumRowToConsume(cost_.getNumRowToConsume());
+    newTotalCost.setNumRowToProduce(additionalCost.getNumRowToConsume());
+    cost_ = newTotalCost;
+  }
+
+  protected void setSink(DataSink sink) {
+    appendCost(sink.getProcessingCost());
+    sink_ = sink;
+  }
+
+  protected void appendNode(PlanNode node) {
+    appendCost(node.getProcessingCost());
+    nodes_.add(node);
+  }
+
+  /**
+   * Gather {@link CoreCount} rooted from this segment and populate
+   * subtreeCoreBuilder with {@link CoreCount} of child-blocking-subtree.
+   * @param fragmentCoreState A map holding per-fragment core state.
+   * @param subtreeCoreBuilder An ImmutableList builder to populate.
+   * @return A {@link CoreCount} value of segment tree
+   * rooted at this segment.
+   */
+  protected CoreCount traverseBlockingAwareCores(
+      Map<PlanFragmentId, Pair<CoreCount, List<CoreCount>>> fragmentCoreState,
+      ImmutableList.Builder<CoreCount> subtreeCoreBuilder) {
+    CoreCount segmentCore = new CoreCount(getRootId(), cost_.getNumInstancesExpected());
+    // If not in input segment, gather cost of children first.
+    for (CostingSegment childSegment : getChildren()) {
+      CoreCount childSegmentCores =
+          childSegment.traverseBlockingAwareCores(fragmentCoreState, subtreeCoreBuilder);
+      if (childSegmentCores.total() > 0) {
+        segmentCore = CoreCount.max(segmentCore, childSegmentCores);
+      }
+    }
+
+    // Look up child fragment that is connected through this segment.
+    for (PlanNode node : nodes_) {
+      for (int i = 0; i < node.getChildCount(); i++) {
+        PlanFragment childFragment = node.getChild(i).getFragment();
+        if (childFragment == node.getFragment()) continue;
+
+        Pair<CoreCount, List<CoreCount>> childCores =
+            fragmentCoreState.get(childFragment.getId());
+        Preconditions.checkNotNull(childCores);
+
+        if (childFragment.hasBlockingNode()) {
+          CoreCount childCoreCount =
+              childFragment.maxCore(childCores.first, CoreCount.sum(childCores.second));
+          subtreeCoreBuilder.add(childCoreCount);
+        } else {
+          Preconditions.checkState(node instanceof ExchangeNode);
+          Preconditions.checkState(i == 0);
+          segmentCore = CoreCount.sum(segmentCore, childCores.first);
+          subtreeCoreBuilder.addAll(childCores.second);
+        }
+      }
+    }
+
+    return segmentCore;
+  }
+
+  /**
+   * Try to come up with lower parallelism for this segment by comparing the output
+   * ProcessingCost of child segment or input fragment.
+   * <p>
+   * This segment cost is the Consumer cost, while the ProcessingCost of child segment
+   * or input fragment is the Producer cost. It compares between per-row production cost
+   * of Producer versus the per-row consumption cost of Consumer. The expected parallelism
+   * of Consumer (this segment) then is adjusted to get closer to the produce-consume
+   * ratio when compared to the Producer.
+   *
+   * @param nodeStepCount The step count used to increase this fragment's parallelism.
+   *                      Usually equal to number of nodes or just 1.
+   * @param minParallelism The minimum parallelism of this segment.
+   * @param maxParallelism The maximum parallelism this segment is allowed to adjust to.
+   */
+  protected int tryAdjustParallelism(
+      int nodeStepCount, int minParallelism, int maxParallelism) {
+    // TODO: The ratio based adjustment can be further improved by considering the plan
+    //   nodes too. For example, one does not need to make DoP of the parent of a scale
+    //   aggregate fragment the same as the aggregate fragment, say the aggregate fragment
+    //   produces one row and uses 10 instances.
+    int newParallelism = minParallelism;
+    ProcessingCost producerCost = ProcessingCost.zero();
+
+    if (getChildCount() > 0) {
+      for (CostingSegment childSegment : getChildren()) {
+        newParallelism = Math.max(newParallelism,
+            childSegment.tryAdjustParallelism(
+                nodeStepCount, minParallelism, maxParallelism));
+      }
+      producerCost = mergeCostingSegment(getChildren()).getProcessingCost();
+    }
+
+    // If this segment has UnionNode, it may have ExchangeNode belonging to this segment.
+    List<ProcessingCost> childOutputCosts =
+        nodes_.stream()
+            .filter(Predicates.instanceOf(ExchangeNode.class))
+            .map(p -> p.getChild(0).getFragment().getLastCostingSegment())
+            .collect(Collectors.toList());
+
+    if (!childOutputCosts.isEmpty()) {
+      if (producerCost.getTotalCost() > 0) childOutputCosts.add(producerCost);
+      producerCost = ProcessingCost.fullMergeCosts(childOutputCosts);
+    }
+
+    ProcessingCost.tryAdjustConsumerParallelism(
+        nodeStepCount, minParallelism, maxParallelism, producerCost, cost_);
+    newParallelism = cost_.getNumInstancesExpected();
+    Preconditions.checkState(newParallelism >= minParallelism);
+    Preconditions.checkState(newParallelism <= maxParallelism);
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Adjust ProcessingCost on {}. minParallelism={} maxParallelism={} "
+              + "newParallelism={} consumerCost={} consumerInstCount={} producerCost={} "
+              + "producerInstCount={}",
+          getRootId(), minParallelism, maxParallelism, newParallelism,
+          cost_.getTotalCost(), cost_.getNumInstancesExpected(),
+          producerCost.getTotalCost(), producerCost.getNumInstancesExpected());
+    }
+
+    return newParallelism;
+  }
+
+  /**
+   * Merge given list of CostingSegment into a new combined CostingSegment.
+   * <p>
+   * The resulting CostingSegment will contain all nodes, children segments, and sum of
+   * ProcessingCost from all given CostingSegment.
+   *
+   * @param costingSegments List of CostingSegment to merge. Must not be empty and must
+   *                        not contain output segment (segment with DataSink set).
+   * @return A combined CostingSegment.
+   */
+  protected static CostingSegment mergeCostingSegment(
+      List<CostingSegment> costingSegments) {
+    Preconditions.checkNotNull(costingSegments);
+    Preconditions.checkArgument(!costingSegments.isEmpty());
+
+    if (costingSegments.size() == 1) return costingSegments.get(0);
+    CostingSegment mergedCost = new CostingSegment();
+    List<ProcessingCost> allCosts = Lists.newArrayList();
+    for (CostingSegment costingSegment : costingSegments) {
+      Preconditions.checkArgument(!costingSegment.isOutputSegment());
+      mergedCost.nodes_.addAll(costingSegment.nodes_);
+      mergedCost.addChildren(costingSegment.getChildren());
+      allCosts.add(costingSegment.getProcessingCost());
+    }
+    mergedCost.cost_ = ProcessingCost.fullMergeCosts(allCosts);
+    return mergedCost;
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
index 44232b53d..62f2a50ca 100644
--- a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
@@ -208,7 +208,10 @@ public class ExchangeNode extends PlanNode {
 
     if (isBroadcastExchange()) {
       processingCost_ = ProcessingCost.broadcastCost(processingCost_,
-          () -> getNumReceivers());
+          ()
+              -> fragment_.hasAdjustedInstanceCount() ?
+              fragment_.getAdjustedInstanceCount() :
+              getNumReceivers());
     }
   }
 
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
index 399e45829..8c6df6174 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
@@ -378,4 +378,10 @@ public class HdfsTableSink extends TableSink {
     }
     return num_instances;
   }
+
+  @Override
+  public void computeRowConsumptionAndProductionToCost() {
+    super.computeRowConsumptionAndProductionToCost();
+    fragment_.setFixedInstanceCount(fragment_.getNumInstances());
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
index feed83381..ab268c0c4 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
@@ -163,4 +163,12 @@ public class JoinBuildSink extends DataSink {
   public void collectExprs(List<Expr> exprs) {
     exprs.addAll(buildExprs_);
   }
+
+  @Override
+  public void computeRowConsumptionAndProductionToCost() {
+    super.computeRowConsumptionAndProductionToCost();
+    if (isShared()) {
+      fragment_.setFixedInstanceCount(getNumInstances());
+    }
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
index 1093014bf..c50877cbf 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -17,28 +17,39 @@
 
 package org.apache.impala.planner;
 
+import java.math.RoundingMode;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
+
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.InternalException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.common.PrintUtils;
 import org.apache.impala.common.TreeNode;
 import org.apache.impala.planner.JoinNode.DistributionMode;
 import org.apache.impala.planner.PlanNode.ExecPhaseResourceProfiles;
 import org.apache.impala.planner.RuntimeFilterGenerator.RuntimeFilter;
+import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPartitionType;
 import org.apache.impala.thrift.TPlanFragment;
 import org.apache.impala.thrift.TPlanFragmentTree;
 import org.apache.impala.thrift.TQueryOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.math.IntMath;
+import com.google.common.math.LongMath;
 
 /**
  * PlanFragments form a tree structure via their ExchangeNodes. A tree of fragments
@@ -77,6 +88,7 @@ import com.google.common.base.Predicates;
  * - toThrift()
  */
 public class PlanFragment extends TreeNode<PlanFragment> {
+  private final static Logger LOG = LoggerFactory.getLogger(PlanFragment.class);
   private final PlanFragmentId fragmentId_;
   private PlanId planId_;
   private CohortId cohortId_;
@@ -129,6 +141,32 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   // on a backend.
   private long consumedGlobalRuntimeFiltersMemReservationBytes_ = 0;
 
+  // The root of segment costs tree of this fragment.
+  // Individual element of the tree describe processing cost of subset of plan nodes
+  // that is divided by blocking PlanNode/DataSink boundary. Together, they describe total
+  // processing cost of this fragment. Set in computeCostingSegment().
+  private CostingSegment rootSegment_;
+
+  // Maximum allowed parallelism based on minimum processing load per fragment.
+  // Set in getCostBasedMaxParallelism().
+  private int costBasedMaxParallelism_ = -1;
+
+  // An adjusted number of instance based on ProcessingCost calculation.
+  // A positive value implies that the instance count has been adjusted, either through
+  // traverseEffectiveParallelism() or by fragment member (PlanNode or DataSink) calling
+  // setFixedInstanceCount(). Internally, this must be set through
+  // setAdjustedInstanceCount().
+  private int adjustedInstanceCount_ = -1;
+
+  // Mark if this fragment has a fixed instance count dictated by any of its PlanNode or
+  // DataSink member.
+  private boolean isFixedParallelism_ = false;
+
+  // The original instance count before ProcessingCost based adjustment.
+  // Set in setEffectiveNumInstance() and only set if instance count differ between
+  // the original plan vs the ProcessingCost based plan.
+  private int originalInstanceCount_ = -1;
+
   public long getProducedRuntimeFiltersMemReservationBytes() {
     return producedRuntimeFiltersMemReservationBytes_;
   }
@@ -187,6 +225,89 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     }
   }
 
+  /**
+   * Compute processing cost of PlanNodes and DataSink of this fragment, and aggregate
+   * them into {@link CostingSegment} rooted at {@link #rootSegment_}.
+   * <p>For example, given the following fragment plan:
+   * <pre>
+   * F03:PLAN FRAGMENT [HASH(i_class)] hosts=3 instances=3
+   * fragment-costs=[34550429, 2159270, 23752870, 1]
+   * 08:TOP-N [LIMIT=100]
+   * |  cost=900
+   * |
+   * 07:ANALYTIC
+   * |  cost=23751970
+   * |
+   * 06:SORT
+   * |  cost=2159270
+   * |
+   * 12:AGGREGATE [FINALIZE]
+   * |  cost=34548320
+   * |
+   * 11:EXCHANGE [HASH(i_class)]
+   *    cost=2109
+   * </pre>
+   * The post-order traversal of {@link #rootSegment_} tree show processing cost detail of
+   * {@code [(2109+34548320), 2159270, (23751970+900), 1]}.
+   * The DataSink with cost 1 is a separate segment since the last PlanNode (TOP-N) is a
+   * blocking node.
+   *
+   * @param queryOptions A query options for this query.
+   */
+  public void computeCostingSegment(TQueryOptions queryOptions) {
+    for (PlanNode node : collectPlanNodes()) {
+      node.computeProcessingCost(queryOptions);
+      node.computeRowConsumptionAndProductionToCost();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("ProcessingCost Node " + node.getProcessingCost().debugString());
+      }
+    }
+    sink_.computeProcessingCost(queryOptions);
+    sink_.computeRowConsumptionAndProductionToCost();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("ProcessingCost Sink " + sink_.getProcessingCost().debugString());
+    }
+
+    CostingSegment topSegment = collectCostingSegmentHelper(planRoot_);
+
+    if (isBlockingNode(planRoot_)) {
+      rootSegment_ = new CostingSegment(sink_);
+      rootSegment_.addChild(topSegment);
+    } else {
+      topSegment.setSink(sink_);
+      rootSegment_ = topSegment;
+    }
+  }
+
+  private CostingSegment collectCostingSegmentHelper(PlanNode root) {
+    Preconditions.checkNotNull(root);
+
+    List<CostingSegment> blockingChildSegments = Lists.newArrayList();
+    List<CostingSegment> nonBlockingChildSegments = Lists.newArrayList();
+    for (PlanNode child : root.getChildren()) {
+      if (child.getFragment() != this) continue;
+      CostingSegment childCostingSegment = collectCostingSegmentHelper(child);
+
+      if (isBlockingNode(child)) {
+        blockingChildSegments.add(childCostingSegment);
+      } else {
+        nonBlockingChildSegments.add(childCostingSegment);
+      }
+    }
+
+    CostingSegment thisSegment;
+    if (nonBlockingChildSegments.isEmpty()) {
+      // No child or all children are blocking nodes.
+      thisSegment = new CostingSegment(root);
+    } else {
+      thisSegment = CostingSegment.mergeCostingSegment(nonBlockingChildSegments);
+      thisSegment.appendNode(root);
+    }
+
+    if (!blockingChildSegments.isEmpty()) thisSegment.addChildren(blockingChildSegments);
+    return thisSegment;
+  }
+
   /**
    * Do any final work to set up the ExchangeNodes and DataStreamSinks for this fragment.
    * If this fragment has partitioned joins, ensures that the corresponding partition
@@ -462,6 +583,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     result.setBackend_min_mem_reservation_bytes(
         perBackendResourceProfile_.getMinMemReservationBytes());
     result.setThread_reservation(perInstanceResourceProfile_.getThreadReservation());
+    result.setEffective_instance_count(getAdjustedInstanceCount());
     return result;
   }
 
@@ -496,7 +618,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
       prefix = "  ";
       rootPrefix = "  ";
       detailPrefix = prefix + "|  ";
-      str.append(getFragmentHeaderString("", "", queryOptions.getMt_dop()));
+      str.append(getFragmentHeaderString("", "", queryOptions, detailLevel));
       if (sink_ != null && sink_ instanceof DataStreamSink) {
         str.append(
             sink_.getExplainString(rootPrefix, detailPrefix, queryOptions, detailLevel));
@@ -504,7 +626,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     } else if (detailLevel == TExplainLevel.EXTENDED) {
       // Print a fragment prefix displaying the # nodes and # instances
       str.append(
-          getFragmentHeaderString(rootPrefix, detailPrefix, queryOptions.getMt_dop()));
+          getFragmentHeaderString(rootPrefix, detailPrefix, queryOptions, detailLevel));
       rootPrefix = prefix;
     }
 
@@ -530,12 +652,17 @@ public class PlanFragment extends TreeNode<PlanFragment> {
    * Get a header string for a fragment in an explain plan.
    */
   public String getFragmentHeaderString(String firstLinePrefix, String detailPrefix,
-      int mt_dop) {
+      TQueryOptions queryOptions, TExplainLevel explainLevel) {
+    int mt_dop = queryOptions.getMt_dop();
     StringBuilder builder = new StringBuilder();
     builder.append(String.format("%s%s:PLAN FRAGMENT [%s]", firstLinePrefix,
         fragmentId_.toString(), dataPartition_.getExplainString()));
     builder.append(PrintUtils.printNumHosts(" ", getNumNodes()));
     builder.append(PrintUtils.printNumInstances(" ", getNumInstances()));
+    if (ProcessingCost.isComputeCost(queryOptions)
+        && originalInstanceCount_ != getNumInstances()) {
+      builder.append(" (adjusted from " + originalInstanceCount_ + ")");
+    }
     builder.append("\n");
     String perHostPrefix = mt_dop == 0 ?
         "Per-Host Resources: " : "Per-Host Shared Resources: ";
@@ -589,6 +716,25 @@ public class PlanFragment extends TreeNode<PlanFragment> {
       builder.append(perInstanceExplainString);
       builder.append("\n");
     }
+    if (ProcessingCost.isComputeCost(queryOptions) && rootSegment_ != null
+        && explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+      // Print processing cost.
+      builder.append(detailPrefix);
+      builder.append("max-parallelism=");
+      if (isFixedParallelism_
+          || ((sink_ instanceof JoinBuildSink) && !((JoinBuildSink) sink_).isShared())) {
+        builder.append(getAdjustedInstanceCount());
+      } else {
+        builder.append(getCostBasedMaxParallelism());
+      }
+      builder.append(" fragment-costs=");
+      builder.append(costingSegmentSummary());
+      builder.append("\n");
+      if (explainLevel.ordinal() >= TExplainLevel.VERBOSE.ordinal()) {
+        builder.append(explainProcessingCosts(detailPrefix, false));
+        builder.append("\n");
+      }
+    }
     return builder.toString();
   }
 
@@ -709,4 +855,292 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     // We add one to prevent having a hash seed of 0.
     return planRoot_.getId().asInt() + 1;
   }
+
+  /**
+   * Get maximum allowed parallelism based on minimum processing load per fragment.
+   * <p>This is controlled by {@code min_processing_per_thread} flag. Only valid after
+   * {@link #computeCostingSegment(TQueryOptions)} has been called.
+   *
+   * @return maximum allowed parallelism based on minimum processing load per fragment.
+   */
+  protected int getCostBasedMaxParallelism() {
+    if (costBasedMaxParallelism_ >= 0) return costBasedMaxParallelism_;
+
+    ProcessingCost maxCostingSegment = ProcessingCost.zero();
+    List<CostingSegment> allSegments = rootSegment_.getNodesPostOrder();
+    for (CostingSegment costingSegment : allSegments) {
+      maxCostingSegment =
+          ProcessingCost.maxCost(maxCostingSegment, costingSegment.getProcessingCost());
+    }
+
+    long maxParallelism = LongMath.divide(maxCostingSegment.getTotalCost(),
+        BackendConfig.INSTANCE.getMinProcessingPerThread(), RoundingMode.CEILING);
+    // Round up to the nearest multiple of numNodes.
+    // Little over-parallelize is better than under-parallelize.
+    int numNodes = getNumNodes();
+    maxParallelism =
+        LongMath.divide(maxParallelism, numNodes, RoundingMode.CEILING) * numNodes;
+
+    if (maxParallelism <= 0) {
+      costBasedMaxParallelism_ = 1;
+    } else if (maxParallelism <= Integer.MAX_VALUE) {
+      costBasedMaxParallelism_ = (int) maxParallelism;
+    } else {
+      // Floor Integer.MAX_VALUE to the nearest multiple of numNodes.
+      costBasedMaxParallelism_ = Integer.MAX_VALUE - (Integer.MAX_VALUE % numNodes);
+    }
+    return costBasedMaxParallelism_;
+  }
+
+  protected boolean hasBlockingNode() {
+    if (sink_ instanceof JoinBuildSink) return true;
+    for (PlanNode p : collectPlanNodes()) {
+      if (isBlockingNode(p)) return true;
+    }
+    return false;
+  }
+
+  protected boolean hasAdjustedInstanceCount() { return adjustedInstanceCount_ > 0; }
+
+  protected void setFixedInstanceCount(int count) {
+    isFixedParallelism_ = true;
+    setAdjustedInstanceCount(count);
+  }
+
+  private void setAdjustedInstanceCount(int count) {
+    Preconditions.checkState(count > 0,
+        getId() + " adjusted instance count (" + count + ") is not positive number.");
+    boolean isFirstAdjustment = adjustedInstanceCount_ <= 0;
+    adjustedInstanceCount_ = count;
+    if (rootSegment_ != null) {
+      List<CostingSegment> costingSegments = rootSegment_.getNodesPostOrder();
+      for (CostingSegment costingSegment : costingSegments) {
+        // Reset for each segment cost since it might be overriden during
+        // tryLowerParallelism().
+        costingSegment.getProcessingCost().setNumInstanceExpected(
+            this::getAdjustedInstanceCount);
+      }
+    }
+
+    if (isFirstAdjustment) {
+      // Set num instance expected for ProcessingCost attached to PlanNodes and DataSink.
+      for (PlanNode node : collectPlanNodes()) {
+        node.getProcessingCost().setNumInstanceExpected(this::getAdjustedInstanceCount);
+      }
+      sink_.getProcessingCost().setNumInstanceExpected(this::getAdjustedInstanceCount);
+    }
+  }
+
+  protected int getAdjustedInstanceCount() { return adjustedInstanceCount_; }
+
+  protected ProcessingCost getLastCostingSegment() {
+    return rootSegment_.getProcessingCost();
+  }
+
+  private List<Long> costingSegmentSummary() {
+    return rootSegment_.getNodesPostOrder()
+        .stream()
+        .map(s -> ((CostingSegment) s).getProcessingCost().getTotalCost())
+        .collect(Collectors.toList());
+  }
+
+  private String explainProcessingCosts(String linePrefix, boolean fullExplain) {
+    return rootSegment_.getNodesPreOrder()
+        .stream()
+        .map(s
+            -> ((CostingSegment) s)
+                   .getProcessingCost()
+                   .getExplainString(linePrefix, fullExplain))
+        .collect(Collectors.joining("\n"));
+  }
+
+  private String debugProcessingCosts() { return explainProcessingCosts("", true); }
+
+  /**
+   * Validates that properties related to processing cost of this fragment are complete
+   * and valid.
+   */
+  private void validateProcessingCosts() {
+    Preconditions.checkState(hasAdjustedInstanceCount());
+    Preconditions.checkNotNull(rootSegment_);
+    List<CostingSegment> costingSegments = rootSegment_.getNodesPreOrder();
+    for (CostingSegment costingSegment : costingSegments) {
+      ProcessingCost cost = costingSegment.getProcessingCost();
+      Preconditions.checkState(cost.isValid());
+      Preconditions.checkState(
+          cost.getNumInstancesExpected() == getAdjustedInstanceCount());
+    }
+  }
+
+  /**
+   * Traverse down the query tree starting from this fragment and calculate the effective
+   * parallelism of each PlanFragments.
+   *
+   * @param minThreadPerNode Minimum thread per fragment per node based on
+   *                         {@code processing_cost_min_threads} flag.
+   * @param maxThreadPerNode Maximum thread per fragment per node based on
+   *                         TExecutorGroupSet.num_cores_per_executor flag.
+   * @param parentParallelism Number of instance of parent fragment.
+   */
+  protected void traverseEffectiveParallelism(
+      int minThreadPerNode, int maxThreadPerNode, int parentParallelism) {
+    Preconditions.checkNotNull(
+        rootSegment_, "ProcessingCost Fragment %s has not been computed!", getId());
+    int nodeStepCount = getNumInstances() % getNumNodes() == 0 ? getNumNodes() : 1;
+
+    // step 1: Set initial parallelism to the maximum possible.
+    //   Subsequent steps after this will not exceed maximum parallelism sets here.
+    boolean canTryLower =
+        adjustToMaxParallelism(maxThreadPerNode, parentParallelism, nodeStepCount);
+
+    if (canTryLower) {
+      // step 2: Try lower parallelism by comparing output ProcessingCost of the input
+      //   child fragment against this fragment's segment costs.
+      Preconditions.checkState(getChildCount() > 0);
+      Preconditions.checkState(getChild(0).getSink() instanceof DataStreamSink);
+
+      // Check if this fragment parallelism can be lowered.
+      int maxParallelism = getAdjustedInstanceCount();
+      int effectiveParallelism = rootSegment_.tryAdjustParallelism(
+          nodeStepCount, minThreadPerNode, maxParallelism);
+      setAdjustedInstanceCount(effectiveParallelism);
+      if (LOG.isTraceEnabled() && effectiveParallelism != maxParallelism) {
+        logCountAdjustmentTrace(maxParallelism, effectiveParallelism,
+            "Lower parallelism based on load and produce-consume rate ratio.");
+      }
+    }
+    validateProcessingCosts();
+
+    // step 3: Compute the parallelism of join build fragment.
+    //   Child parallelism may be enforced to follow this fragment's parallelism.
+    // TODO: This code assume that probe side of the join always have higher per-instance
+    //   cost than the build side. If this assumption is false and the child is a
+    //   non-shared join build fragment, then this fragment should increase its
+    //   parallelism to match the child fragment parallelism.
+    for (PlanFragment child : getChildren()) {
+      if (child.getSink() instanceof JoinBuildSink) {
+        child.traverseEffectiveParallelism(
+            minThreadPerNode, maxThreadPerNode, getAdjustedInstanceCount());
+      }
+    }
+  }
+
+  /**
+   * Adjust parallelism of this fragment to the maximum allowed.
+   *
+   * @param maxThreadPerNode Maximum thread per fragment per node based on
+   *                         TExecutorGroupSet.num_cores_per_executor flag.
+   * @param parentParallelism Parallelism of parent fragment.
+   * @param nodeStepCount The step count used to increase this fragment's parallelism.
+   *                      Usually equal to number of nodes or just 1.
+   * @return True if it is possible to lower this fragment's parallelism through
+   * ProcessingCost comparison. False if the parallelism should not be changed anymore.
+   */
+  private boolean adjustToMaxParallelism(
+      int maxThreadPerNode, int parentParallelism, int nodeStepCount) {
+    boolean canTryLower = true;
+    // Compute maximum allowed parallelism.
+    int maxParallelism = getNumInstances();
+    if (isFixedParallelism_) {
+      maxParallelism = getAdjustedInstanceCount();
+      canTryLower = false;
+    } else if ((sink_ instanceof JoinBuildSink) && !((JoinBuildSink) sink_).isShared()) {
+      // This is a non-shared (PARTITIONED) join build fragment.
+      // Parallelism of this fragment is equal to its parent parallelism.
+      Preconditions.checkState(parentParallelism > 0);
+      if (LOG.isTraceEnabled() && maxParallelism != parentParallelism) {
+        logCountAdjustmentTrace(maxParallelism, parentParallelism,
+            "Partitioned join build fragment follow parent's parallelism.");
+      }
+      maxParallelism = parentParallelism;
+      canTryLower = false; // no need to compute effective parallelism anymore.
+    } else {
+      // TODO: Fragment with UnionNode but without ScanNode should have its parallelism
+      //   bounded by the maximum parallelism between its exchanging child.
+      //   For now, it wont get here since fragment with UnionNode has fixed parallelism
+      //   (equal to MT_DOP, and previouslyAdjusted == true).
+      maxParallelism = IntMath.saturatedMultiply(maxThreadPerNode, getNumNodes());
+      int costBasedMaxParallelism = Math.max(nodeStepCount, getCostBasedMaxParallelism());
+      if (costBasedMaxParallelism < maxParallelism) {
+        maxParallelism = costBasedMaxParallelism;
+      }
+
+      if (LOG.isTraceEnabled() && maxParallelism != getNumInstances()) {
+        if (maxParallelism == maxThreadPerNode) {
+          logCountAdjustmentTrace(
+              getNumInstances(), maxParallelism, "Follow maxThreadPerNode.");
+        } else {
+          logCountAdjustmentTrace(
+              getNumInstances(), maxParallelism, "Follow minimum work per thread.");
+        }
+      }
+    }
+
+    // Initialize this fragment's parallelism to the maxParallelism.
+    setAdjustedInstanceCount(maxParallelism);
+    return canTryLower;
+  }
+
+  /**
+   * Compute {@link CoreCount} of this fragment and populate it into 'fragmentCoreState'.
+   * @param fragmentCoreState A map holding per-fragment core state.
+   * All successor of this fragment must already have its CoreCount registered into this
+   * map.
+   */
+  protected void computeBlockingAwareCores(
+      Map<PlanFragmentId, Pair<CoreCount, List<CoreCount>>> fragmentCoreState) {
+    Preconditions.checkNotNull(
+        rootSegment_, "ProcessingCost Fragment %s has not been computed!", getId());
+    ImmutableList.Builder<CoreCount> subtreeCoreBuilder =
+        new ImmutableList.Builder<CoreCount>();
+    CoreCount coreReq =
+        rootSegment_.traverseBlockingAwareCores(fragmentCoreState, subtreeCoreBuilder);
+    fragmentCoreState.put(getId(), Pair.create(coreReq, subtreeCoreBuilder.build()));
+  }
+
+  protected CoreCount maxCore(CoreCount core1, CoreCount core2) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("At {}, compare {} ({}) vs {} ({})", getId(), core1, core1.total(), core2,
+          core2.total());
+    }
+    return CoreCount.max(core1, core2);
+  }
+
+  /**
+   * Override parallelism of this fragment with adjusted parallelism from CPU costing
+   * algorithm.
+   * <p>Only valid after {@link #traverseEffectiveParallelism(int, int, int)}
+   * called.
+   */
+  protected void setEffectiveNumInstance() {
+    validateProcessingCosts();
+    if (originalInstanceCount_ <= 0) {
+      originalInstanceCount_ = getNumInstances();
+    }
+
+    if (LOG.isTraceEnabled() && originalInstanceCount_ != getAdjustedInstanceCount()) {
+      logCountAdjustmentTrace(originalInstanceCount_, getAdjustedInstanceCount(),
+          "Finalize effective parallelism.");
+    }
+
+    for (PlanNode node : collectPlanNodes()) {
+      node.numInstances_ = getAdjustedInstanceCount();
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("ProcessingCost Fragment {}:\n{}", getId(), debugProcessingCosts());
+    }
+  }
+
+  private void logCountAdjustmentTrace(int oldCount, int newCount, String reason) {
+    LOG.trace("{} adjust instance count from {} to {}. {}", getId(), oldCount, newCount,
+        reason);
+  }
+
+  private static boolean isBlockingNode(PlanNode node) {
+    // Preaggregation node can behave like final aggregation node when it does not
+    // passedthrough any row. From CPU costing perspective, treat both final aggregation
+    // and preaggregation as a blocking node. Otherwise, follow PlanNode.isBlockingNode().
+    return node.isBlockingNode() || node instanceof AggregationNode;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanNode.java b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
index 6dc9709c5..2bf5ea579 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -424,8 +424,8 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
       PlanFragment childFragment = children_.get(0).fragment_;
       if (fragment_ != childFragment && detailLevel == TExplainLevel.EXTENDED) {
         // we're crossing a fragment boundary - print the fragment header.
-        expBuilder.append(childFragment.getFragmentHeaderString(prefix, prefix,
-            queryOptions.getMt_dop()));
+        expBuilder.append(childFragment.getFragmentHeaderString(
+            prefix, prefix, queryOptions, detailLevel));
       }
       expBuilder.append(
           children_.get(0).getExplainString(prefix, prefix, queryOptions, detailLevel));
@@ -934,6 +934,11 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
         "Processing cost of PlanNode " + getDisplayLabel() + " is invalid!");
     processingCost_.setNumRowToConsume(getInputCardinality());
     processingCost_.setNumRowToProduce(getCardinality());
+    if (isLeafNode()
+        && (!fragment_.hasAdjustedInstanceCount()
+            || fragment_.getAdjustedInstanceCount() < getNumInstances())) {
+      fragment_.setFixedInstanceCount(getNumInstances());
+    }
   }
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
index 133aec186..8da761a32 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
@@ -226,4 +226,10 @@ public class PlanRootSink extends DataSink {
   public void collectExprs(List<Expr> exprs) {
     exprs.addAll(outputExprs_);
   }
+
+  @Override
+  public void computeRowConsumptionAndProductionToCost() {
+    super.computeRowConsumptionAndProductionToCost();
+    fragment_.setFixedInstanceCount(fragment_.getNumInstances());
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/Planner.java b/fe/src/main/java/org/apache/impala/planner/Planner.java
index 2e077fe31..a12b08824 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -20,6 +20,7 @@ package org.apache.impala.planner;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 import org.apache.impala.analysis.AnalysisContext;
@@ -38,7 +39,7 @@ import org.apache.impala.catalog.FeHBaseTable;
 import org.apache.impala.catalog.FeKuduTable;
 import org.apache.impala.catalog.FeTable;
 import org.apache.impala.common.ImpalaException;
-import org.apache.impala.common.NotImplementedException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.common.PrintUtils;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.service.BackendConfig;
@@ -59,6 +60,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import static org.apache.impala.analysis.ToSqlOptions.SHOW_IMPLICIT_CASTS;
 
@@ -212,7 +214,6 @@ public class Planner {
             List<String> mentionedColumns = insertStmt.getMentionedColumns();
             Preconditions.checkState(!mentionedColumns.isEmpty());
             List<ColumnLabel> targetColLabels = new ArrayList<>();
-            String tblFullName = targetTable.getFullName();
             for (String column: mentionedColumns) {
               targetColLabels.add(new ColumnLabel(column, targetTable.getTableName()));
             }
@@ -307,6 +308,15 @@ public class Planner {
         str.append(String.format("Dedicated Coordinator Resource Estimate: Memory=%s\n",
             PrintUtils.printBytesRoundedToMb(request.getDedicated_coord_mem_estimate())));
       }
+
+      TQueryOptions queryOptions =
+          request.getQuery_ctx().getClient_request().getQuery_options();
+      if (ProcessingCost.isComputeCost(queryOptions)
+          && explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+        str.append("Effective parallelism: ");
+        str.append(request.getCores_required());
+        str.append("\n");
+      }
       hasHeader = true;
     }
     // Warn if the planner is running in DEBUG mode.
@@ -398,6 +408,96 @@ public class Planner {
     return str.toString();
   }
 
+  /**
+   * Adjust effective parallelism of each plan fragment of query after considering
+   * processing cost rate and blocking operator.
+   * <p>
+   * Only valid after {@link PlanFragment#computeCostingSegment(TQueryOptions)} has
+   * been called for all plan fragments in the list.
+   */
+  private static void computeEffectiveParallelism(
+      List<PlanFragment> postOrderFragments, int minThreadPerNode, int maxThreadPerNode) {
+    for (PlanFragment fragment : postOrderFragments) {
+      if (!(fragment.getSink() instanceof JoinBuildSink)) {
+        // Only adjust parallelism of non-join build fragment.
+        // Join build fragment will be adjusted later by fragment hosting the join node.
+        fragment.traverseEffectiveParallelism(minThreadPerNode, maxThreadPerNode, -1);
+      }
+    }
+
+    for (PlanFragment fragment : postOrderFragments) {
+      fragment.setEffectiveNumInstance();
+    }
+  }
+
+  /**
+   * This method returns the effective CPU requirement of a query when considering
+   * processing cost rate and blocking operator.
+   * <p>
+   * Only valid after {@link #computeEffectiveParallelism(List, int, int)} has
+   * been called over the plan fragment list.
+   */
+  private static CoreCount computeBlockingAwareCores(
+      List<PlanFragment> postOrderFragments) {
+    // fragmentCoreState is a mapping between a fragment (through its PlanFragmentId) and
+    // its CoreCount. The first element of the pair is the CoreCount of subtree rooted at
+    // that fragment. The second element of the pair is the CoreCount of blocking-child
+    // subtrees under that fragment. The effective CoreCount of a fragment is derived from
+    // the pair through the following formula:
+    //   max(Pair.first, sum(Pair.second))
+    Map<PlanFragmentId, Pair<CoreCount, List<CoreCount>>> fragmentCoreState =
+        Maps.newHashMap();
+
+    for (PlanFragment fragment : postOrderFragments) {
+      fragment.computeBlockingAwareCores(fragmentCoreState);
+    }
+
+    PlanFragment root = postOrderFragments.get(postOrderFragments.size() - 1);
+    Pair<CoreCount, List<CoreCount>> rootCores = fragmentCoreState.get(root.getId());
+
+    return root.maxCore(rootCores.first, CoreCount.sum(rootCores.second));
+  }
+
+  /**
+   * Compute processing cost of each plan fragment in the query plan and adjust each
+   * fragment parallelism according to producer-consumer rate between them.
+   */
+  public static void computeProcessingCost(List<PlanFragment> planRoots,
+      TQueryExecRequest request, PlannerContext planCtx, int numCoresPerExecutor) {
+    TQueryOptions queryOptions = planCtx.getRootAnalyzer().getQueryOptions();
+
+    if (!ProcessingCost.isComputeCost(queryOptions)) {
+      request.setCores_required(-1);
+      return;
+    }
+
+    // TODO: remove dependence on MT_DOP in the future.
+    //   We still depend on MT_DOP here since many aspect of query planning is still
+    //   controlled through MT_DOP.
+    int mtDop = queryOptions.getMt_dop();
+    int numNode = planCtx.getRootAnalyzer().numExecutorsForPlanning();
+    int minThreads = queryOptions.getProcessing_cost_min_threads();
+    int maxThreads = Math.max(minThreads, Math.max(mtDop, numCoresPerExecutor));
+
+    PlanFragment rootFragment = planRoots.get(0);
+    List<PlanFragment> postOrderFragments = rootFragment.getNodesPostOrder();
+    for (PlanFragment fragment : postOrderFragments) {
+      fragment.computeCostingSegment(queryOptions);
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Computing effective parallelism. numNode=" + numNode + " mtDop=" + mtDop
+          + " numCoresPerExecutor=" + numCoresPerExecutor + " minThreads=" + minThreads
+          + " maxThreads=" + maxThreads);
+    }
+
+    computeEffectiveParallelism(postOrderFragments, minThreads, maxThreads);
+    CoreCount effectiveCores = computeBlockingAwareCores(postOrderFragments);
+    request.setCores_required(effectiveCores.total());
+
+    LOG.info("CoreCount=" + effectiveCores);
+  }
+
   /**
    * Computes the per-host resource profile for the given plans, i.e. the peak resources
    * consumed by all fragment instances belonging to the query per host. Sets the
diff --git a/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java b/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
index df695bddc..6093a28b3 100644
--- a/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
+++ b/fe/src/main/java/org/apache/impala/planner/ProcessingCost.java
@@ -20,6 +20,7 @@ package org.apache.impala.planner;
 import com.google.common.base.Preconditions;
 import com.google.common.math.LongMath;
 
+import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TQueryOptions;
 
 import java.math.RoundingMode;
@@ -96,8 +97,7 @@ public abstract class ProcessingCost implements Cloneable {
   }
 
   public static boolean isComputeCost(TQueryOptions queryOptions) {
-    // TODO: Replace with proper check in IMPALA-11604 part 2.
-    return false;
+    return queryOptions.getMt_dop() > 0 && queryOptions.isCompute_processing_cost();
   }
 
   /**
@@ -207,10 +207,8 @@ public abstract class ProcessingCost implements Cloneable {
   }
 
   private int getNumInstanceMax() {
-    // TODO: replace minProcessingCostPerThread with backend flag.
-    long minProcessingCostPerThread = 10000000L;
     long maxInstance = LongMath.divide(getTotalCost(),
-        minProcessingCostPerThread, RoundingMode.CEILING);
+        BackendConfig.INSTANCE.getMinProcessingPerThread(), RoundingMode.CEILING);
     if (maxInstance > 0) {
       return maxInstance < Integer.MAX_VALUE ? (int) maxInstance : Integer.MAX_VALUE;
     } else {
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index d3d06b880..f5daefa7e 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -380,4 +380,14 @@ public class BackendConfig {
   public TGeospatialLibrary getGeospatialLibrary() {
     return backendCfg_.geospatial_library;
   }
+
+  public double getQueryCpuCountDivisor() { return backendCfg_.query_cpu_count_divisor; }
+
+  public boolean isProcessingCostUseEqualExprWeight() {
+    return backendCfg_.processing_cost_use_equal_expr_weight;
+  }
+
+  public long getMinProcessingPerThread() {
+    return backendCfg_.min_processing_per_thread;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 40469880e..c9da2bdea 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -25,6 +25,7 @@ import com.google.common.base.Predicates;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import com.google.common.math.IntMath;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.Uninterruptibles;
 import java.io.IOException;
@@ -139,6 +140,7 @@ import org.apache.impala.hooks.QueryEventHookManager;
 import org.apache.impala.planner.HdfsScanNode;
 import org.apache.impala.planner.PlanFragment;
 import org.apache.impala.planner.Planner;
+import org.apache.impala.planner.ProcessingCost;
 import org.apache.impala.planner.ScanNode;
 import org.apache.impala.thrift.TAlterDbParams;
 import org.apache.impala.thrift.TBackendGflags;
@@ -267,6 +269,12 @@ public class Frontend {
       // TExecRequest.query_exec_request field.
       protected long estimated_memory_per_host_ = -1;
 
+      // The processing cores required to execute the query.
+      // Certain queries such as EXPLAIN that do not populate
+      // TExecRequest.query_exec_request. Therefore, cores requirement will be set here
+      // through setCoresRequired().
+      protected int cores_required_ = -1;
+
       // The initial length of content in explain buffer to help return the buffer
       // to the initial position prior to another auto-scaling compilation.
       protected int initialExplainBufLen_ = -1;
@@ -293,6 +301,9 @@ public class Frontend {
       public long getEstimatedMemoryPerHost() { return estimated_memory_per_host_; }
       public void setEstimatedMemoryPerHost(long x) { estimated_memory_per_host_ = x; }
 
+      public int getCoresRequired() { return cores_required_; }
+      public void setCoresRequired(int x) { cores_required_ = x; }
+
       // Capture the current state and initialize before iterative compilations begin.
       public void captureState() {
         disableAuthorization_ = false;
@@ -1738,6 +1749,8 @@ public class Frontend {
 
     // Compute resource requirements of the final plans.
     TQueryExecRequest result = new TQueryExecRequest();
+    Planner.computeProcessingCost(planRoots, result, planner.getPlannerCtx(),
+        planCtx.compilationState_.getGroupSet().getNum_cores_per_executor());
     Planner.computeResourceReqs(planRoots, queryCtx, result,
         planner.getPlannerCtx(), planner.getAnalysisResult().isQueryStmt());
 
@@ -1774,6 +1787,8 @@ public class Frontend {
     planCtx.compilationState_.setEstimatedMemoryPerHost(
         result.getPer_host_mem_estimate());
 
+    planCtx.compilationState_.setCoresRequired(result.getCores_required());
+
     return result;
   }
 
@@ -1900,14 +1915,18 @@ public class Frontend {
           + " does not map to any known executor group set.");
     }
 
-    // Sort 'executorGroupSets' by max_mem_limit field in ascending order. Use
-    // exec_group_name_prefix to break the tie.
+    // Sort 'executorGroupSets' by
+    //   <max_mem_limit, expected_num_executors * num_cores_per_executor>
+    // in ascending order. Use exec_group_name_prefix to break the tie.
     Collections.sort(result, new Comparator<TExecutorGroupSet>() {
       @Override
       public int compare(TExecutorGroupSet e1, TExecutorGroupSet e2) {
         int i = Long.compare(e1.getMax_mem_limit(), e2.getMax_mem_limit());
         if (i == 0) {
-          i = e1.getExec_group_name_prefix().compareTo(e2.getExec_group_name_prefix());
+          i = Long.compare(expectedTotalCores(e1), expectedTotalCores(e2));
+          if (i == 0) {
+            i = e1.getExec_group_name_prefix().compareTo(e2.getExec_group_name_prefix());
+          }
         }
         return i;
       }
@@ -1922,6 +1941,14 @@ public class Frontend {
     return type == TStmtType.EXPLAIN || type == TStmtType.QUERY || type == TStmtType.DML;
   }
 
+  private static int expectedTotalCores(TExecutorGroupSet execGroupSet) {
+    int numExecutors = execGroupSet.getCurr_num_executors() > 0 ?
+        execGroupSet.getCurr_num_executors() :
+        execGroupSet.getExpected_num_executors();
+    return IntMath.saturatedMultiply(
+        numExecutors, execGroupSet.getNum_cores_per_executor());
+  }
+
   private TExecRequest getTExecRequest(PlanCtx planCtx, EventSequence timeline)
       throws ImpalaException {
     TQueryCtx queryCtx = planCtx.getQueryContext();
@@ -1963,7 +1990,6 @@ public class Frontend {
     planCtx.compilationState_.captureState();
 
     TExecutorGroupSet group_set = null;
-    long per_host_mem_estimate = -1;
     String reason = "Unknown";
     int attempt = 0;
     for (int i = 0; i < num_executor_group_sets; i++) {
@@ -2008,20 +2034,34 @@ public class Frontend {
       }
 
       // Find out the per host memory estimated from two possible sources.
-      per_host_mem_estimate = -1;
+      long per_host_mem_estimate = -1;
+      int cores_requirement = -1;
       if (req.query_exec_request != null) {
         // For non-explain queries
         per_host_mem_estimate = req.query_exec_request.per_host_mem_estimate;
+        cores_requirement = req.query_exec_request.getCores_required();
       } else {
         // For explain queries
         per_host_mem_estimate = planCtx.compilationState_.getEstimatedMemoryPerHost();
+        cores_requirement = planCtx.compilationState_.getCoresRequired();
       }
 
       Preconditions.checkState(per_host_mem_estimate >= 0);
+      boolean cpuReqSatisfied = true;
+      int scaled_cores_requirement = -1;
+      if (ProcessingCost.isComputeCost(queryOptions)) {
+        Preconditions.checkState(cores_requirement > 0);
+        scaled_cores_requirement = (int) Math.min(Integer.MAX_VALUE,
+            Math.ceil(
+                cores_requirement / BackendConfig.INSTANCE.getQueryCpuCountDivisor()));
+        cpuReqSatisfied = scaled_cores_requirement <= expectedTotalCores(group_set);
+      }
 
-      if (per_host_mem_estimate <= group_set.getMax_mem_limit()) {
+      if (per_host_mem_estimate <= group_set.getMax_mem_limit() && cpuReqSatisfied) {
         reason = "suitable group found (estimated per-host memory="
-            + PrintUtils.printBytes(per_host_mem_estimate) + ")";
+            + PrintUtils.printBytes(per_host_mem_estimate)
+            + ", estimated cpu cores required=" + cores_requirement
+            + ", scaled cpu cores=" + scaled_cores_requirement + ")";
 
         // Set the group name prefix in both the returned query options and
         // the query context for non default group setup.
diff --git a/fe/src/main/java/org/apache/impala/util/ExprUtil.java b/fe/src/main/java/org/apache/impala/util/ExprUtil.java
index 2b34835da..6a7e2cedc 100644
--- a/fe/src/main/java/org/apache/impala/util/ExprUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/ExprUtil.java
@@ -111,12 +111,24 @@ public class ExprUtil {
     // TODO: Implement the cost for conjunts once the implemetation for
     // 'Expr' is in place.
     if (exprs == null) return 0;
-    return exprs.size();
+    if (BackendConfig.INSTANCE.isProcessingCostUseEqualExprWeight()) {
+      return exprs.size();
+    } else {
+      float totalCost = 0;
+      for (Expr e : exprs) {
+        totalCost += e.hasCost() ? e.getCost() : 1;
+      }
+      return totalCost;
+    }
   }
 
   public static float computeExprCost(Expr e) {
     if (e == null) return 0;
-    return 1;
+    if (BackendConfig.INSTANCE.isProcessingCostUseEqualExprWeight()) {
+      return 1;
+    } else {
+      return e.hasCost() ? e.getCost() : 1;
+    }
     // TODO Implement a function that can take into consideration of data types,
     // expressions and potentially LLVM translation in BE. The function must also
     // run fast.
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index bf48910ab..e86579a24 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -37,7 +37,6 @@ import org.apache.impala.testutil.TestUtils.IgnoreValueFilter;
 import org.apache.impala.thrift.TRuntimeFilterType;
 import org.apache.impala.thrift.TExecRequest;
 import org.apache.impala.thrift.TExplainLevel;
-import org.apache.impala.thrift.TExplainResult;
 import org.apache.impala.thrift.TJoinDistributionMode;
 import org.apache.impala.thrift.TKuduReplicaSelection;
 import org.apache.impala.thrift.TQueryCtx;
@@ -1358,4 +1357,22 @@ public class PlannerTest extends PlannerTestBase {
     runPlannerTestFile("explain-verbose-mt_dop", "tpcds_parquet",
         ImmutableSet.of(PlannerTestOption.INCLUDE_RESOURCE_HEADER));
   }
+
+  /**
+   * Test that processing cost can adjust effective instance count of fragment.
+   */
+  @Test
+  public void testProcessingCost() {
+    TQueryOptions options = new TQueryOptions();
+    options.setMt_dop(4);
+    options.setCompute_processing_cost(true);
+    options.setMinmax_filter_threshold(0.5);
+    options.setMinmax_filter_sorted_columns(false);
+    options.setMinmax_filter_partition_columns(false);
+    runPlannerTestFile("tpcds-processing-cost", "tpcds_parquet", options,
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN,
+            PlannerTestOption.INCLUDE_RESOURCE_HEADER,
+            PlannerTestOption.VALIDATE_RESOURCES,
+            PlannerTestOption.VALIDATE_CARDINALITY));
+  }
 }
diff --git a/fe/src/test/resources/fair-scheduler-2-groups.xml b/fe/src/test/resources/fair-scheduler-3-groups.xml
similarity index 88%
rename from fe/src/test/resources/fair-scheduler-2-groups.xml
rename to fe/src/test/resources/fair-scheduler-3-groups.xml
index 0472a31e9..81b71e33f 100644
--- a/fe/src/test/resources/fair-scheduler-2-groups.xml
+++ b/fe/src/test/resources/fair-scheduler-3-groups.xml
@@ -1,6 +1,8 @@
 <?xml version="1.0"?>
 <allocations>
   <queue name="root">
+    <queue name="tiny">
+    </queue>
     <queue name="small">
     </queue>
     <queue name="large">
diff --git a/fe/src/test/resources/llama-site-2-groups.xml b/fe/src/test/resources/llama-site-3-groups.xml
similarity index 70%
rename from fe/src/test/resources/llama-site-2-groups.xml
rename to fe/src/test/resources/llama-site-3-groups.xml
index f06b35a28..6fdd67697 100644
--- a/fe/src/test/resources/llama-site-2-groups.xml
+++ b/fe/src/test/resources/llama-site-3-groups.xml
@@ -1,6 +1,28 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <configuration>
   <!-- Default values -->
+  <property>
+    <name>impala.admission-control.pool-queue-timeout-ms.root.tiny</name>
+    <value>100000</value>
+  </property>
+  <property>
+    <name>impala.admission-control.max-query-mem-limit.root.tiny</name>
+   <!-- 64 MB -->
+    <value>67108864</value>
+  </property>
+  <property>
+    <name>impala.admission-control.min-query-mem-limit.root.tiny</name>
+   <!-- 0MB -->
+    <value>0</value>
+  </property>
+  <property>
+    <name>impala.admission-control.max-query-cpu-core-per-node-limit.root.tiny</name>
+    <value>2</value>
+  </property>
+  <property>
+    <name>impala.admission-control.max-query-cpu-core-coordinator-limit.root.tiny</name>
+    <value>2</value>
+  </property>
   <property>
     <name>llama.am.throttling.maximum.placed.reservations.root.small</name>
     <value>10</value>
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
new file mode 100644
index 000000000..d930c697f
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-processing-cost.test
@@ -0,0 +1,8739 @@
+# TPCDS-Q3
+select
+  dt.d_year,
+  item.i_brand_id brand_id,
+  item.i_brand brand,
+  sum(ss_ext_sales_price) sum_agg
+from
+  date_dim dt,
+  store_sales,
+  item
+where
+  dt.d_date_sk = store_sales.ss_sold_date_sk
+  and store_sales.ss_item_sk = item.i_item_sk
+  and item.i_manufact_id = 436
+  and dt.d_moy = 12
+group by
+  dt.d_year,
+  item.i_brand,
+  item.i_brand_id
+order by
+  dt.d_year,
+  sum_agg desc,
+  brand_id
+limit 100
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=26.38MB Threads=11
+Per-Host Resource Estimates: Memory=139MB
+F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[406]
+PLAN-ROOT SINK
+|  output exprs: dt.d_year, item.i_brand_id, item.i_brand, sum(ss_ext_sales_price)
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=400
+|
+12:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
+|  limit: 100
+|  mem-estimate=18.16KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=4 row-size=52B cardinality=100 cost=6
+|  in pipelines: 06(GETNEXT)
+|
+F04:PLAN FRAGMENT [HASH(dt.d_year,item.i_brand,item.i_brand_id)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=10.22MB mem-reservation=1.94MB thread-reservation=1
+max-parallelism=3 fragment-costs=[12331, 300, 6]
+06:TOP-N [LIMIT=100]
+|  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
+|  mem-estimate=5.10KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=4 row-size=52B cardinality=100 cost=300
+|  in pipelines: 06(GETNEXT), 11(OPEN)
+|
+11:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_ext_sales_price)
+|  group by: dt.d_year, item.i_brand, item.i_brand_id
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3 row-size=52B cardinality=3.04K cost=12164
+|  in pipelines: 11(GETNEXT), 01(OPEN)
+|
+10:EXCHANGE [HASH(dt.d_year,item.i_brand,item.i_brand_id)]
+|  mem-estimate=220.26KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=52B cardinality=3.04K cost=167
+|  in pipelines: 01(GETNEXT)
+|
+F03:PLAN FRAGMENT [HASH(store_sales.ss_sold_date_sk)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=11.52MB mem-reservation=2.00MB thread-reservation=1
+max-parallelism=3 fragment-costs=[15408, 167]
+05:AGGREGATE [STREAMING]
+|  output: sum(ss_ext_sales_price)
+|  group by: dt.d_year, item.i_brand, item.i_brand_id
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3 row-size=52B cardinality=3.04K cost=12164
+|  in pipelines: 01(GETNEXT)
+|
+04:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: store_sales.ss_sold_date_sk = dt.d_date_sk
+|  fk/pk conjuncts: store_sales.ss_sold_date_sk = dt.d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=1,2,0 row-size=72B cardinality=3.04K cost=3041
+|  in pipelines: 01(GETNEXT), 00(OPEN)
+|
+|--F06:PLAN FRAGMENT [HASH(store_sales.ss_sold_date_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=3.02MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[6183]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: dt.d_date_sk
+|  |  runtime filters: RF000[bloom] <- dt.d_date_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=6087
+|  |
+|  09:EXCHANGE [HASH(dt.d_date_sk)]
+|  |  mem-estimate=87.33KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0 row-size=12B cardinality=6.09K cost=96
+|  |  in pipelines: 00(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.19MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[74002]
+|  00:SCAN HDFS [tpcds_parquet.date_dim dt, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: dt.d_moy = CAST(12 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: dt.d_moy = CAST(12 AS INT)
+|     parquet dictionary predicates: dt.d_moy = CAST(12 AS INT)
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=0 row-size=12B cardinality=6.09K cost=73906
+|     in pipelines: 00(GETNEXT)
+|
+08:EXCHANGE [HASH(store_sales.ss_sold_date_sk)]
+|  mem-estimate=877.96KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=1,2 row-size=60B cardinality=3.04K cost=203
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=16.80MB mem-reservation=1.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[2925614]
+03:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=01
+|  hash predicates: store_sales.ss_item_sk = item.i_item_sk
+|  fk/pk conjuncts: store_sales.ss_item_sk = item.i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=1,2 row-size=60B cardinality=3.04K cost=2880404
+|  in pipelines: 01(GETNEXT), 02(OPEN)
+|
+|--F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[22]
+|  JOIN BUILD
+|  |  join-table-id=01 plan-id=02 cohort-id=01
+|  |  build expressions: item.i_item_sk
+|  |  runtime filters: RF002[bloom] <- item.i_item_sk, RF003[min_max] <- item.i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=19
+|  |
+|  07:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=44B cardinality=19 cost=3
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.19MB mem-reservation=256.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[18778]
+|  02:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     predicates: item.i_manufact_id = CAST(436 AS INT)
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     parquet statistics predicates: item.i_manufact_id = CAST(436 AS INT)
+|     parquet dictionary predicates: item.i_manufact_id = CAST(436 AS INT)
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=2 row-size=44B cardinality=19 cost=18777
+|     in pipelines: 02(GETNEXT)
+|
+01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF003[min_max] -> store_sales.ss_item_sk, RF000[bloom] -> store_sales.ss_sold_date_sk, RF002[bloom] -> store_sales.ss_item_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=1 row-size=16B cardinality=2.88M cost=45007
+   in pipelines: 01(GETNEXT)
+====
+# TPCDS-Q14a first of two
+with  cross_items as
+ (select i_item_sk ss_item_sk
+ from item,
+ (select iss.i_brand_id brand_id
+     ,iss.i_class_id class_id
+     ,iss.i_category_id category_id
+ from store_sales
+     ,item iss
+     ,date_dim d1
+ where ss_item_sk = iss.i_item_sk
+   and ss_sold_date_sk = d1.d_date_sk
+   and d1.d_year between 1999 AND 1999 + 2
+ intersect
+ select ics.i_brand_id
+     ,ics.i_class_id
+     ,ics.i_category_id
+ from catalog_sales
+     ,item ics
+     ,date_dim d2
+ where cs_item_sk = ics.i_item_sk
+   and cs_sold_date_sk = d2.d_date_sk
+   and d2.d_year between 1999 AND 1999 + 2
+ intersect
+ select iws.i_brand_id
+     ,iws.i_class_id
+     ,iws.i_category_id
+ from web_sales
+     ,item iws
+     ,date_dim d3
+ where ws_item_sk = iws.i_item_sk
+   and ws_sold_date_sk = d3.d_date_sk
+   and d3.d_year between 1999 AND 1999 + 2) t1
+ where i_brand_id = brand_id
+      and i_class_id = class_id
+      and i_category_id = category_id
+),
+ avg_sales as
+ (select avg(quantity*list_price) average_sales
+  from (select ss_quantity quantity
+             ,ss_list_price list_price
+       from store_sales
+           ,date_dim
+       where ss_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2
+       union all
+       select cs_quantity quantity
+             ,cs_list_price list_price
+       from catalog_sales
+           ,date_dim
+       where cs_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2
+       union all
+       select ws_quantity quantity
+             ,ws_list_price list_price
+       from web_sales
+           ,date_dim
+       where ws_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2) x)
+ select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales)
+ from(
+       select 'store' channel, i_brand_id,i_class_id
+             ,i_category_id,sum(ss_quantity*ss_list_price) sales
+             , count(*) number_sales
+       from store_sales
+           ,item
+           ,date_dim
+       where ss_item_sk in (select ss_item_sk from cross_items)
+         and ss_item_sk = i_item_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year = 1999+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales
+       from catalog_sales
+           ,item
+           ,date_dim
+       where cs_item_sk in (select ss_item_sk from cross_items)
+         and cs_item_sk = i_item_sk
+         and cs_sold_date_sk = d_date_sk
+         and d_year = 1999+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales
+       from web_sales
+           ,item
+           ,date_dim
+       where ws_item_sk in (select ss_item_sk from cross_items)
+         and ws_item_sk = i_item_sk
+         and ws_sold_date_sk = d_date_sk
+         and d_year = 1999+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales)
+ ) y
+ group by rollup (channel, i_brand_id,i_class_id,i_category_id)
+ order by channel,i_brand_id,i_class_id,i_category_id
+LIMIT 100
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=780.38MB Threads=141
+Per-Host Resource Estimates: Memory=2.81GB
+F80:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[606]
+PLAN-ROOT SINK
+|  output exprs: CASE valid_tid(104,105,106,107,108) WHEN 104 THEN channel WHEN 105 THEN channel WHEN 106 THEN channel WHEN 107 THEN channel WHEN 108 THEN NULL END, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_brand_id WHEN 105 THEN i_brand_id WHEN 106 THEN i_brand_id WHEN 107 THEN NULL WHEN 108 THEN NULL END, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_class_id WHEN 105 THEN i_class_id WHEN 106 THEN NULL WHEN 107 THEN NULL WHEN 108 THEN NULL END, CASE valid_tid(104,105,10 [...]
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=600
+|
+216:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: CASE valid_tid(104,105,106,107,108) WHEN 104 THEN channel WHEN 105 THEN channel WHEN 106 THEN channel WHEN 107 THEN channel WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_brand_id WHEN 105 THEN i_brand_id WHEN 106 THEN i_brand_id WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_class_id WHEN 105 THEN i_class_id WHEN 106 THEN NULL WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(10 [...]
+|  limit: 100
+|  mem-estimate=16.80KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=110 row-size=48B cardinality=100 cost=6
+|  in pipelines: 129(GETNEXT)
+|
+F79:PLAN FRAGMENT [HASH(CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(channel) WHEN 105 THEN murmur_hash(channel) WHEN 106 THEN murmur_hash(channel) WHEN 107 THEN murmur_hash(channel) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(i_brand_id) WHEN 105 THEN murmur_hash(i_brand_id) WHEN 106 THEN murmur_hash(i_brand_id) WHEN 107 THEN murmur_hash(NULL) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WH [...]
+Per-Instance Resources: mem-estimate=137.09MB mem-reservation=93.62MB thread-reservation=1
+max-parallelism=3 fragment-costs=[17011772, 3936100, 400, 6]
+129:TOP-N [LIMIT=100]
+|  order by: CASE valid_tid(104,105,106,107,108) WHEN 104 THEN channel WHEN 105 THEN channel WHEN 106 THEN channel WHEN 107 THEN channel WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_brand_id WHEN 105 THEN i_brand_id WHEN 106 THEN i_brand_id WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(104,105,106,107,108) WHEN 104 THEN i_class_id WHEN 105 THEN i_class_id WHEN 106 THEN NULL WHEN 107 THEN NULL WHEN 108 THEN NULL END ASC, CASE valid_tid(10 [...]
+|  mem-estimate=4.69KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=110 row-size=48B cardinality=100 cost=400
+|  in pipelines: 129(GETNEXT), 128(OPEN)
+|
+128:AGGREGATE [FINALIZE]
+|  output: aggif(valid_tid(104,105,106,107,108) IN (CAST(104 AS INT), CAST(105 AS INT), CAST(106 AS INT), CAST(107 AS INT), CAST(108 AS INT)), CASE valid_tid(104,105,106,107,108) WHEN CAST(104 AS INT) THEN sum(sales) WHEN CAST(105 AS INT) THEN sum(sales) WHEN CAST(106 AS INT) THEN sum(sales) WHEN CAST(107 AS INT) THEN sum(sales) WHEN CAST(108 AS INT) THEN sum(sales) END), aggif(valid_tid(104,105,106,107,108) IN (CAST(104 AS INT), CAST(105 AS INT), CAST(106 AS INT), CAST(107 AS INT), CAST [...]
+|  group by: CASE valid_tid(104,105,106,107,108) WHEN CAST(104 AS INT) THEN channel WHEN CAST(105 AS INT) THEN channel WHEN CAST(106 AS INT) THEN channel WHEN CAST(107 AS INT) THEN channel WHEN CAST(108 AS INT) THEN NULL END, CASE valid_tid(104,105,106,107,108) WHEN CAST(104 AS INT) THEN i_brand_id WHEN CAST(105 AS INT) THEN i_brand_id WHEN CAST(106 AS INT) THEN i_brand_id WHEN CAST(107 AS INT) THEN NULL WHEN CAST(108 AS INT) THEN NULL END, CASE valid_tid(104,105,106,107,108) WHEN CAST(1 [...]
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=109 row-size=52B cardinality=562.30K cost=3936100
+|  in pipelines: 128(GETNEXT), 215(OPEN)
+|
+215:AGGREGATE [FINALIZE]
+|  Class 0
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: channel, i_brand_id, i_class_id, i_category_id
+|  Class 1
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: channel, i_brand_id, i_class_id, NULL
+|  Class 2
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: channel, i_brand_id, NULL, NULL
+|  Class 3
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: channel, NULL, NULL, NULL
+|  Class 4
+|    output: sum:merge(sales), sum:merge(number_sales)
+|    group by: NULL, NULL, NULL, NULL
+|  mem-estimate=120.09MB mem-reservation=76.62MB thread-reservation=0
+|  tuple-ids=104N,105N,106N,107N,108N row-size=240B cardinality=562.30K cost=16869000
+|  in pipelines: 215(GETNEXT), 151(OPEN), 179(OPEN), 207(OPEN)
+|
+214:EXCHANGE [HASH(CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(channel) WHEN 105 THEN murmur_hash(channel) WHEN 106 THEN murmur_hash(channel) WHEN 107 THEN murmur_hash(channel) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 104 THEN murmur_hash(i_brand_id) WHEN 105 THEN murmur_hash(i_brand_id) WHEN 106 THEN murmur_hash(i_brand_id) WHEN 107 THEN murmur_hash(NULL) WHEN 108 THEN murmur_hash(NULL) END,CASE valid_tid(104,105,106,107,108) WHEN 10 [...]
+|  mem-estimate=13.05MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=104N,105N,106N,107N,108N row-size=240B cardinality=562.30K cost=142772
+|  in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT)
+|
+F78:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Instance Resources: mem-estimate=65.22MB mem-reservation=28.94MB thread-reservation=1
+max-parallelism=12 fragment-costs=[749813, 429878, 215944, 13294224, 142772]
+127:AGGREGATE [STREAMING]
+|  Class 0
+|    output: sum(sales), sum(number_sales)
+|    group by: channel, i_brand_id, i_class_id, i_category_id
+|  Class 1
+|    output: sum(sales), sum(number_sales)
+|    group by: channel, i_brand_id, i_class_id, NULL
+|  Class 2
+|    output: sum(sales), sum(number_sales)
+|    group by: channel, i_brand_id, NULL, NULL
+|  Class 3
+|    output: sum(sales), sum(number_sales)
+|    group by: channel, NULL, NULL, NULL
+|  Class 4
+|    output: sum(sales), sum(number_sales)
+|    group by: NULL, NULL, NULL, NULL
+|  mem-estimate=50.00MB mem-reservation=27.00MB thread-reservation=0
+|  tuple-ids=104N,105N,106N,107N,108N row-size=240B cardinality=562.30K cost=8308890
+|  in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT)
+|
+00:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=102 row-size=48B cardinality=276.96K cost=4985334
+|  in pipelines: 151(GETNEXT), 179(GETNEXT), 207(GETNEXT)
+|
+|--126:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  |  join table id: 08
+|  |  predicates: sum(ws_quantity * ws_list_price) > avg(quantity * list_price)
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=88,99 row-size=52B cardinality=42.85K cost=0
+|  |  in pipelines: 207(GETNEXT), 212(OPEN)
+|  |
+|  |--F89:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  JOIN BUILD
+|  |  |  join-table-id=08 plan-id=09 cohort-id=01
+|  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |
+|  |  213:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=99 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 212(GETNEXT)
+|  |  |
+|  |  F77:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  212:AGGREGATE [FINALIZE]
+|  |  |  output: avg:merge(quantity * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=99 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 212(GETNEXT), 125(OPEN)
+|  |  |
+|  |  211:EXCHANGE [UNPARTITIONED]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=98 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 125(GETNEXT)
+|  |  |
+|  |  F76:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[40389768, 1]
+|  |  125:AGGREGATE
+|  |  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=98 row-size=16B cardinality=1 cost=5041336
+|  |  |  in pipelines: 125(GETNEXT), 116(OPEN), 119(OPEN), 122(OPEN)
+|  |  |
+|  |  115:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=96 row-size=8B cardinality=5.04M cost=30248016
+|  |  |  in pipelines: 116(GETNEXT), 119(GETNEXT), 122(GETNEXT)
+|  |  |
+|  |  |--124:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=11
+|  |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=94,95 row-size=20B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 122(GETNEXT), 123(OPEN)
+|  |  |  |
+|  |  |  |--F92:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=11 plan-id=12 cohort-id=04
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF107[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  210:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=95 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 123(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F75:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  123:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=95 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 123(GETNEXT)
+|  |  |  |
+|  |  |  122:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF107[min_max] -> ws_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=94 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 122(GETNEXT)
+|  |  |
+|  |  |--121:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=10
+|  |  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=92,93 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 119(GETNEXT), 120(OPEN)
+|  |  |  |
+|  |  |  |--F91:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=04
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF105[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  209:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=93 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 120(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F73:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  120:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=93 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 120(GETNEXT)
+|  |  |  |
+|  |  |  119:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF105[min_max] -> cs_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=92 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 119(GETNEXT)
+|  |  |
+|  |  118:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=09
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=90,91 row-size=20B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 116(GETNEXT), 117(OPEN)
+|  |  |
+|  |  |--F90:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=09 plan-id=10 cohort-id=04
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  208:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=91 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 117(GETNEXT)
+|  |  |  |
+|  |  |  F71:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  117:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=91 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 117(GETNEXT)
+|  |  |
+|  |  116:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=90 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 116(GETNEXT)
+|  |
+|  207:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ws_quantity * ws_list_price), count:merge(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=88 row-size=36B cardinality=42.85K cost=214270
+|  |  in pipelines: 207(GETNEXT), 85(OPEN)
+|  |
+|  206:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  mem-estimate=833.29KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=88 row-size=36B cardinality=42.85K cost=1674
+|  |  in pipelines: 85(GETNEXT)
+|  |
+|  F52:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=43.88MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=2 fragment-costs=[1033413, 1674]
+|  114:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price), count(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=88 row-size=36B cardinality=42.85K cost=214270
+|  |  in pipelines: 85(GETNEXT)
+|  |
+|  113:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=36
+|  |  hash predicates: ws_item_sk = tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=68,70,69 row-size=52B cardinality=42.85K cost=42854
+|  |  in pipelines: 85(GETNEXT), 204(OPEN)
+|  |
+|  |--F117:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[18397]
+|  |  JOIN BUILD
+|  |  |  join-table-id=36 plan-id=37 cohort-id=01
+|  |  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  |  runtime filters: RF073[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |
+|  |  205:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=170 row-size=8B cardinality=17.98K cost=422
+|  |  |  in pipelines: 204(GETNEXT)
+|  |  |
+|  |  F68:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  204:AGGREGATE [FINALIZE]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=170 row-size=8B cardinality=17.98K cost=17975
+|  |  |  in pipelines: 204(GETNEXT), 191(OPEN)
+|  |  |
+|  |  203:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=170 row-size=8B cardinality=17.98K cost=211
+|  |  |  in pipelines: 191(GETNEXT)
+|  |  |
+|  |  F58:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  |  112:AGGREGATE [STREAMING]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=170 row-size=8B cardinality=17.98K cost=148800
+|  |  |  in pipelines: 191(GETNEXT)
+|  |  |
+|  |  109:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=37
+|  |  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=75,71 row-size=32B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 191(GETNEXT), 88(OPEN)
+|  |  |
+|  |  |--F118:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=2.30MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=37 plan-id=38 cohort-id=11
+|  |  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  |  runtime filters: RF081[min_max] <- i_brand_id, RF082[min_max] <- i_category_id, RF083[min_max] <- i_class_id
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |  |
+|  |  |  202:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=71 row-size=20B cardinality=18.00K cost=422
+|  |  |  |  in pipelines: 88(GETNEXT)
+|  |  |  |
+|  |  |  F67:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  88:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=71 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 88(GETNEXT)
+|  |  |
+|  |  108:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=38
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 191(GETNEXT), 200(OPEN)
+|  |  |
+|  |  |--F119:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=38 plan-id=39 cohort-id=11
+|  |  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  201:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=169 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 200(GETNEXT)
+|  |  |  |
+|  |  |  F66:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  |  200:AGGREGATE [FINALIZE]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=169 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 200(GETNEXT), 100(OPEN)
+|  |  |  |
+|  |  |  199:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=169 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 100(GETNEXT)
+|  |  |  |
+|  |  |  F63:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  |  107:AGGREGATE [STREAMING]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=169 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  |  in pipelines: 100(GETNEXT)
+|  |  |  |
+|  |  |  104:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=39
+|  |  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=81,82,83 row-size=40B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 100(GETNEXT), 102(OPEN)
+|  |  |  |
+|  |  |  |--F120:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=39 plan-id=40 cohort-id=12
+|  |  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  |  runtime filters: RF099[min_max] <- d3.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  198:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=83 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  |  in pipelines: 102(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F65:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  102:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=83 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 102(GETNEXT)
+|  |  |  |
+|  |  |  103:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=40
+|  |  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=81,82 row-size=32B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 100(GETNEXT), 101(OPEN)
+|  |  |  |
+|  |  |  |--F121:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=40 plan-id=41 cohort-id=12
+|  |  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  |  runtime filters: RF101[min_max] <- iws.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  197:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=82 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  |  in pipelines: 101(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F64:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  101:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=82 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 101(GETNEXT)
+|  |  |  |
+|  |  |  100:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF099[min_max] -> ws_sold_date_sk, RF101[min_max] -> ws_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=81 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 100(GETNEXT)
+|  |  |
+|  |  106:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=41
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 191(GETNEXT), 195(OPEN)
+|  |  |
+|  |  |--F122:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=41 plan-id=42 cohort-id=11
+|  |  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  196:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=168 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 195(GETNEXT)
+|  |  |  |
+|  |  |  F62:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  |  195:AGGREGATE [FINALIZE]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=168 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 195(GETNEXT), 95(OPEN)
+|  |  |  |
+|  |  |  194:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=168 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 95(GETNEXT)
+|  |  |  |
+|  |  |  F59:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  |  105:AGGREGATE [STREAMING]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=168 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  |  in pipelines: 95(GETNEXT)
+|  |  |  |
+|  |  |  99:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=42
+|  |  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=77,78,79 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 95(GETNEXT), 97(OPEN)
+|  |  |  |
+|  |  |  |--F123:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=42 plan-id=43 cohort-id=13
+|  |  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  |  runtime filters: RF095[min_max] <- d2.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  193:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=79 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 97(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F61:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  97:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=79 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 97(GETNEXT)
+|  |  |  |
+|  |  |  98:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=43
+|  |  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=77,78 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 95(GETNEXT), 96(OPEN)
+|  |  |  |
+|  |  |  |--F124:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=43 plan-id=44 cohort-id=13
+|  |  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  |  runtime filters: RF097[min_max] <- ics.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  192:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=78 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  |  in pipelines: 96(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F60:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  96:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=78 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 96(GETNEXT)
+|  |  |  |
+|  |  |  95:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF095[min_max] -> cs_sold_date_sk, RF097[min_max] -> cs_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=77 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 95(GETNEXT)
+|  |  |
+|  |  191:AGGREGATE [FINALIZE]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 191(GETNEXT), 89(OPEN)
+|  |  |
+|  |  190:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 89(GETNEXT)
+|  |  |
+|  |  F55:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  |  94:AGGREGATE [STREAMING]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=75 row-size=12B cardinality=148.80K cost=8641212
+|  |  |  in pipelines: 89(GETNEXT)
+|  |  |
+|  |  93:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=44
+|  |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=72,73,74 row-size=40B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 89(GETNEXT), 91(OPEN)
+|  |  |
+|  |  |--F125:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=44 plan-id=45 cohort-id=11
+|  |  |  |  build expressions: d1.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  189:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=74 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 91(GETNEXT)
+|  |  |  |
+|  |  |  F57:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  91:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=74 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 91(GETNEXT)
+|  |  |
+|  |  92:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=45
+|  |  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=72,73 row-size=32B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 89(GETNEXT), 90(OPEN)
+|  |  |
+|  |  |--F126:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=45 plan-id=46 cohort-id=11
+|  |  |  |  build expressions: iss.i_item_sk
+|  |  |  |  runtime filters: RF093[min_max] <- iss.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  188:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=73 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 90(GETNEXT)
+|  |  |  |
+|  |  |  F56:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  90:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     runtime filters: RF081[min_max] -> iss.i_brand_id, RF082[min_max] -> iss.i_category_id, RF083[min_max] -> iss.i_class_id
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=73 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 90(GETNEXT)
+|  |  |
+|  |  89:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF093[min_max] -> ss_item_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=72 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 89(GETNEXT)
+|  |
+|  111:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=46
+|  |  hash predicates: ws_item_sk = i_item_sk
+|  |  fk/pk conjuncts: ws_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=68,70,69 row-size=52B cardinality=42.85K cost=42854
+|  |  in pipelines: 85(GETNEXT), 86(OPEN)
+|  |
+|  |--F127:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  JOIN BUILD
+|  |  |  join-table-id=46 plan-id=47 cohort-id=01
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF075[min_max] <- i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  187:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=69 row-size=20B cardinality=18.00K cost=844
+|  |  |  in pipelines: 86(GETNEXT)
+|  |  |
+|  |  F54:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  86:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF073[min_max] -> tpcds_parquet.item.i_item_sk
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=69 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 86(GETNEXT)
+|  |
+|  110:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=47
+|  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=68,70 row-size=32B cardinality=42.85K cost=719384
+|  |  in pipelines: 85(GETNEXT), 87(OPEN)
+|  |
+|  |--F128:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=2 fragment-costs=[112]
+|  |  JOIN BUILD
+|  |  |  join-table-id=47 plan-id=48 cohort-id=01
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF076[bloom] <- d_date_sk, RF077[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  186:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=70 row-size=12B cardinality=108 cost=4
+|  |  |  in pipelines: 87(GETNEXT)
+|  |  |
+|  |  F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  87:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=70 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 87(GETNEXT)
+|  |
+|  85:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|     HDFS partitions=1/1 files=2 size=45.09MB
+|     runtime filters: RF077[min_max] -> ws_sold_date_sk, RF073[min_max] -> ws_item_sk, RF075[min_max] -> ws_item_sk, RF076[bloom] -> ws_sold_date_sk
+|     stored statistics:
+|       table: rows=719.38K size=45.09MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|     tuple-ids=68 row-size=20B cardinality=719.38K cost=14051
+|     in pipelines: 85(GETNEXT)
+|
+|--84:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  |  join table id: 04
+|  |  predicates: sum(cs_quantity * cs_list_price) > avg(quantity * list_price)
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=54,65 row-size=52B cardinality=85.31K cost=0
+|  |  in pipelines: 179(GETNEXT), 184(OPEN)
+|  |
+|  |--F85:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=01
+|  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |
+|  |  185:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=65 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 184(GETNEXT)
+|  |  |
+|  |  F51:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  184:AGGREGATE [FINALIZE]
+|  |  |  output: avg:merge(quantity * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=65 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 184(GETNEXT), 83(OPEN)
+|  |  |
+|  |  183:EXCHANGE [UNPARTITIONED]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=64 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 83(GETNEXT)
+|  |  |
+|  |  F50:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[40389768, 1]
+|  |  83:AGGREGATE
+|  |  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=64 row-size=16B cardinality=1 cost=5041336
+|  |  |  in pipelines: 83(GETNEXT), 74(OPEN), 77(OPEN), 80(OPEN)
+|  |  |
+|  |  73:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=62 row-size=8B cardinality=5.04M cost=30248016
+|  |  |  in pipelines: 74(GETNEXT), 77(GETNEXT), 80(GETNEXT)
+|  |  |
+|  |  |--82:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=07
+|  |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=60,61 row-size=20B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 80(GETNEXT), 81(OPEN)
+|  |  |  |
+|  |  |  |--F88:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=07 plan-id=08 cohort-id=03
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF071[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  182:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=61 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 81(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F49:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  81:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=61 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 81(GETNEXT)
+|  |  |  |
+|  |  |  80:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF071[min_max] -> ws_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=60 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 80(GETNEXT)
+|  |  |
+|  |  |--79:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=06
+|  |  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=58,59 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 77(GETNEXT), 78(OPEN)
+|  |  |  |
+|  |  |  |--F87:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=06 plan-id=07 cohort-id=03
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF069[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  181:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=59 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 78(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F47:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  78:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=59 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 78(GETNEXT)
+|  |  |  |
+|  |  |  77:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF069[min_max] -> cs_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=58 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 77(GETNEXT)
+|  |  |
+|  |  76:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=05
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=56,57 row-size=20B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 74(GETNEXT), 75(OPEN)
+|  |  |
+|  |  |--F86:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=05 plan-id=06 cohort-id=03
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  180:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=57 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 75(GETNEXT)
+|  |  |  |
+|  |  |  F45:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  75:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=57 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 75(GETNEXT)
+|  |  |
+|  |  74:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=56 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 74(GETNEXT)
+|  |
+|  179:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(cs_quantity * cs_list_price), count:merge(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=54 row-size=36B cardinality=85.31K cost=426545
+|  |  in pipelines: 179(GETNEXT), 43(OPEN)
+|  |
+|  178:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  mem-estimate=1.09MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=54 row-size=36B cardinality=85.31K cost=3333
+|  |  in pipelines: 43(GETNEXT)
+|  |
+|  F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=59.88MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[2066867, 3333]
+|  72:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price), count(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=54 row-size=36B cardinality=85.31K cost=426545
+|  |  in pipelines: 43(GETNEXT)
+|  |
+|  71:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=24
+|  |  hash predicates: cs_item_sk = tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=34,36,35 row-size=52B cardinality=85.31K cost=85309
+|  |  in pipelines: 43(GETNEXT), 176(OPEN)
+|  |
+|  |--F105:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[18608]
+|  |  JOIN BUILD
+|  |  |  join-table-id=24 plan-id=25 cohort-id=01
+|  |  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  |  runtime filters: RF037[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |
+|  |  177:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=148 row-size=8B cardinality=17.98K cost=633
+|  |  |  in pipelines: 176(GETNEXT)
+|  |  |
+|  |  F42:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  176:AGGREGATE [FINALIZE]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=148 row-size=8B cardinality=17.98K cost=17975
+|  |  |  in pipelines: 176(GETNEXT), 163(OPEN)
+|  |  |
+|  |  175:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=148 row-size=8B cardinality=17.98K cost=211
+|  |  |  in pipelines: 163(GETNEXT)
+|  |  |
+|  |  F32:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  |  70:AGGREGATE [STREAMING]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=148 row-size=8B cardinality=17.98K cost=148800
+|  |  |  in pipelines: 163(GETNEXT)
+|  |  |
+|  |  67:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=25
+|  |  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=41,37 row-size=32B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 163(GETNEXT), 46(OPEN)
+|  |  |
+|  |  |--F106:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=2.30MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=25 plan-id=26 cohort-id=08
+|  |  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  |  runtime filters: RF045[min_max] <- i_brand_id, RF046[min_max] <- i_category_id, RF047[min_max] <- i_class_id
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |  |
+|  |  |  174:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=37 row-size=20B cardinality=18.00K cost=422
+|  |  |  |  in pipelines: 46(GETNEXT)
+|  |  |  |
+|  |  |  F41:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  46:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=37 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 46(GETNEXT)
+|  |  |
+|  |  66:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=26
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 163(GETNEXT), 172(OPEN)
+|  |  |
+|  |  |--F107:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=26 plan-id=27 cohort-id=08
+|  |  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  173:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=147 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 172(GETNEXT)
+|  |  |  |
+|  |  |  F40:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  |  172:AGGREGATE [FINALIZE]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=147 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 172(GETNEXT), 58(OPEN)
+|  |  |  |
+|  |  |  171:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=147 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 58(GETNEXT)
+|  |  |  |
+|  |  |  F37:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  |  65:AGGREGATE [STREAMING]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=147 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  |  in pipelines: 58(GETNEXT)
+|  |  |  |
+|  |  |  62:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=27
+|  |  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=47,48,49 row-size=40B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 58(GETNEXT), 60(OPEN)
+|  |  |  |
+|  |  |  |--F108:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=27 plan-id=28 cohort-id=09
+|  |  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  |  runtime filters: RF063[min_max] <- d3.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  170:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=49 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  |  in pipelines: 60(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F39:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  60:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=49 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 60(GETNEXT)
+|  |  |  |
+|  |  |  61:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=28
+|  |  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=47,48 row-size=32B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 58(GETNEXT), 59(OPEN)
+|  |  |  |
+|  |  |  |--F109:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=28 plan-id=29 cohort-id=09
+|  |  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  |  runtime filters: RF065[min_max] <- iws.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  169:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=48 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  |  in pipelines: 59(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F38:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  59:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=48 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 59(GETNEXT)
+|  |  |  |
+|  |  |  58:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF063[min_max] -> ws_sold_date_sk, RF065[min_max] -> ws_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=47 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 58(GETNEXT)
+|  |  |
+|  |  64:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=29
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 163(GETNEXT), 167(OPEN)
+|  |  |
+|  |  |--F110:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=29 plan-id=30 cohort-id=08
+|  |  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  168:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=146 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 167(GETNEXT)
+|  |  |  |
+|  |  |  F36:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  |  167:AGGREGATE [FINALIZE]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=146 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 167(GETNEXT), 53(OPEN)
+|  |  |  |
+|  |  |  166:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=146 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 53(GETNEXT)
+|  |  |  |
+|  |  |  F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  |  63:AGGREGATE [STREAMING]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=146 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  |  in pipelines: 53(GETNEXT)
+|  |  |  |
+|  |  |  57:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=30
+|  |  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=43,44,45 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 53(GETNEXT), 55(OPEN)
+|  |  |  |
+|  |  |  |--F111:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=30 plan-id=31 cohort-id=10
+|  |  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  |  runtime filters: RF059[min_max] <- d2.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  165:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=45 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 55(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F35:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  55:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=45 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 55(GETNEXT)
+|  |  |  |
+|  |  |  56:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=31
+|  |  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=43,44 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 53(GETNEXT), 54(OPEN)
+|  |  |  |
+|  |  |  |--F112:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=31 plan-id=32 cohort-id=10
+|  |  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  |  runtime filters: RF061[min_max] <- ics.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  164:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=44 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  |  in pipelines: 54(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F34:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  54:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=44 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 54(GETNEXT)
+|  |  |  |
+|  |  |  53:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF059[min_max] -> cs_sold_date_sk, RF061[min_max] -> cs_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=43 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 53(GETNEXT)
+|  |  |
+|  |  163:AGGREGATE [FINALIZE]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 163(GETNEXT), 47(OPEN)
+|  |  |
+|  |  162:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 47(GETNEXT)
+|  |  |
+|  |  F29:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  |  52:AGGREGATE [STREAMING]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=41 row-size=12B cardinality=148.80K cost=8641212
+|  |  |  in pipelines: 47(GETNEXT)
+|  |  |
+|  |  51:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=32
+|  |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=38,39,40 row-size=40B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 47(GETNEXT), 49(OPEN)
+|  |  |
+|  |  |--F113:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=32 plan-id=33 cohort-id=08
+|  |  |  |  build expressions: d1.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  161:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=40 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 49(GETNEXT)
+|  |  |  |
+|  |  |  F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  49:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=40 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 49(GETNEXT)
+|  |  |
+|  |  50:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=33
+|  |  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=38,39 row-size=32B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 47(GETNEXT), 48(OPEN)
+|  |  |
+|  |  |--F114:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=33 plan-id=34 cohort-id=08
+|  |  |  |  build expressions: iss.i_item_sk
+|  |  |  |  runtime filters: RF057[min_max] <- iss.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  160:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 48(GETNEXT)
+|  |  |  |
+|  |  |  F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  48:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     runtime filters: RF045[min_max] -> iss.i_brand_id, RF046[min_max] -> iss.i_category_id, RF047[min_max] -> iss.i_class_id
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=39 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 48(GETNEXT)
+|  |  |
+|  |  47:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF057[min_max] -> ss_item_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=38 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 47(GETNEXT)
+|  |
+|  69:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=34
+|  |  hash predicates: cs_item_sk = i_item_sk
+|  |  fk/pk conjuncts: cs_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=34,36,35 row-size=52B cardinality=85.31K cost=85309
+|  |  in pipelines: 43(GETNEXT), 44(OPEN)
+|  |
+|  |--F115:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  JOIN BUILD
+|  |  |  join-table-id=34 plan-id=35 cohort-id=01
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF039[min_max] <- i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  159:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=35 row-size=20B cardinality=18.00K cost=1266
+|  |  |  in pipelines: 44(GETNEXT)
+|  |  |
+|  |  F28:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  44:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF037[min_max] -> tpcds_parquet.item.i_item_sk
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=35 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 44(GETNEXT)
+|  |
+|  68:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=35
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=34,36 row-size=32B cardinality=85.31K cost=1441548
+|  |  in pipelines: 43(GETNEXT), 45(OPEN)
+|  |
+|  |--F116:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=35 plan-id=36 cohort-id=01
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF040[bloom] <- d_date_sk, RF041[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  158:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=36 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 45(GETNEXT)
+|  |  |
+|  |  F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  45:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=36 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 45(GETNEXT)
+|  |
+|  43:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     runtime filters: RF041[min_max] -> cs_sold_date_sk, RF037[min_max] -> cs_item_sk, RF039[min_max] -> cs_item_sk, RF040[bloom] -> cs_sold_date_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|     tuple-ids=34 row-size=20B cardinality=1.44M cost=28156
+|     in pipelines: 43(GETNEXT)
+|
+42:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  join table id: 00
+|  predicates: sum(ss_quantity * ss_list_price) > avg(quantity * list_price)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=20,31 row-size=52B cardinality=148.80K cost=0
+|  in pipelines: 151(GETNEXT), 156(OPEN)
+|
+|--F81:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[3]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |
+|  157:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=31 row-size=16B cardinality=1 cost=3
+|  |  in pipelines: 156(GETNEXT)
+|  |
+|  F25:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[2, 1]
+|  156:AGGREGATE [FINALIZE]
+|  |  output: avg:merge(quantity * list_price)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=31 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 156(GETNEXT), 41(OPEN)
+|  |
+|  155:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=30 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 41(GETNEXT)
+|  |
+|  F24:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[40389768, 1]
+|  41:AGGREGATE
+|  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=30 row-size=16B cardinality=1 cost=5041336
+|  |  in pipelines: 41(GETNEXT), 32(OPEN), 35(OPEN), 38(OPEN)
+|  |
+|  31:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=28 row-size=8B cardinality=5.04M cost=30248016
+|  |  in pipelines: 32(GETNEXT), 35(GETNEXT), 38(GETNEXT)
+|  |
+|  |--40:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=03
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=26,27 row-size=20B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 38(GETNEXT), 39(OPEN)
+|  |  |
+|  |  |--F84:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=03 plan-id=04 cohort-id=02
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF035[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  154:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 39(GETNEXT)
+|  |  |  |
+|  |  |  F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  39:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=27 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 39(GETNEXT)
+|  |  |
+|  |  38:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF035[min_max] -> ws_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=26 row-size=12B cardinality=719.38K cost=8431
+|  |     in pipelines: 38(GETNEXT)
+|  |
+|  |--37:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=24,25 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 35(GETNEXT), 36(OPEN)
+|  |  |
+|  |  |--F83:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=02
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF033[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  153:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=25 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 36(GETNEXT)
+|  |  |  |
+|  |  |  F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  36:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=25 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 36(GETNEXT)
+|  |  |
+|  |  35:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF033[min_max] -> cs_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=24 row-size=12B cardinality=1.44M cost=16894
+|  |     in pipelines: 35(GETNEXT)
+|  |
+|  34:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=01
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=22,23 row-size=20B cardinality=2.88M cost=2880404
+|  |  in pipelines: 32(GETNEXT), 33(OPEN)
+|  |
+|  |--F82:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |
+|  |  152:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=23 row-size=8B cardinality=7.30K cost=258
+|  |  |  in pipelines: 33(GETNEXT)
+|  |  |
+|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146755]
+|  |  33:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=23 row-size=8B cardinality=7.30K cost=146669
+|  |     in pipelines: 33(GETNEXT)
+|  |
+|  32:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=22 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 32(GETNEXT)
+|
+151:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_quantity * ss_list_price), count:merge(*)
+|  group by: i_brand_id, i_class_id, i_category_id
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=20 row-size=36B cardinality=148.80K cost=744000
+|  in pipelines: 151(GETNEXT), 01(OPEN)
+|
+150:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  mem-estimate=2.17MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=20 row-size=36B cardinality=148.80K cost=5813
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+Per-Instance Resources: mem-estimate=27.88MB mem-reservation=3.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[4130519, 5813]
+30:AGGREGATE [STREAMING]
+|  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*)
+|  group by: i_brand_id, i_class_id, i_category_id
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=20 row-size=36B cardinality=148.80K cost=852755
+|  in pipelines: 01(GETNEXT)
+|
+29:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash-table-id=12
+|  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,2,1 row-size=52B cardinality=170.55K cost=170551
+|  in pipelines: 01(GETNEXT), 148(OPEN)
+|
+|--F93:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.92MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[18608]
+|  JOIN BUILD
+|  |  join-table-id=12 plan-id=13 cohort-id=01
+|  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  runtime filters: RF000[bloom] <- tpcds_parquet.item.i_item_sk, RF001[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |
+|  149:EXCHANGE [BROADCAST]
+|  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=126 row-size=8B cardinality=17.98K cost=633
+|  |  in pipelines: 148(GETNEXT)
+|  |
+|  F16:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[18186, 211]
+|  148:AGGREGATE [FINALIZE]
+|  |  group by: tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=126 row-size=8B cardinality=17.98K cost=17975
+|  |  in pipelines: 148(GETNEXT), 135(OPEN)
+|  |
+|  147:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=126 row-size=8B cardinality=17.98K cost=211
+|  |  in pipelines: 135(GETNEXT)
+|  |
+|  F06:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  28:AGGREGATE [STREAMING]
+|  |  group by: tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=126 row-size=8B cardinality=17.98K cost=148800
+|  |  in pipelines: 135(GETNEXT)
+|  |
+|  25:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=13
+|  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=7,3 row-size=32B cardinality=148.80K cost=446400
+|  |  in pipelines: 135(GETNEXT), 04(OPEN)
+|  |
+|  |--F94:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=5.30MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=3.00MB
+|  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  JOIN BUILD
+|  |  |  join-table-id=13 plan-id=14 cohort-id=05
+|  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  runtime filters: RF006[bloom] <- i_brand_id, RF007[bloom] <- i_category_id, RF008[bloom] <- i_class_id, RF009[min_max] <- i_brand_id, RF010[min_max] <- i_category_id, RF011[min_max] <- i_class_id
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |
+|  |  146:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=20B cardinality=18.00K cost=422
+|  |  |  in pipelines: 04(GETNEXT)
+|  |  |
+|  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  04:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=3 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 04(GETNEXT)
+|  |
+|  24:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=14
+|  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 135(GETNEXT), 144(OPEN)
+|  |
+|  |--F95:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=5.76MB mem-reservation=4.88MB thread-reservation=1 runtime-filters-memory=2.00MB
+|  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  JOIN BUILD
+|  |  |  join-table-id=14 plan-id=15 cohort-id=05
+|  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  runtime filters: RF012[bloom] <- iws.i_brand_id, RF013[bloom] <- iws.i_category_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |
+|  |  145:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=125 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 144(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  144:AGGREGATE [FINALIZE]
+|  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=125 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 144(GETNEXT), 16(OPEN)
+|  |  |
+|  |  143:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=125 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 16(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  23:AGGREGATE [STREAMING]
+|  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=125 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  in pipelines: 16(GETNEXT)
+|  |  |
+|  |  20:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=15
+|  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=13,14,15 row-size=40B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 16(GETNEXT), 18(OPEN)
+|  |  |
+|  |  |--F96:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=15 plan-id=16 cohort-id=06
+|  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  runtime filters: RF027[min_max] <- d3.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  142:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=15 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  in pipelines: 18(GETNEXT)
+|  |  |  |
+|  |  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  18:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=15 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 18(GETNEXT)
+|  |  |
+|  |  19:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=16
+|  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=13,14 row-size=32B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 16(GETNEXT), 17(OPEN)
+|  |  |
+|  |  |--F97:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=16 plan-id=17 cohort-id=06
+|  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  runtime filters: RF029[min_max] <- iws.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  141:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  in pipelines: 17(GETNEXT)
+|  |  |  |
+|  |  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  17:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=14 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 17(GETNEXT)
+|  |  |
+|  |  16:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF027[min_max] -> ws_sold_date_sk, RF029[min_max] -> ws_item_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=13 row-size=12B cardinality=719.38K cost=8431
+|  |     in pipelines: 16(GETNEXT)
+|  |
+|  22:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=17
+|  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 135(GETNEXT), 139(OPEN)
+|  |
+|  |--F98:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  JOIN BUILD
+|  |  |  join-table-id=17 plan-id=18 cohort-id=05
+|  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |
+|  |  140:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=124 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 139(GETNEXT)
+|  |  |
+|  |  F10:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  139:AGGREGATE [FINALIZE]
+|  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=124 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 139(GETNEXT), 11(OPEN)
+|  |  |
+|  |  138:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=124 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 11(GETNEXT)
+|  |  |
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  21:AGGREGATE [STREAMING]
+|  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=124 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  in pipelines: 11(GETNEXT)
+|  |  |
+|  |  15:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=18
+|  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10,11 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 11(GETNEXT), 13(OPEN)
+|  |  |
+|  |  |--F99:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=18 plan-id=19 cohort-id=07
+|  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  runtime filters: RF023[min_max] <- d2.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  137:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=11 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 13(GETNEXT)
+|  |  |  |
+|  |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  13:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=11 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 13(GETNEXT)
+|  |  |
+|  |  14:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=19
+|  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 11(GETNEXT), 12(OPEN)
+|  |  |
+|  |  |--F100:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=19 plan-id=20 cohort-id=07
+|  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  runtime filters: RF025[min_max] <- ics.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  136:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=10 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 12(GETNEXT)
+|  |  |  |
+|  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  12:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=10 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 12(GETNEXT)
+|  |  |
+|  |  11:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF023[min_max] -> cs_sold_date_sk, RF025[min_max] -> cs_item_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=9 row-size=12B cardinality=1.44M cost=16894
+|  |     in pipelines: 11(GETNEXT)
+|  |
+|  135:AGGREGATE [FINALIZE]
+|  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 135(GETNEXT), 05(OPEN)
+|  |
+|  134:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=2325
+|  |  in pipelines: 05(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  10:AGGREGATE [STREAMING]
+|  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=8641212
+|  |  in pipelines: 05(GETNEXT)
+|  |
+|  09:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=20
+|  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5,6 row-size=40B cardinality=2.88M cost=2880404
+|  |  in pipelines: 05(GETNEXT), 07(OPEN)
+|  |
+|  |--F101:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  JOIN BUILD
+|  |  |  join-table-id=20 plan-id=21 cohort-id=05
+|  |  |  build expressions: d1.d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |
+|  |  133:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=6 row-size=8B cardinality=7.30K cost=258
+|  |  |  in pipelines: 07(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146755]
+|  |  07:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=6 row-size=8B cardinality=7.30K cost=146669
+|  |     in pipelines: 07(GETNEXT)
+|  |
+|  08:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=21
+|  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5 row-size=32B cardinality=2.88M cost=2880404
+|  |  in pipelines: 05(GETNEXT), 06(OPEN)
+|  |
+|  |--F102:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  JOIN BUILD
+|  |  |  join-table-id=21 plan-id=22 cohort-id=05
+|  |  |  build expressions: iss.i_item_sk
+|  |  |  runtime filters: RF021[min_max] <- iss.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  132:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=20B cardinality=18.00K cost=1266
+|  |  |  in pipelines: 06(GETNEXT)
+|  |  |
+|  |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=5.00MB mem-reservation=5.00MB thread-reservation=0 runtime-filters-memory=5.00MB
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  06:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF009[min_max] -> iss.i_brand_id, RF010[min_max] -> iss.i_category_id, RF011[min_max] -> iss.i_class_id, RF006[bloom] -> iss.i_brand_id, RF007[bloom] -> iss.i_category_id, RF008[bloom] -> iss.i_class_id, RF012[bloom] -> iss.i_brand_id, RF013[bloom] -> iss.i_category_id
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=5 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 06(GETNEXT)
+|  |
+|  05:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF021[min_max] -> ss_item_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=4 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 05(GETNEXT)
+|
+27:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=22
+|  hash predicates: ss_item_sk = i_item_sk
+|  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,2,1 row-size=52B cardinality=170.55K cost=170551
+|  in pipelines: 01(GETNEXT), 02(OPEN)
+|
+|--F103:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=9.12MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[19266]
+|  JOIN BUILD
+|  |  join-table-id=22 plan-id=23 cohort-id=01
+|  |  build expressions: i_item_sk
+|  |  runtime filters: RF002[bloom] <- i_item_sk, RF003[min_max] <- i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |
+|  131:EXCHANGE [BROADCAST]
+|  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1 row-size=20B cardinality=18.00K cost=1266
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[774]
+|  02:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=1 row-size=20B cardinality=18.00K cost=352
+|     in pipelines: 02(GETNEXT)
+|
+26:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=23
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,2 row-size=32B cardinality=170.55K cost=2880404
+|  in pipelines: 01(GETNEXT), 03(OPEN)
+|
+|--F104:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[114]
+|  JOIN BUILD
+|  |  join-table-id=23 plan-id=24 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF004[bloom] <- d_date_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |
+|  130:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=12B cardinality=108 cost=6
+|  |  in pipelines: 03(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[146957]
+|  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(11 AS INT)
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=2 row-size=12B cardinality=108 cost=146955
+|     in pipelines: 03(GETNEXT)
+|
+01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF001[min_max] -> ss_item_sk, RF003[min_max] -> ss_item_sk, RF004[bloom] -> ss_sold_date_sk, RF000[bloom] -> ss_item_sk, RF002[bloom] -> ss_item_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=0 row-size=20B cardinality=2.88M cost=56258
+   in pipelines: 01(GETNEXT)
+====
+# TPCDS-Q14b second of two queries
+WITH cross_items AS
+
+  (SELECT i_item_sk ss_item_sk
+   FROM item,
+     (SELECT iss.i_brand_id brand_id,
+             iss.i_class_id class_id,
+             iss.i_category_id category_id
+      FROM store_sales,
+           item iss,
+           date_dim d1
+      WHERE ss_item_sk = iss.i_item_sk
+        AND ss_sold_date_sk = d1.d_date_sk
+        AND d1.d_year BETWEEN 1999 AND 1999 + 2 INTERSECT
+        SELECT ics.i_brand_id,
+               ics.i_class_id,
+               ics.i_category_id
+        FROM catalog_sales,
+             item ics,
+             date_dim d2 WHERE cs_item_sk = ics.i_item_sk
+        AND cs_sold_date_sk = d2.d_date_sk
+        AND d2.d_year BETWEEN 1999 AND 1999 + 2 INTERSECT
+        SELECT iws.i_brand_id,
+               iws.i_class_id,
+               iws.i_category_id
+        FROM web_sales,
+             item iws,
+             date_dim d3 WHERE ws_item_sk = iws.i_item_sk
+        AND ws_sold_date_sk = d3.d_date_sk
+        AND d3.d_year BETWEEN 1999 AND 1999 + 2) x
+   WHERE i_brand_id = brand_id
+     AND i_class_id = class_id
+     AND i_category_id = category_id ),
+     avg_sales AS
+
+  (SELECT avg(quantity*list_price) average_sales
+   FROM
+     (SELECT ss_quantity quantity,
+             ss_list_price list_price
+      FROM store_sales,
+           date_dim
+      WHERE ss_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2
+      UNION ALL SELECT cs_quantity quantity,
+                       cs_list_price list_price
+      FROM catalog_sales,
+           date_dim
+      WHERE cs_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2
+      UNION ALL SELECT ws_quantity quantity,
+                       ws_list_price list_price
+      FROM web_sales,
+           date_dim
+      WHERE ws_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2) x)
+SELECT this_year.channel ty_channel,
+       this_year.i_brand_id ty_brand,
+       this_year.i_class_id ty_class,
+       this_year.i_category_id ty_category,
+       this_year.sales ty_sales,
+       this_year.number_sales ty_number_sales,
+       last_year.channel ly_channel,
+       last_year.i_brand_id ly_brand,
+       last_year.i_class_id ly_class,
+       last_year.i_category_id ly_category,
+       last_year.sales ly_sales,
+       last_year.number_sales ly_number_sales
+FROM
+  (SELECT 'store' channel,
+                  i_brand_id,
+                  i_class_id,
+                  i_category_id,
+                  sum(ss_quantity*ss_list_price) sales,
+                  count(*) number_sales
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_item_sk IN
+
+       (SELECT ss_item_sk
+        FROM cross_items)
+     AND ss_item_sk = i_item_sk
+     AND ss_sold_date_sk = d_date_sk
+     AND d_week_seq =
+       (SELECT d_week_seq
+        FROM date_dim
+        WHERE d_year = 1999 + 1
+          AND d_moy = 12
+          AND d_dom = 11)
+   GROUP BY i_brand_id,
+            i_class_id,
+            i_category_id
+   HAVING sum(ss_quantity*ss_list_price) >
+     (SELECT average_sales
+      FROM avg_sales)) this_year,
+  (SELECT 'store' channel,
+                  i_brand_id,
+                  i_class_id,
+                  i_category_id,
+                  sum(ss_quantity*ss_list_price) sales,
+                  count(*) number_sales
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_item_sk IN
+
+       (SELECT ss_item_sk
+        FROM cross_items)
+     AND ss_item_sk = i_item_sk
+     AND ss_sold_date_sk = d_date_sk
+     AND d_week_seq =
+       (SELECT d_week_seq
+        FROM date_dim
+        WHERE d_year = 1999
+          AND d_moy = 12
+          AND d_dom = 11)
+   GROUP BY i_brand_id,
+            i_class_id,
+            i_category_id
+   HAVING sum(ss_quantity*ss_list_price) >
+     (SELECT average_sales
+      FROM avg_sales)) last_year
+WHERE this_year.i_brand_id= last_year.i_brand_id
+  AND this_year.i_class_id = last_year.i_class_id
+  AND this_year.i_category_id = last_year.i_category_id
+ORDER BY this_year.channel,
+         this_year.i_brand_id,
+         this_year.i_class_id,
+         this_year.i_category_id
+LIMIT 100
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=441.81MB Threads=103
+Per-Host Resource Estimates: Memory=1.78GB
+F56:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[61]
+PLAN-ROOT SINK
+|  output exprs: channel, i_brand_id, i_class_id, i_category_id, sales, number_sales, channel, i_brand_id, i_class_id, i_category_id, sales, number_sales
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=60
+|
+152:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: channel ASC, i_brand_id ASC, i_class_id ASC, i_category_id ASC
+|  limit: 100
+|  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=74 row-size=96B cardinality=5 cost=1
+|  in pipelines: 91(GETNEXT)
+|
+F19:PLAN FRAGMENT [HASH(i_brand_id,i_class_id,i_category_id)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=10.13MB mem-reservation=1.94MB thread-reservation=1
+max-parallelism=3 fragment-costs=[1366, 833, 1]
+91:TOP-N [LIMIT=100]
+|  order by: channel ASC, i_brand_id ASC, i_class_id ASC, i_category_id ASC
+|  mem-estimate=480B mem-reservation=0B thread-reservation=0
+|  tuple-ids=74 row-size=96B cardinality=5 cost=20
+|  in pipelines: 91(GETNEXT), 115(OPEN)
+|
+90:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: i_brand_id = i_brand_id, i_category_id = i_category_id, i_class_id = i_class_id
+|  fk/pk conjuncts: i_brand_id = i_brand_id, i_category_id = i_category_id, i_class_id = i_class_id
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=22,33,59,70 row-size=104B cardinality=5 cost=813
+|  in pipelines: 115(GETNEXT), 145(OPEN)
+|
+|--F57:PLAN FRAGMENT [HASH(i_brand_id,i_class_id,i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=14.94MB mem-reservation=6.88MB thread-reservation=1 runtime-filters-memory=3.00MB
+|  |  max-parallelism=3 fragment-costs=[1366, 813]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  runtime filters: RF000[bloom] <- i_brand_id, RF001[bloom] <- i_category_id, RF002[bloom] <- i_class_id, RF003[min_max] <- i_brand_id, RF004[min_max] <- i_category_id, RF005[min_max] <- i_class_id
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=813
+|  |
+|  89:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  |  join table id: 01
+|  |  predicates: sum(ss_quantity * ss_list_price) > avg(quantity * list_price)
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=59,70 row-size=52B cardinality=271 cost=0
+|  |  in pipelines: 145(GETNEXT), 150(OPEN)
+|  |
+|  |--F58:PLAN FRAGMENT [HASH(i_brand_id,i_class_id,i_category_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |
+|  |  151:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=70 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 150(GETNEXT)
+|  |  |
+|  |  F55:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  150:AGGREGATE [FINALIZE]
+|  |  |  output: avg:merge(quantity * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=70 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 150(GETNEXT), 88(OPEN)
+|  |  |
+|  |  149:EXCHANGE [UNPARTITIONED]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=69 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 88(GETNEXT)
+|  |  |
+|  |  F54:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[40389768, 1]
+|  |  88:AGGREGATE
+|  |  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=69 row-size=16B cardinality=1 cost=5041336
+|  |  |  in pipelines: 88(GETNEXT), 79(OPEN), 82(OPEN), 85(OPEN)
+|  |  |
+|  |  78:UNION
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=67 row-size=8B cardinality=5.04M cost=30248016
+|  |  |  in pipelines: 79(GETNEXT), 82(GETNEXT), 85(GETNEXT)
+|  |  |
+|  |  |--87:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=04
+|  |  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=65,66 row-size=20B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 85(GETNEXT), 86(OPEN)
+|  |  |  |
+|  |  |  |--F61:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=04 plan-id=05 cohort-id=03
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF081[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  148:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=66 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 86(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F53:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  86:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=66 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 86(GETNEXT)
+|  |  |  |
+|  |  |  85:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF081[min_max] -> ws_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=65 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 85(GETNEXT)
+|  |  |
+|  |  |--84:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=03
+|  |  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=63,64 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 82(GETNEXT), 83(OPEN)
+|  |  |  |
+|  |  |  |--F60:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF079[min_max] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  147:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=64 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 83(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F51:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  83:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=64 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 83(GETNEXT)
+|  |  |  |
+|  |  |  82:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF079[min_max] -> cs_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=63 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 82(GETNEXT)
+|  |  |
+|  |  81:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=61,62 row-size=20B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 79(GETNEXT), 80(OPEN)
+|  |  |
+|  |  |--F59:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  146:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=62 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 80(GETNEXT)
+|  |  |  |
+|  |  |  F49:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  80:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=62 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 80(GETNEXT)
+|  |  |
+|  |  79:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=61 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 79(GETNEXT)
+|  |
+|  145:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ss_quantity * ss_list_price), count:merge(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=59 row-size=36B cardinality=271 cost=1355
+|  |  in pipelines: 145(GETNEXT), 45(OPEN)
+|  |
+|  144:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  mem-estimate=130.21KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=59 row-size=36B cardinality=271 cost=11
+|  |  in pipelines: 45(GETNEXT)
+|  |
+|  F28:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=26.47MB mem-reservation=3.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[11579229, 11]
+|  77:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*)
+|  |  group by: i_brand_id, i_class_id, i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=59 row-size=36B cardinality=271 cost=1355
+|  |  in pipelines: 45(GETNEXT)
+|  |
+|  76:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=05
+|  |  hash predicates: d_week_seq = d_week_seq
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=37,38,39 row-size=48B cardinality=271 cost=2880404
+|  |  in pipelines: 45(GETNEXT), 71(OPEN)
+|  |
+|  |--F62:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[4]
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=02
+|  |  |  build expressions: d_week_seq
+|  |  |  runtime filters: RF044[bloom] <- d_week_seq, RF045[min_max] <- d_week_seq
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1
+|  |  |
+|  |  143:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 71(GETNEXT)
+|  |  |
+|  |  F46:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=80.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1, 1]
+|  |  71:CARDINALITY CHECK
+|  |  |  limit: 1
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=16B cardinality=1 cost=0
+|  |  |  in pipelines: 71(GETNEXT), 70(OPEN)
+|  |  |
+|  |  142:EXCHANGE [UNPARTITIONED]
+|  |  |  limit: 2
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=16B cardinality=2 cost=1
+|  |  |  in pipelines: 70(GETNEXT)
+|  |  |
+|  |  F45:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[220290]
+|  |  70:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(1999 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(1999 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(1999 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|  |     limit: 2
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=57 row-size=16B cardinality=2 cost=220289
+|  |     in pipelines: 70(GETNEXT)
+|  |
+|  75:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=06
+|  |  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=37,38,39 row-size=48B cardinality=2.88M cost=2880404
+|  |  in pipelines: 45(GETNEXT), 140(OPEN)
+|  |
+|  |--F63:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[18608]
+|  |  JOIN BUILD
+|  |  |  join-table-id=06 plan-id=07 cohort-id=02
+|  |  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  |  runtime filters: RF047[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |
+|  |  141:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=114 row-size=8B cardinality=17.98K cost=633
+|  |  |  in pipelines: 140(GETNEXT)
+|  |  |
+|  |  F44:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  140:AGGREGATE [FINALIZE]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=114 row-size=8B cardinality=17.98K cost=17975
+|  |  |  in pipelines: 140(GETNEXT), 127(OPEN)
+|  |  |
+|  |  139:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=114 row-size=8B cardinality=17.98K cost=211
+|  |  |  in pipelines: 127(GETNEXT)
+|  |  |
+|  |  F34:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  |  74:AGGREGATE [STREAMING]
+|  |  |  group by: tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=114 row-size=8B cardinality=17.98K cost=148800
+|  |  |  in pipelines: 127(GETNEXT)
+|  |  |
+|  |  69:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=07
+|  |  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=44,40 row-size=32B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 127(GETNEXT), 48(OPEN)
+|  |  |
+|  |  |--F64:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=2.30MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=07 plan-id=08 cohort-id=04
+|  |  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  |  runtime filters: RF055[min_max] <- i_brand_id, RF056[min_max] <- i_category_id, RF057[min_max] <- i_class_id
+|  |  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |  |
+|  |  |  138:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=40 row-size=20B cardinality=18.00K cost=422
+|  |  |  |  in pipelines: 48(GETNEXT)
+|  |  |  |
+|  |  |  F43:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  48:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=40 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 48(GETNEXT)
+|  |  |
+|  |  68:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=08
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 127(GETNEXT), 136(OPEN)
+|  |  |
+|  |  |--F65:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=08 plan-id=09 cohort-id=04
+|  |  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  137:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=112 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 136(GETNEXT)
+|  |  |  |
+|  |  |  F42:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  |  136:AGGREGATE [FINALIZE]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=112 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 136(GETNEXT), 60(OPEN)
+|  |  |  |
+|  |  |  135:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=112 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 60(GETNEXT)
+|  |  |  |
+|  |  |  F39:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  |  67:AGGREGATE [STREAMING]
+|  |  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=112 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  |  in pipelines: 60(GETNEXT)
+|  |  |  |
+|  |  |  64:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=09
+|  |  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=50,51,52 row-size=40B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 60(GETNEXT), 62(OPEN)
+|  |  |  |
+|  |  |  |--F66:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=09 plan-id=10 cohort-id=05
+|  |  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  |  runtime filters: RF073[min_max] <- d3.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  134:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=52 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  |  in pipelines: 62(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F41:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  62:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=52 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 62(GETNEXT)
+|  |  |  |
+|  |  |  63:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=10
+|  |  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=50,51 row-size=32B cardinality=719.38K cost=719384
+|  |  |  |  in pipelines: 60(GETNEXT), 61(OPEN)
+|  |  |  |
+|  |  |  |--F67:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=05
+|  |  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  |  runtime filters: RF075[min_max] <- iws.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  133:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=51 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  |  in pipelines: 61(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F40:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  61:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=51 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 61(GETNEXT)
+|  |  |  |
+|  |  |  60:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     runtime filters: RF073[min_max] -> ws_sold_date_sk, RF075[min_max] -> ws_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=50 row-size=12B cardinality=719.38K cost=8431
+|  |  |     in pipelines: 60(GETNEXT)
+|  |  |
+|  |  66:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  |  hash-table-id=11
+|  |  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 127(GETNEXT), 131(OPEN)
+|  |  |
+|  |  |--F68:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=11 plan-id=12 cohort-id=04
+|  |  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |  |
+|  |  |  132:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=111 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 131(GETNEXT)
+|  |  |  |
+|  |  |  F38:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  |  131:AGGREGATE [FINALIZE]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=111 row-size=12B cardinality=148.80K cost=446400
+|  |  |  |  in pipelines: 131(GETNEXT), 55(OPEN)
+|  |  |  |
+|  |  |  130:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=111 row-size=12B cardinality=148.80K cost=2325
+|  |  |  |  in pipelines: 55(GETNEXT)
+|  |  |  |
+|  |  |  F35:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  |  65:AGGREGATE [STREAMING]
+|  |  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=111 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  |  in pipelines: 55(GETNEXT)
+|  |  |  |
+|  |  |  59:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=12
+|  |  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=46,47,48 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 55(GETNEXT), 57(OPEN)
+|  |  |  |
+|  |  |  |--F69:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=12 plan-id=13 cohort-id=06
+|  |  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  |  runtime filters: RF069[min_max] <- d2.d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |  |
+|  |  |  |  129:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=48 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  |  in pipelines: 57(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F37:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  |  57:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=48 row-size=8B cardinality=7.30K cost=146669
+|  |  |  |     in pipelines: 57(GETNEXT)
+|  |  |  |
+|  |  |  58:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=13
+|  |  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=46,47 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  |  in pipelines: 55(GETNEXT), 56(OPEN)
+|  |  |  |
+|  |  |  |--F70:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=13 plan-id=14 cohort-id=06
+|  |  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  |  runtime filters: RF071[min_max] <- ics.i_item_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  128:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=47 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  |  in pipelines: 56(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F36:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  |  56:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |  |     tuple-ids=47 row-size=20B cardinality=18.00K cost=352
+|  |  |  |     in pipelines: 56(GETNEXT)
+|  |  |  |
+|  |  |  55:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |  |     runtime filters: RF069[min_max] -> cs_sold_date_sk, RF071[min_max] -> cs_item_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=1.44M size=96.62MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=46 row-size=12B cardinality=1.44M cost=16894
+|  |  |     in pipelines: 55(GETNEXT)
+|  |  |
+|  |  127:AGGREGATE [FINALIZE]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 127(GETNEXT), 49(OPEN)
+|  |  |
+|  |  126:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 49(GETNEXT)
+|  |  |
+|  |  F31:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  |  54:AGGREGATE [STREAMING]
+|  |  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=44 row-size=12B cardinality=148.80K cost=8641212
+|  |  |  in pipelines: 49(GETNEXT)
+|  |  |
+|  |  53:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=14
+|  |  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=41,42,43 row-size=40B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 49(GETNEXT), 51(OPEN)
+|  |  |
+|  |  |--F71:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=14 plan-id=15 cohort-id=04
+|  |  |  |  build expressions: d1.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  125:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=43 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 51(GETNEXT)
+|  |  |  |
+|  |  |  F33:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  51:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=43 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 51(GETNEXT)
+|  |  |
+|  |  52:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=15
+|  |  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=41,42 row-size=32B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 49(GETNEXT), 50(OPEN)
+|  |  |
+|  |  |--F72:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=15 plan-id=16 cohort-id=04
+|  |  |  |  build expressions: iss.i_item_sk
+|  |  |  |  runtime filters: RF067[min_max] <- iss.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  124:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=42 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 50(GETNEXT)
+|  |  |  |
+|  |  |  F32:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  50:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     runtime filters: RF055[min_max] -> iss.i_brand_id, RF056[min_max] -> iss.i_category_id, RF057[min_max] -> iss.i_class_id
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=42 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 50(GETNEXT)
+|  |  |
+|  |  49:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF067[min_max] -> ss_item_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=41 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 49(GETNEXT)
+|  |
+|  73:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=16
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=37,38,39 row-size=48B cardinality=2.88M cost=2880404
+|  |  in pipelines: 45(GETNEXT), 47(OPEN)
+|  |
+|  |--F73:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=19.57MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[75620]
+|  |  JOIN BUILD
+|  |  |  join-table-id=16 plan-id=17 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=73049
+|  |  |
+|  |  123:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=582.70KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=39 row-size=8B cardinality=73.05K cost=2571
+|  |  |  in pipelines: 47(GETNEXT)
+|  |  |
+|  |  F30:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1428]
+|  |  47:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     runtime filters: RF045[min_max] -> d_week_seq, RF044[bloom] -> d_week_seq
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=39 row-size=8B cardinality=73.05K cost=571
+|  |     in pipelines: 47(GETNEXT)
+|  |
+|  72:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=17
+|  |  hash predicates: ss_item_sk = i_item_sk
+|  |  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=37,38 row-size=40B cardinality=2.88M cost=2880404
+|  |  in pipelines: 45(GETNEXT), 46(OPEN)
+|  |
+|  |--F74:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  JOIN BUILD
+|  |  |  join-table-id=17 plan-id=18 cohort-id=02
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF051[min_max] <- i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  122:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=38 row-size=20B cardinality=18.00K cost=1266
+|  |  |  in pipelines: 46(GETNEXT)
+|  |  |
+|  |  F29:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  46:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF047[min_max] -> tpcds_parquet.item.i_item_sk
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=38 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 46(GETNEXT)
+|  |
+|  45:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF047[min_max] -> ss_item_sk, RF051[min_max] -> ss_item_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|     tuple-ids=37 row-size=20B cardinality=2.88M cost=56258
+|     in pipelines: 45(GETNEXT)
+|
+44:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  join table id: 18
+|  predicates: sum(ss_quantity * ss_list_price) > avg(quantity * list_price)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=22,33 row-size=52B cardinality=271 cost=0
+|  in pipelines: 115(GETNEXT), 120(OPEN)
+|
+|--F75:PLAN FRAGMENT [HASH(i_brand_id,i_class_id,i_category_id)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[3]
+|  JOIN BUILD
+|  |  join-table-id=18 plan-id=19 cohort-id=01
+|  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |
+|  121:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=33 row-size=16B cardinality=1 cost=3
+|  |  in pipelines: 120(GETNEXT)
+|  |
+|  F27:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[2, 1]
+|  120:AGGREGATE [FINALIZE]
+|  |  output: avg:merge(quantity * list_price)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=33 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 120(GETNEXT), 43(OPEN)
+|  |
+|  119:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=32 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 43(GETNEXT)
+|  |
+|  F26:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=48.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[40389768, 1]
+|  43:AGGREGATE
+|  |  output: avg(CAST(quantity AS DECIMAL(10,0)) * list_price)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=32 row-size=16B cardinality=1 cost=5041336
+|  |  in pipelines: 43(GETNEXT), 34(OPEN), 37(OPEN), 40(OPEN)
+|  |
+|  33:UNION
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=30 row-size=8B cardinality=5.04M cost=30248016
+|  |  in pipelines: 34(GETNEXT), 37(GETNEXT), 40(GETNEXT)
+|  |
+|  |--42:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=21
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=28,29 row-size=20B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 40(GETNEXT), 41(OPEN)
+|  |  |
+|  |  |--F78:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=21 plan-id=22 cohort-id=07
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF043[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  118:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=29 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 41(GETNEXT)
+|  |  |  |
+|  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  41:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=29 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 41(GETNEXT)
+|  |  |
+|  |  40:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF043[min_max] -> ws_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=28 row-size=12B cardinality=719.38K cost=8431
+|  |     in pipelines: 40(GETNEXT)
+|  |
+|  |--39:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=20
+|  |  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=26,27 row-size=20B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 37(GETNEXT), 38(OPEN)
+|  |  |
+|  |  |--F77:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=20 plan-id=21 cohort-id=07
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF041[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  117:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 38(GETNEXT)
+|  |  |  |
+|  |  |  F23:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  38:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=27 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 38(GETNEXT)
+|  |  |
+|  |  37:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF041[min_max] -> cs_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=26 row-size=12B cardinality=1.44M cost=16894
+|  |     in pipelines: 37(GETNEXT)
+|  |
+|  36:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=19
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=24,25 row-size=20B cardinality=2.88M cost=2880404
+|  |  in pipelines: 34(GETNEXT), 35(OPEN)
+|  |
+|  |--F76:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  JOIN BUILD
+|  |  |  join-table-id=19 plan-id=20 cohort-id=07
+|  |  |  build expressions: d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |
+|  |  116:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=25 row-size=8B cardinality=7.30K cost=258
+|  |  |  in pipelines: 35(GETNEXT)
+|  |  |
+|  |  F21:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146755]
+|  |  35:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     parquet dictionary predicates: d_year <= CAST(2001 AS INT), d_year >= CAST(1999 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=25 row-size=8B cardinality=7.30K cost=146669
+|  |     in pipelines: 35(GETNEXT)
+|  |
+|  34:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=24 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 34(GETNEXT)
+|
+115:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_quantity * ss_list_price), count:merge(*)
+|  group by: i_brand_id, i_class_id, i_category_id
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=22 row-size=36B cardinality=271 cost=1355
+|  in pipelines: 115(GETNEXT), 00(OPEN)
+|
+114:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  mem-estimate=130.21KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=22 row-size=36B cardinality=271 cost=11
+|  in pipelines: 00(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+Per-Instance Resources: mem-estimate=26.47MB mem-reservation=3.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[11579229, 11]
+32:AGGREGATE [STREAMING]
+|  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_list_price), count(*)
+|  group by: i_brand_id, i_class_id, i_category_id
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=22 row-size=36B cardinality=271 cost=1355
+|  in pipelines: 00(GETNEXT)
+|
+31:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash-table-id=22
+|  hash predicates: d_week_seq = d_week_seq
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1,2 row-size=48B cardinality=271 cost=2880404
+|  in pipelines: 00(GETNEXT), 26(OPEN)
+|
+|--F79:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[4]
+|  JOIN BUILD
+|  |  join-table-id=22 plan-id=23 cohort-id=01
+|  |  build expressions: d_week_seq
+|  |  runtime filters: RF006[bloom] <- d_week_seq, RF007[min_max] <- d_week_seq
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1
+|  |
+|  113:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=20 row-size=16B cardinality=1 cost=3
+|  |  in pipelines: 26(GETNEXT)
+|  |
+|  F18:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=80.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1, 1]
+|  26:CARDINALITY CHECK
+|  |  limit: 1
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=20 row-size=16B cardinality=1 cost=0
+|  |  in pipelines: 26(GETNEXT), 25(OPEN)
+|  |
+|  112:EXCHANGE [UNPARTITIONED]
+|  |  limit: 2
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=20 row-size=16B cardinality=2 cost=1
+|  |  in pipelines: 25(GETNEXT)
+|  |
+|  F17:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[220290]
+|  25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2000 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2000 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_dom = CAST(11 AS INT), d_moy = CAST(12 AS INT)
+|     limit: 2
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=20 row-size=16B cardinality=2 cost=220289
+|     in pipelines: 25(GETNEXT)
+|
+30:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash-table-id=23
+|  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1,2 row-size=48B cardinality=2.88M cost=2880404
+|  in pipelines: 00(GETNEXT), 110(OPEN)
+|
+|--F80:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.92MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[18608]
+|  JOIN BUILD
+|  |  join-table-id=23 plan-id=24 cohort-id=01
+|  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  runtime filters: RF008[bloom] <- tpcds_parquet.item.i_item_sk, RF009[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |
+|  111:EXCHANGE [BROADCAST]
+|  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=91 row-size=8B cardinality=17.98K cost=633
+|  |  in pipelines: 110(GETNEXT)
+|  |
+|  F16:PLAN FRAGMENT [HASH(tpcds_parquet.item.i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[18186, 211]
+|  110:AGGREGATE [FINALIZE]
+|  |  group by: tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=91 row-size=8B cardinality=17.98K cost=17975
+|  |  in pipelines: 110(GETNEXT), 97(OPEN)
+|  |
+|  109:EXCHANGE [HASH(tpcds_parquet.item.i_item_sk)]
+|  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=91 row-size=8B cardinality=17.98K cost=211
+|  |  in pipelines: 97(GETNEXT)
+|  |
+|  F06:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=20.76MB mem-reservation=4.88MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[448725, 1488000, 211]
+|  29:AGGREGATE [STREAMING]
+|  |  group by: tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=91 row-size=8B cardinality=17.98K cost=148800
+|  |  in pipelines: 97(GETNEXT)
+|  |
+|  24:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=24
+|  |  hash predicates: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  fk/pk conjuncts: iss.i_brand_id = i_brand_id, iss.i_category_id = i_category_id, iss.i_class_id = i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=7,3 row-size=32B cardinality=148.80K cost=446400
+|  |  in pipelines: 97(GETNEXT), 03(OPEN)
+|  |
+|  |--F81:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=4.30MB mem-reservation=3.94MB thread-reservation=1 runtime-filters-memory=2.00MB
+|  |  |  max-parallelism=3 fragment-costs=[54422]
+|  |  JOIN BUILD
+|  |  |  join-table-id=24 plan-id=25 cohort-id=08
+|  |  |  build expressions: i_brand_id, i_category_id, i_class_id
+|  |  |  runtime filters: RF014[bloom] <- i_brand_id, RF015[bloom] <- i_category_id, RF017[min_max] <- i_brand_id, RF018[min_max] <- i_category_id, RF019[min_max] <- i_class_id
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=54000
+|  |  |
+|  |  108:EXCHANGE [HASH(i_brand_id,i_class_id,i_category_id)]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=20B cardinality=18.00K cost=422
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  F15:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.28MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  03:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=3 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 03(GETNEXT)
+|  |
+|  23:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=25
+|  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM iws.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM iws.i_category_id, iss.i_class_id IS NOT DISTINCT FROM iws.i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 97(GETNEXT), 106(OPEN)
+|  |
+|  |--F82:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  JOIN BUILD
+|  |  |  join-table-id=25 plan-id=26 cohort-id=08
+|  |  |  build expressions: iws.i_brand_id, iws.i_category_id, iws.i_class_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |
+|  |  107:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=89 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 106(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=10.88MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[448725, 2325]
+|  |  106:AGGREGATE [FINALIZE]
+|  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=89 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 106(GETNEXT), 15(OPEN)
+|  |  |
+|  |  105:EXCHANGE [HASH(iws.i_brand_id,iws.i_class_id,iws.i_category_id)]
+|  |  |  mem-estimate=903.88KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=89 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 15(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=42.12MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[3605351, 2325]
+|  |  22:AGGREGATE [STREAMING]
+|  |  |  group by: iws.i_brand_id, iws.i_class_id, iws.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=89 row-size=12B cardinality=148.80K cost=2158152
+|  |  |  in pipelines: 15(GETNEXT)
+|  |  |
+|  |  19:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=26
+|  |  |  hash predicates: ws_sold_date_sk = d3.d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d3.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=13,14,15 row-size=40B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 15(GETNEXT), 17(OPEN)
+|  |  |
+|  |  |--F83:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[7477]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=26 plan-id=27 cohort-id=09
+|  |  |  |  build expressions: d3.d_date_sk
+|  |  |  |  runtime filters: RF035[min_max] <- d3.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  104:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=15 row-size=8B cardinality=7.30K cost=172
+|  |  |  |  in pipelines: 17(GETNEXT)
+|  |  |  |
+|  |  |  F13:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  17:SCAN HDFS [tpcds_parquet.date_dim d3, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d3.d_year <= CAST(2001 AS INT), d3.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=15 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 17(GETNEXT)
+|  |  |
+|  |  18:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=27
+|  |  |  hash predicates: ws_item_sk = iws.i_item_sk
+|  |  |  fk/pk conjuncts: ws_item_sk = iws.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=13,14 row-size=32B cardinality=719.38K cost=719384
+|  |  |  in pipelines: 15(GETNEXT), 16(OPEN)
+|  |  |
+|  |  |--F84:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[18844]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=27 plan-id=28 cohort-id=09
+|  |  |  |  build expressions: iws.i_item_sk
+|  |  |  |  runtime filters: RF037[min_max] <- iws.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  103:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=18.00K cost=844
+|  |  |  |  in pipelines: 16(GETNEXT)
+|  |  |  |
+|  |  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  16:SCAN HDFS [tpcds_parquet.item iws, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=14 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 16(GETNEXT)
+|  |  |
+|  |  15:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF035[min_max] -> ws_sold_date_sk, RF037[min_max] -> ws_item_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=13 row-size=12B cardinality=719.38K cost=8431
+|  |     in pipelines: 15(GETNEXT)
+|  |
+|  21:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=28
+|  |  hash predicates: iss.i_brand_id IS NOT DISTINCT FROM ics.i_brand_id, iss.i_category_id IS NOT DISTINCT FROM ics.i_category_id, iss.i_class_id IS NOT DISTINCT FROM ics.i_class_id
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 97(GETNEXT), 101(OPEN)
+|  |
+|  |--F85:PLAN FRAGMENT [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=3.49MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[448725]
+|  |  JOIN BUILD
+|  |  |  join-table-id=28 plan-id=29 cohort-id=08
+|  |  |  build expressions: ics.i_brand_id, ics.i_category_id, ics.i_class_id
+|  |  |  mem-estimate=2.88MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0 cost=446400
+|  |  |
+|  |  102:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=88 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 101(GETNEXT)
+|  |  |
+|  |  F10:PLAN FRAGMENT [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=10.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[448725, 2325]
+|  |  101:AGGREGATE [FINALIZE]
+|  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=88 row-size=12B cardinality=148.80K cost=446400
+|  |  |  in pipelines: 101(GETNEXT), 10(OPEN)
+|  |  |
+|  |  100:EXCHANGE [HASH(ics.i_brand_id,ics.i_class_id,ics.i_category_id)]
+|  |  |  mem-estimate=629.25KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=88 row-size=12B cardinality=148.80K cost=2325
+|  |  |  in pipelines: 10(GETNEXT)
+|  |  |
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=58.19MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[7224634, 2325]
+|  |  20:AGGREGATE [STREAMING]
+|  |  |  group by: ics.i_brand_id, ics.i_class_id, ics.i_category_id
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=88 row-size=12B cardinality=148.80K cost=4324644
+|  |  |  in pipelines: 10(GETNEXT)
+|  |  |
+|  |  14:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=29
+|  |  |  hash predicates: cs_sold_date_sk = d2.d_date_sk
+|  |  |  fk/pk conjuncts: cs_sold_date_sk = d2.d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10,11 row-size=40B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 10(GETNEXT), 12(OPEN)
+|  |  |
+|  |  |--F86:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=29 plan-id=30 cohort-id=10
+|  |  |  |  build expressions: d2.d_date_sk
+|  |  |  |  runtime filters: RF031[min_max] <- d2.d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |  |
+|  |  |  99:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=11 row-size=8B cardinality=7.30K cost=258
+|  |  |  |  in pipelines: 12(GETNEXT)
+|  |  |  |
+|  |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146755]
+|  |  |  12:SCAN HDFS [tpcds_parquet.date_dim d2, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     parquet dictionary predicates: d2.d_year <= CAST(2001 AS INT), d2.d_year >= CAST(1999 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=11 row-size=8B cardinality=7.30K cost=146669
+|  |  |     in pipelines: 12(GETNEXT)
+|  |  |
+|  |  13:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=30
+|  |  |  hash predicates: cs_item_sk = ics.i_item_sk
+|  |  |  fk/pk conjuncts: cs_item_sk = ics.i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10 row-size=32B cardinality=1.44M cost=1441548
+|  |  |  in pipelines: 10(GETNEXT), 11(OPEN)
+|  |  |
+|  |  |--F87:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=30 plan-id=31 cohort-id=10
+|  |  |  |  build expressions: ics.i_item_sk
+|  |  |  |  runtime filters: RF033[min_max] <- ics.i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  98:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=10 row-size=20B cardinality=18.00K cost=1266
+|  |  |  |  in pipelines: 11(GETNEXT)
+|  |  |  |
+|  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[774]
+|  |  |  11:SCAN HDFS [tpcds_parquet.item ics, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |  |     tuple-ids=10 row-size=20B cardinality=18.00K cost=352
+|  |  |     in pipelines: 11(GETNEXT)
+|  |  |
+|  |  10:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=3 size=96.62MB
+|  |     runtime filters: RF031[min_max] -> cs_sold_date_sk, RF033[min_max] -> cs_item_sk
+|  |     stored statistics:
+|  |       table: rows=1.44M size=96.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|  |     mem-estimate=48.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=9 row-size=12B cardinality=1.44M cost=16894
+|  |     in pipelines: 10(GETNEXT)
+|  |
+|  97:AGGREGATE [FINALIZE]
+|  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=446400
+|  |  in pipelines: 97(GETNEXT), 04(OPEN)
+|  |
+|  96:EXCHANGE [HASH(iss.i_brand_id,iss.i_class_id,iss.i_category_id)]
+|  |  mem-estimate=773.25KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=2325
+|  |  in pipelines: 04(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Instance Resources: mem-estimate=26.19MB mem-reservation=5.50MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[14435775, 2325]
+|  09:AGGREGATE [STREAMING]
+|  |  group by: iss.i_brand_id, iss.i_class_id, iss.i_category_id
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=7 row-size=12B cardinality=148.80K cost=8641212
+|  |  in pipelines: 04(GETNEXT)
+|  |
+|  08:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=31
+|  |  hash predicates: ss_sold_date_sk = d1.d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d1.d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5,6 row-size=40B cardinality=2.88M cost=2880404
+|  |  in pipelines: 04(GETNEXT), 06(OPEN)
+|  |
+|  |--F88:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.82MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[7563]
+|  |  JOIN BUILD
+|  |  |  join-table-id=31 plan-id=32 cohort-id=08
+|  |  |  build expressions: d1.d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=7305
+|  |  |
+|  |  95:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=69.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=6 row-size=8B cardinality=7.30K cost=258
+|  |  |  in pipelines: 06(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146755]
+|  |  06:SCAN HDFS [tpcds_parquet.date_dim d1, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     parquet dictionary predicates: d1.d_year <= CAST(2001 AS INT), d1.d_year >= CAST(1999 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=6 row-size=8B cardinality=7.30K cost=146669
+|  |     in pipelines: 06(GETNEXT)
+|  |
+|  07:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=32
+|  |  hash predicates: ss_item_sk = iss.i_item_sk
+|  |  fk/pk conjuncts: ss_item_sk = iss.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5 row-size=32B cardinality=2.88M cost=2880404
+|  |  in pipelines: 04(GETNEXT), 05(OPEN)
+|  |
+|  |--F89:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.12MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[19266]
+|  |  JOIN BUILD
+|  |  |  join-table-id=32 plan-id=33 cohort-id=08
+|  |  |  build expressions: iss.i_item_sk
+|  |  |  runtime filters: RF029[min_max] <- iss.i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  94:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=20B cardinality=18.00K cost=1266
+|  |  |  in pipelines: 05(GETNEXT)
+|  |  |
+|  |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  |  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[774]
+|  |  05:SCAN HDFS [tpcds_parquet.item iss, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF017[min_max] -> iss.i_brand_id, RF018[min_max] -> iss.i_category_id, RF019[min_max] -> iss.i_class_id, RF014[bloom] -> iss.i_brand_id, RF015[bloom] -> iss.i_category_id
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=5 row-size=20B cardinality=18.00K cost=352
+|  |     in pipelines: 05(GETNEXT)
+|  |
+|  04:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF029[min_max] -> ss_item_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=4 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 04(GETNEXT)
+|
+28:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=33
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=0,1,2 row-size=48B cardinality=2.88M cost=2880404
+|  in pipelines: 00(GETNEXT), 02(OPEN)
+|
+|--F90:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=20.57MB mem-reservation=20.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[75620]
+|  JOIN BUILD
+|  |  join-table-id=33 plan-id=34 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF010[bloom] <- d_date_sk
+|  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=73049
+|  |
+|  93:EXCHANGE [BROADCAST]
+|  |  mem-estimate=582.70KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=8B cardinality=73.05K cost=2571
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1428]
+|  02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     runtime filters: RF007[min_max] -> d_week_seq, RF006[bloom] -> d_week_seq
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=2 row-size=8B cardinality=73.05K cost=571
+|     in pipelines: 02(GETNEXT)
+|
+27:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=34
+|  hash predicates: ss_item_sk = i_item_sk
+|  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1 row-size=40B cardinality=2.88M cost=2880404
+|  in pipelines: 00(GETNEXT), 01(OPEN)
+|
+|--F91:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=9.12MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[19266]
+|  JOIN BUILD
+|  |  join-table-id=34 plan-id=35 cohort-id=01
+|  |  build expressions: i_item_sk
+|  |  runtime filters: RF012[bloom] <- i_item_sk, RF013[min_max] <- i_item_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |
+|  92:EXCHANGE [BROADCAST]
+|  |  mem-estimate=375.56KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1 row-size=20B cardinality=18.00K cost=1266
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=4.00MB mem-reservation=4.00MB thread-reservation=0 runtime-filters-memory=4.00MB
+|  Per-Instance Resources: mem-estimate=16.09MB mem-reservation=256.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[774]
+|  01:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     runtime filters: RF003[min_max] -> tpcds_parquet.item.i_brand_id, RF004[min_max] -> tpcds_parquet.item.i_category_id, RF005[min_max] -> tpcds_parquet.item.i_class_id, RF009[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_brand_id, RF001[bloom] -> tpcds_parquet.item.i_category_id, RF002[bloom] -> tpcds_parquet.item.i_class_id, RF008[bloom] -> tpcds_parquet.item.i_item_sk
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=1 row-size=20B cardinality=18.00K cost=352
+|     in pipelines: 01(GETNEXT)
+|
+00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF009[min_max] -> ss_item_sk, RF013[min_max] -> ss_item_sk, RF008[bloom] -> ss_item_sk, RF010[bloom] -> ss_sold_date_sk, RF012[bloom] -> ss_item_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=0 row-size=20B cardinality=2.88M cost=56258
+   in pipelines: 00(GETNEXT)
+====
+# TPCDS-Q23-1 First of two queries.
+with frequent_ss_items as
+ (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt
+  from store_sales
+      ,date_dim
+      ,item
+  where ss_sold_date_sk = d_date_sk
+    and ss_item_sk = i_item_sk
+    and d_year in (2000,2000+1,2000+2,2000+3)
+  group by substr(i_item_desc,1,30),i_item_sk,d_date
+  having count(*) >4),
+ max_store_sales as
+ (select max(csales) tpcds_cmax
+  from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales
+        from store_sales
+            ,customer
+            ,date_dim
+        where ss_customer_sk = c_customer_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year in (2000,2000+1,2000+2,2000+3)
+        group by c_customer_sk) x),
+ best_ss_customer as
+ (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales
+  from store_sales
+      ,customer
+  where ss_customer_sk = c_customer_sk
+  group by c_customer_sk
+  having sum(ss_quantity*ss_sales_price) > (50/100.0) * (select
+  *
+from
+ max_store_sales))
+  select  sum(sales)
+ from (select cs_quantity*cs_list_price sales
+       from catalog_sales
+           ,date_dim
+       where d_year = 2000
+         and d_moy = 2
+         and cs_sold_date_sk = d_date_sk
+         and cs_item_sk in (select item_sk from frequent_ss_items)
+         and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer)
+      union all
+      select ws_quantity*ws_list_price sales
+       from web_sales
+           ,date_dim
+       where d_year = 2000
+         and d_moy = 2
+         and ws_sold_date_sk = d_date_sk
+         and ws_item_sk in (select item_sk from frequent_ss_items)
+         and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) y
+ limit 100;
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=572.88MB Threads=75
+Per-Host Resource Estimates: Memory=1.39GB
+F31:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=32.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[2, 0]
+PLAN-ROOT SINK
+|  output exprs: sum(sales)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0 cost=0
+|
+89:AGGREGATE [FINALIZE]
+|  output: sum:merge(sales)
+|  limit: 100
+|  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=46 row-size=16B cardinality=1 cost=1
+|  in pipelines: 89(GETNEXT), 49(OPEN)
+|
+88:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=46 row-size=16B cardinality=1 cost=1
+|  in pipelines: 49(GETNEXT)
+|
+F30:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Instance Resources: mem-estimate=10.93MB mem-reservation=1.94MB thread-reservation=1
+max-parallelism=12 fragment-costs=[202344, 202344, 584489, 1]
+49:AGGREGATE
+|  output: sum(sales)
+|  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=46 row-size=16B cardinality=1 cost=128163
+|  in pipelines: 49(GETNEXT), 52(OPEN), 71(OPEN)
+|
+00:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=44 row-size=8B cardinality=128.16K cost=256326
+|  in pipelines: 52(GETNEXT), 71(GETNEXT)
+|
+|--48:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
+|  |  hash-table-id=08
+|  |  hash predicates: c_customer_sk = ws_bill_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=22,23 row-size=36B cardinality=42.85K cost=100000
+|  |  in pipelines: 71(GETNEXT), 25(OPEN)
+|  |
+|  |--F40:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Instance Resources: mem-estimate=3.76MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=12 fragment-costs=[44696]
+|  |  JOIN BUILD
+|  |  |  join-table-id=08 plan-id=09 cohort-id=01
+|  |  |  build expressions: ws_bill_customer_sk
+|  |  |  runtime filters: RF016[bloom] <- ws_bill_customer_sk, RF017[min_max] <- ws_bill_customer_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=42854
+|  |  |
+|  |  87:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  |  mem-estimate=841.29KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=22,23 row-size=36B cardinality=42.85K cost=1842
+|  |  |  in pipelines: 25(GETNEXT)
+|  |  |
+|  |  F23:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=34.06MB mem-reservation=8.00MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[780941]
+|  |  47:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  |  hash-table-id=09
+|  |  |  hash predicates: ws_item_sk = i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=22,23 row-size=36B cardinality=42.85K cost=42854
+|  |  |  in pipelines: 25(GETNEXT), 85(OPEN)
+|  |  |
+|  |  |--F41:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[18397]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=09 plan-id=10 cohort-id=05
+|  |  |  |  build expressions: i_item_sk
+|  |  |  |  runtime filters: RF025[min_max] <- i_item_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |  |
+|  |  |  86:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=68 row-size=8B cardinality=17.98K cost=422
+|  |  |  |  in pipelines: 85(GETNEXT)
+|  |  |  |
+|  |  |  F29:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  |  85:AGGREGATE [FINALIZE]
+|  |  |  |  group by: i_item_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=68 row-size=8B cardinality=17.98K cost=17975
+|  |  |  |  in pipelines: 85(GETNEXT), 83(OPEN)
+|  |  |  |
+|  |  |  84:EXCHANGE [HASH(i_item_sk)]
+|  |  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=68 row-size=8B cardinality=17.98K cost=211
+|  |  |  |  in pipelines: 83(GETNEXT)
+|  |  |  |
+|  |  |  F28:PLAN FRAGMENT [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=67.04MB mem-reservation=36.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[9542330, 235454, 211]
+|  |  |  46:AGGREGATE [STREAMING]
+|  |  |  |  group by: i_item_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=68 row-size=8B cardinality=17.98K cost=235454
+|  |  |  |  in pipelines: 83(GETNEXT)
+|  |  |  |
+|  |  |  83:AGGREGATE [FINALIZE]
+|  |  |  |  output: count:merge(*)
+|  |  |  |  group by: substr(i_item_desc, 1, 30), i_item_sk, d_date
+|  |  |  |  having: count(*) > CAST(4 AS BIGINT)
+|  |  |  |  mem-estimate=46.41MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=50B cardinality=235.45K cost=9418164
+|  |  |  |  in pipelines: 83(GETNEXT), 27(OPEN)
+|  |  |  |
+|  |  |  82:EXCHANGE [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)]
+|  |  |  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=50B cardinality=2.35M cost=124166
+|  |  |  |  in pipelines: 27(GETNEXT)
+|  |  |  |
+|  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=33.63MB mem-reservation=17.50MB thread-reservation=1
+|  |  |  max-parallelism=12 fragment-costs=[14686864, 124166]
+|  |  |  32:AGGREGATE [STREAMING]
+|  |  |  |  output: count(*)
+|  |  |  |  group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date
+|  |  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  |  tuple-ids=27 row-size=50B cardinality=2.35M cost=9418164
+|  |  |  |  in pipelines: 27(GETNEXT)
+|  |  |  |
+|  |  |  31:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=10
+|  |  |  |  hash predicates: ss_item_sk = i_item_sk
+|  |  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=24,25,26 row-size=162B cardinality=2.35M cost=2354541
+|  |  |  |  in pipelines: 27(GETNEXT), 29(OPEN)
+|  |  |  |
+|  |  |  |--F42:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[24564]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=10 plan-id=11 cohort-id=06
+|  |  |  |  |  build expressions: i_item_sk
+|  |  |  |  |  runtime filters: RF029[min_max] <- i_item_sk
+|  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |  |  |
+|  |  |  |  81:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=26 row-size=120B cardinality=18.00K cost=6564
+|  |  |  |  |  in pipelines: 29(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[4306]
+|  |  |  |  29:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=18.00K size=1.73MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |  |  |     tuple-ids=26 row-size=120B cardinality=18.00K cost=2118
+|  |  |  |     in pipelines: 29(GETNEXT)
+|  |  |  |
+|  |  |  30:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=11
+|  |  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=24,25 row-size=42B cardinality=2.35M cost=2880404
+|  |  |  |  in pipelines: 27(GETNEXT), 28(OPEN)
+|  |  |  |
+|  |  |  |--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  max-parallelism=3 fragment-costs=[1641]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=11 plan-id=12 cohort-id=06
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF030[bloom] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |
+|  |  |  |  80:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=25 row-size=30B cardinality=1.49K cost=150
+|  |  |  |  |  in pipelines: 28(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[75240]
+|  |  |  |  28:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |  |  |     tuple-ids=25 row-size=30B cardinality=1.49K cost=75190
+|  |  |  |     in pipelines: 28(GETNEXT)
+|  |  |  |
+|  |  |  27:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |  |     runtime filters: RF029[min_max] -> ss_item_sk, RF030[bloom] -> ss_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=2.88M size=200.96MB
+|  |  |       partitions: 1824/1824 rows=2.88M
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=24 row-size=12B cardinality=2.88M cost=33755
+|  |  |     in pipelines: 27(GETNEXT)
+|  |  |
+|  |  45:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=12
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=22,23 row-size=36B cardinality=42.85K cost=719384
+|  |  |  in pipelines: 25(GETNEXT), 26(OPEN)
+|  |  |
+|  |  |--F44:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=2 fragment-costs=[112]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=12 plan-id=13 cohort-id=05
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF026[bloom] <- d_date_sk, RF027[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  |
+|  |  |  79:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=23 row-size=12B cardinality=108 cost=4
+|  |  |  |  in pipelines: 26(GETNEXT)
+|  |  |  |
+|  |  |  F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146957]
+|  |  |  26:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=23 row-size=12B cardinality=108 cost=146955
+|  |  |     in pipelines: 26(GETNEXT)
+|  |  |
+|  |  25:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF027[min_max] -> ws_sold_date_sk, RF025[min_max] -> ws_item_sk, RF026[bloom] -> ws_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
+|  |     tuple-ids=22 row-size=24B cardinality=719.38K cost=16861
+|  |     in pipelines: 25(GETNEXT)
+|  |
+|  44:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  |  join table id: 13
+|  |  predicates: sum(ss_quantity * ss_sales_price) > CAST(0.500000 AS DECIMAL(10,6)) * max(csales)
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=32,39 row-size=36B cardinality=100.00K cost=0
+|  |  in pipelines: 71(GETNEXT), 77(OPEN)
+|  |
+|  |--F45:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  JOIN BUILD
+|  |  |  join-table-id=13 plan-id=14 cohort-id=01
+|  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |
+|  |  78:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=39 row-size=16B cardinality=1 cost=3
+|  |  |  in pipelines: 77(GETNEXT)
+|  |  |
+|  |  F22:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  77:AGGREGATE [FINALIZE]
+|  |  |  output: max:merge(csales)
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=39 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 77(GETNEXT), 43(OPEN)
+|  |  |
+|  |  76:EXCHANGE [UNPARTITIONED]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=39 row-size=16B cardinality=1 cost=1
+|  |  |  in pipelines: 43(GETNEXT)
+|  |  |
+|  |  F21:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[202344, 100000, 1]
+|  |  43:AGGREGATE
+|  |  |  output: max(sum(ss_quantity * ss_sales_price))
+|  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=39 row-size=16B cardinality=1 cost=100000
+|  |  |  in pipelines: 43(GETNEXT), 75(OPEN)
+|  |  |
+|  |  75:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  |  group by: c_customer_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=37 row-size=20B cardinality=100.00K cost=200000
+|  |  |  in pipelines: 75(GETNEXT), 37(OPEN)
+|  |  |
+|  |  74:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=37 row-size=20B cardinality=100.00K cost=2344
+|  |  |  in pipelines: 37(GETNEXT)
+|  |  |
+|  |  F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=26.28MB mem-reservation=6.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[9989034, 2344]
+|  |  42:AGGREGATE [STREAMING]
+|  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  |  group by: c_customer_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=37 row-size=20B cardinality=100.00K cost=4709082
+|  |  |  in pipelines: 37(GETNEXT)
+|  |  |
+|  |  41:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=14
+|  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=34,36,35 row-size=28B cardinality=2.35M cost=2354541
+|  |  |  in pipelines: 37(GETNEXT), 38(OPEN)
+|  |  |
+|  |  |--F46:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=14 plan-id=15 cohort-id=07
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF021[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |
+|  |  |  73:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=35 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  in pipelines: 38(GETNEXT)
+|  |  |  |
+|  |  |  F20:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  38:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=35 row-size=4B cardinality=100.00K cost=391
+|  |  |     in pipelines: 38(GETNEXT)
+|  |  |
+|  |  40:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=15
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=34,36 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  in pipelines: 37(GETNEXT), 39(OPEN)
+|  |  |
+|  |  |--F47:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=3 fragment-costs=[1545]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=15 plan-id=16 cohort-id=07
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF022[bloom] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |
+|  |  |  72:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=36 row-size=8B cardinality=1.49K cost=54
+|  |  |  |  in pipelines: 39(GETNEXT)
+|  |  |  |
+|  |  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[73638]
+|  |  |  39:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=36 row-size=8B cardinality=1.49K cost=73620
+|  |  |     in pipelines: 39(GETNEXT)
+|  |  |
+|  |  37:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF021[min_max] -> ss_customer_sk, RF022[bloom] -> ss_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=34 row-size=16B cardinality=2.88M cost=45007
+|  |     in pipelines: 37(GETNEXT)
+|  |
+|  71:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  group by: c_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=32 row-size=20B cardinality=100.00K cost=200000
+|  |  in pipelines: 71(GETNEXT), 33(OPEN)
+|  |
+|  70:EXCHANGE [HASH(c_customer_sk)]
+|  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=32 row-size=20B cardinality=100.00K cost=2344
+|  |  in pipelines: 33(GETNEXT)
+|  |
+|  F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=27.12MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[8674967, 2344]
+|  36:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  group by: c_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=32 row-size=20B cardinality=100.00K cost=5760808
+|  |  in pipelines: 33(GETNEXT)
+|  |
+|  35:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=17
+|  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=30,31 row-size=16B cardinality=2.88M cost=2880404
+|  |  in pipelines: 33(GETNEXT), 34(OPEN)
+|  |
+|  |--F49:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  JOIN BUILD
+|  |  |  join-table-id=17 plan-id=18 cohort-id=01
+|  |  |  build expressions: c_customer_sk
+|  |  |  runtime filters: RF019[min_max] <- c_customer_sk
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |
+|  |  69:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=31 row-size=4B cardinality=100.00K cost=2346
+|  |  |  in pipelines: 34(GETNEXT)
+|  |  |
+|  |  F16:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1173]
+|  |  34:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     runtime filters: RF017[min_max] -> tpcds_parquet.customer.c_customer_sk, RF016[bloom] -> tpcds_parquet.customer.c_customer_sk
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=31 row-size=4B cardinality=100.00K cost=391
+|  |     in pipelines: 34(GETNEXT)
+|  |
+|  33:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF017[min_max] -> tpcds_parquet.store_sales.ss_customer_sk, RF019[min_max] -> ss_customer_sk, RF016[bloom] -> tpcds_parquet.store_sales.ss_customer_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|     tuple-ids=30 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 33(GETNEXT)
+|
+24:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: c_customer_sk = cs_bill_customer_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1 row-size=36B cardinality=85.31K cost=100000
+|  in pipelines: 52(GETNEXT), 01(OPEN)
+|
+|--F32:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=12 fragment-costs=[88975]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: cs_bill_customer_sk
+|  |  runtime filters: RF000[bloom] <- cs_bill_customer_sk, RF001[min_max] <- cs_bill_customer_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=85309
+|  |
+|  68:EXCHANGE [HASH(cs_bill_customer_sk)]
+|  |  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,1 row-size=36B cardinality=85.31K cost=3666
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=50.06MB mem-reservation=8.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[1564310]
+|  23:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash-table-id=01
+|  |  hash predicates: cs_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=0,1 row-size=36B cardinality=85.31K cost=85309
+|  |  in pipelines: 01(GETNEXT), 66(OPEN)
+|  |
+|  |--F33:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=7.92MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[18608]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF009[min_max] <- i_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=17975
+|  |  |
+|  |  67:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=176.43KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=8B cardinality=17.98K cost=633
+|  |  |  in pipelines: 66(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [HASH(i_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=10.08MB mem-reservation=1.94MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[18186, 211]
+|  |  66:AGGREGATE [FINALIZE]
+|  |  |  group by: i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=57 row-size=8B cardinality=17.98K cost=17975
+|  |  |  in pipelines: 66(GETNEXT), 64(OPEN)
+|  |  |
+|  |  65:EXCHANGE [HASH(i_item_sk)]
+|  |  |  mem-estimate=82.81KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=57 row-size=8B cardinality=17.98K cost=211
+|  |  |  in pipelines: 64(GETNEXT)
+|  |  |
+|  |  F13:PLAN FRAGMENT [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=67.04MB mem-reservation=36.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[9542330, 235454, 211]
+|  |  22:AGGREGATE [STREAMING]
+|  |  |  group by: i_item_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=57 row-size=8B cardinality=17.98K cost=235454
+|  |  |  in pipelines: 64(GETNEXT)
+|  |  |
+|  |  64:AGGREGATE [FINALIZE]
+|  |  |  output: count:merge(*)
+|  |  |  group by: substr(i_item_desc, 1, 30), i_item_sk, d_date
+|  |  |  having: count(*) > CAST(4 AS BIGINT)
+|  |  |  mem-estimate=46.41MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  |  tuple-ids=5 row-size=50B cardinality=235.45K cost=9418164
+|  |  |  in pipelines: 64(GETNEXT), 03(OPEN)
+|  |  |
+|  |  63:EXCHANGE [HASH(substr(i_item_desc, 1, 30),i_item_sk,d_date)]
+|  |  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5 row-size=50B cardinality=2.35M cost=124166
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=33.63MB mem-reservation=17.50MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[14686864, 124166]
+|  |  08:AGGREGATE [STREAMING]
+|  |  |  output: count(*)
+|  |  |  group by: substr(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), i_item_sk, d_date
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=5 row-size=50B cardinality=2.35M cost=9418164
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  07:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: ss_item_sk = i_item_sk
+|  |  |  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=2,3,4 row-size=162B cardinality=2.35M cost=2354541
+|  |  |  in pipelines: 03(GETNEXT), 05(OPEN)
+|  |  |
+|  |  |--F34:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[24564]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
+|  |  |  |  build expressions: i_item_sk
+|  |  |  |  runtime filters: RF013[min_max] <- i_item_sk
+|  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |  |
+|  |  |  62:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=4 row-size=120B cardinality=18.00K cost=6564
+|  |  |  |  in pipelines: 05(GETNEXT)
+|  |  |  |
+|  |  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[4306]
+|  |  |  05:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |  |     stored statistics:
+|  |  |       table: rows=18.00K size=1.73MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |  |     tuple-ids=4 row-size=120B cardinality=18.00K cost=2118
+|  |  |     in pipelines: 05(GETNEXT)
+|  |  |
+|  |  06:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=03
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=2,3 row-size=42B cardinality=2.35M cost=2880404
+|  |  |  in pipelines: 03(GETNEXT), 04(OPEN)
+|  |  |
+|  |  |--F35:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=3 fragment-costs=[1641]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF014[bloom] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |
+|  |  |  61:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=3 row-size=30B cardinality=1.49K cost=150
+|  |  |  |  in pipelines: 04(GETNEXT)
+|  |  |  |
+|  |  |  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[75240]
+|  |  |  04:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |  |     tuple-ids=3 row-size=30B cardinality=1.49K cost=75190
+|  |  |     in pipelines: 04(GETNEXT)
+|  |  |
+|  |  03:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF013[min_max] -> ss_item_sk, RF014[bloom] -> ss_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=2 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 03(GETNEXT)
+|  |
+|  21:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=04
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=0,1 row-size=36B cardinality=85.31K cost=1441548
+|  |  in pipelines: 01(GETNEXT), 02(OPEN)
+|  |
+|  |--F36:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  60:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=1 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 02(GETNEXT)
+|  |  |
+|  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=1 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 02(GETNEXT)
+|  |
+|  01:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     runtime filters: RF011[min_max] -> cs_sold_date_sk, RF009[min_max] -> cs_item_sk, RF010[bloom] -> cs_sold_date_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
+|     tuple-ids=0 row-size=24B cardinality=1.44M cost=33787
+|     in pipelines: 01(GETNEXT)
+|
+20:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
+|  join table id: 05
+|  predicates: sum(ss_quantity * ss_sales_price) > CAST(0.500000 AS DECIMAL(10,6)) * max(csales)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=10,17 row-size=36B cardinality=100.00K cost=0
+|  in pipelines: 52(GETNEXT), 58(OPEN)
+|
+|--F37:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[3]
+|  JOIN BUILD
+|  |  join-table-id=05 plan-id=06 cohort-id=01
+|  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |
+|  59:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=17 row-size=16B cardinality=1 cost=3
+|  |  in pipelines: 58(GETNEXT)
+|  |
+|  F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=1 fragment-costs=[2, 1]
+|  58:AGGREGATE [FINALIZE]
+|  |  output: max:merge(csales)
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=17 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 58(GETNEXT), 19(OPEN)
+|  |
+|  57:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=17 row-size=16B cardinality=1 cost=1
+|  |  in pipelines: 19(GETNEXT)
+|  |
+|  F06:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=2.88MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[202344, 100000, 1]
+|  19:AGGREGATE
+|  |  output: max(sum(ss_quantity * ss_sales_price))
+|  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=17 row-size=16B cardinality=1 cost=100000
+|  |  in pipelines: 19(GETNEXT), 56(OPEN)
+|  |
+|  56:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  group by: c_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=15 row-size=20B cardinality=100.00K cost=200000
+|  |  in pipelines: 56(GETNEXT), 13(OPEN)
+|  |
+|  55:EXCHANGE [HASH(c_customer_sk)]
+|  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=15 row-size=20B cardinality=100.00K cost=2344
+|  |  in pipelines: 13(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=26.28MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[9989034, 2344]
+|  18:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  group by: c_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=15 row-size=20B cardinality=100.00K cost=4709082
+|  |  in pipelines: 13(GETNEXT)
+|  |
+|  17:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=06
+|  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=12,14,13 row-size=28B cardinality=2.35M cost=2354541
+|  |  in pipelines: 13(GETNEXT), 14(OPEN)
+|  |
+|  |--F38:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=20.39MB mem-reservation=20.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  JOIN BUILD
+|  |  |  join-table-id=06 plan-id=07 cohort-id=04
+|  |  |  build expressions: c_customer_sk
+|  |  |  runtime filters: RF004[bloom] <- c_customer_sk, RF005[min_max] <- c_customer_sk
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |
+|  |  54:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=13 row-size=4B cardinality=100.00K cost=2346
+|  |  |  in pipelines: 14(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1173]
+|  |  14:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=13 row-size=4B cardinality=100.00K cost=391
+|  |     in pipelines: 14(GETNEXT)
+|  |
+|  16:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=07
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=12,14 row-size=24B cardinality=2.35M cost=2880404
+|  |  in pipelines: 13(GETNEXT), 15(OPEN)
+|  |
+|  |--F39:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[1545]
+|  |  JOIN BUILD
+|  |  |  join-table-id=07 plan-id=08 cohort-id=04
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF006[bloom] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |
+|  |  53:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=14 row-size=8B cardinality=1.49K cost=54
+|  |  |  in pipelines: 15(GETNEXT)
+|  |  |
+|  |  F04:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[73638]
+|  |  15:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=14 row-size=8B cardinality=1.49K cost=73620
+|  |     in pipelines: 15(GETNEXT)
+|  |
+|  13:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF005[min_max] -> ss_customer_sk, RF006[bloom] -> ss_sold_date_sk, RF004[bloom] -> ss_customer_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|     tuple-ids=12 row-size=16B cardinality=2.88M cost=45007
+|     in pipelines: 13(GETNEXT)
+|
+52:AGGREGATE [FINALIZE]
+|  output: sum:merge(ss_quantity * ss_sales_price)
+|  group by: c_customer_sk
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10 row-size=20B cardinality=100.00K cost=200000
+|  in pipelines: 52(GETNEXT), 09(OPEN)
+|
+51:EXCHANGE [HASH(c_customer_sk)]
+|  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=10 row-size=20B cardinality=100.00K cost=2344
+|  in pipelines: 09(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=27.12MB mem-reservation=6.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[8674967, 2344]
+12:AGGREGATE [STREAMING]
+|  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  group by: c_customer_sk
+|  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=10 row-size=20B cardinality=100.00K cost=5760808
+|  in pipelines: 09(GETNEXT)
+|
+11:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=16
+|  hash predicates: ss_customer_sk = c_customer_sk
+|  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=8,9 row-size=16B cardinality=2.88M cost=2880404
+|  in pipelines: 09(GETNEXT), 10(OPEN)
+|
+|--F48:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=20.39MB mem-reservation=20.00MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[102346]
+|  JOIN BUILD
+|  |  join-table-id=16 plan-id=17 cohort-id=01
+|  |  build expressions: c_customer_sk
+|  |  runtime filters: RF002[bloom] <- c_customer_sk, RF003[min_max] <- c_customer_sk
+|  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |
+|  50:EXCHANGE [BROADCAST]
+|  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=9 row-size=4B cardinality=100.00K cost=2346
+|  |  in pipelines: 10(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1173]
+|  10:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|     HDFS partitions=1/1 files=1 size=5.49MB
+|     runtime filters: RF001[min_max] -> tpcds_parquet.customer.c_customer_sk, RF000[bloom] -> tpcds_parquet.customer.c_customer_sk
+|     stored statistics:
+|       table: rows=100.00K size=5.49MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=9 row-size=4B cardinality=100.00K cost=391
+|     in pipelines: 10(GETNEXT)
+|
+09:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_customer_sk, RF003[min_max] -> ss_customer_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_customer_sk, RF002[bloom] -> ss_customer_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=8 row-size=12B cardinality=2.88M cost=33755
+   in pipelines: 09(GETNEXT)
+====
+# TPCDS-Q23b second of two queries
+WITH frequent_ss_items AS
+  (SELECT itemdesc,
+          i_item_sk item_sk,
+          d_date solddate,
+          count(*) cnt
+   FROM store_sales,
+        date_dim,
+     (SELECT SUBSTRING(i_item_desc, 1, 30) itemdesc,
+             *
+      FROM item) sq1
+   WHERE ss_sold_date_sk = d_date_sk
+     AND ss_item_sk = i_item_sk
+     AND d_year IN (2000,
+                    2000+1,
+                    2000+2,
+                    2000+3)
+   GROUP BY itemdesc,
+            i_item_sk,
+            d_date
+   HAVING count(*) >4),
+     max_store_sales AS
+  (SELECT max(csales) tpcds_cmax
+   FROM
+     (SELECT c_customer_sk,
+             sum(ss_quantity*ss_sales_price) csales
+      FROM store_sales,
+           customer,
+           date_dim
+      WHERE ss_customer_sk = c_customer_sk
+        AND ss_sold_date_sk = d_date_sk
+        AND d_year IN (2000,
+                       2000+1,
+                       2000+2,
+                       2000+3)
+      GROUP BY c_customer_sk) sq2),
+     best_ss_customer AS
+  (SELECT c_customer_sk,
+          sum(ss_quantity*ss_sales_price) ssales
+   FROM store_sales,
+        customer,
+        max_store_sales
+   WHERE ss_customer_sk = c_customer_sk
+   GROUP BY c_customer_sk
+   HAVING sum(ss_quantity*ss_sales_price) > (50/100.0) * max(tpcds_cmax))
+SELECT c_last_name,
+       c_first_name,
+       sales
+FROM
+  (SELECT c_last_name,
+          c_first_name,
+          sum(cs_quantity*cs_list_price) sales
+   FROM catalog_sales,
+        customer,
+        date_dim,
+        frequent_ss_items,
+        best_ss_customer
+   WHERE d_year = 2000
+     AND d_moy = 2
+     AND cs_sold_date_sk = d_date_sk
+     AND cs_item_sk = item_sk
+     AND cs_bill_customer_sk = best_ss_customer.c_customer_sk
+     AND cs_bill_customer_sk = customer.c_customer_sk
+   GROUP BY c_last_name,
+            c_first_name
+   UNION ALL SELECT c_last_name,
+                    c_first_name,
+                    sum(ws_quantity*ws_list_price) sales
+   FROM web_sales,
+        customer,
+        date_dim,
+        frequent_ss_items,
+        best_ss_customer
+   WHERE d_year = 2000
+     AND d_moy = 2
+     AND ws_sold_date_sk = d_date_sk
+     AND ws_item_sk = item_sk
+     AND ws_bill_customer_sk = best_ss_customer.c_customer_sk
+     AND ws_bill_customer_sk = customer.c_customer_sk
+   GROUP BY c_last_name,
+            c_first_name) sq3
+ORDER BY c_last_name,
+         c_first_name,
+         sales
+LIMIT 100;
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=623.75MB Threads=75
+Per-Host Resource Estimates: Memory=1.44GB
+F35:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.07MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[306]
+PLAN-ROOT SINK
+|  output exprs: c_last_name, c_first_name, sales
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=300
+|
+96:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: c_last_name ASC, c_first_name ASC, sales ASC
+|  limit: 100
+|  mem-estimate=67.28KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=44 row-size=52B cardinality=100 cost=6
+|  in pipelines: 53(GETNEXT)
+|
+F34:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Instance Resources: mem-estimate=10.34MB mem-reservation=1.94MB thread-reservation=1
+max-parallelism=12 fragment-costs=[32154, 29405, 300, 6]
+53:TOP-N [LIMIT=100]
+|  order by: c_last_name ASC, c_first_name ASC, sales ASC
+|  mem-estimate=5.08KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=44 row-size=52B cardinality=100 cost=300
+|  in pipelines: 53(GETNEXT), 74(OPEN), 95(OPEN)
+|
+00:UNION
+|  pass-through-operands: all
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=42 row-size=52B cardinality=20.15K cost=0
+|  in pipelines: 74(GETNEXT), 95(GETNEXT)
+|
+|--95:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ws_quantity * ws_list_price)
+|  |  group by: c_last_name, c_first_name
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=41 row-size=52B cardinality=9.63K cost=28878
+|  |  in pipelines: 95(GETNEXT), 78(OPEN)
+|  |
+|  94:EXCHANGE [HASH(c_last_name,c_first_name)]
+|  |  mem-estimate=330.72KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=41 row-size=52B cardinality=9.63K cost=527
+|  |  in pipelines: 78(GETNEXT)
+|  |
+|  F20:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=67.04MB mem-reservation=36.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[9542330, 264332, 527]
+|  52:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ws_quantity AS DECIMAL(10,0)) * ws_list_price)
+|  |  group by: c_last_name, c_first_name
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=41 row-size=52B cardinality=9.63K cost=28878
+|  |  in pipelines: 78(GETNEXT)
+|  |
+|  51:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=10
+|  |  hash predicates: i_item_sk = ws_item_sk
+|  |  fk/pk conjuncts: i_item_sk = ws_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=28,21,23,39,22 row-size=162B cardinality=9.63K cost=235454
+|  |  in pipelines: 78(GETNEXT), 27(OPEN)
+|  |
+|  |--F46:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=13.78MB mem-reservation=12.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[13235]
+|  |  JOIN BUILD
+|  |  |  join-table-id=10 plan-id=11 cohort-id=05
+|  |  |  build expressions: ws_item_sk
+|  |  |  runtime filters: RF018[bloom] <- ws_item_sk, RF019[min_max] <- ws_item_sk
+|  |  |  mem-estimate=11.50MB mem-reservation=11.50MB spill-buffer=128.00KB thread-reservation=0 cost=9626
+|  |  |
+|  |  93:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=1.28MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=21,23,39,22 row-size=112B cardinality=9.63K cost=3609
+|  |  |  in pipelines: 27(GETNEXT)
+|  |  |
+|  |  F32:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=1018.27KB mem-reservation=0B thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[11619]
+|  |  50:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  |  hash-table-id=11
+|  |  |  hash predicates: ws_bill_customer_sk = customer.c_customer_sk
+|  |  |  fk/pk conjuncts: ws_bill_customer_sk = customer.c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=21,23,39,22 row-size=112B cardinality=9.63K cost=9626
+|  |  |  in pipelines: 27(GETNEXT), 28(OPEN)
+|  |  |
+|  |  |--F47:PLAN FRAGMENT [HASH(ws_bill_customer_sk)] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.60MB mem-reservation=4.75MB thread-reservation=1
+|  |  |  |  max-parallelism=2 fragment-costs=[104294]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=11 plan-id=12 cohort-id=06
+|  |  |  |  build expressions: customer.c_customer_sk
+|  |  |  |  runtime filters: RF025[min_max] <- customer.c_customer_sk
+|  |  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |
+|  |  |  92:EXCHANGE [HASH(customer.c_customer_sk)]
+|  |  |  |  mem-estimate=3.85MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=22 row-size=40B cardinality=100.00K cost=4294
+|  |  |  |  in pipelines: 28(GETNEXT)
+|  |  |  |
+|  |  |  F31:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.34MB mem-reservation=1.00MB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[8197]
+|  |  |  28:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |  |     tuple-ids=22 row-size=40B cardinality=100.00K cost=3903
+|  |  |     in pipelines: 28(GETNEXT)
+|  |  |
+|  |  91:EXCHANGE [HASH(ws_bill_customer_sk)]
+|  |  |  mem-estimate=506.41KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=21,23,39 row-size=72B cardinality=9.63K cost=790
+|  |  |  in pipelines: 27(GETNEXT)
+|  |  |
+|  |  F21:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  |  Per-Instance Resources: mem-estimate=32.66MB mem-reservation=8.00MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[779889]
+|  |  49:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=12
+|  |  |  hash predicates: ws_bill_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ws_bill_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=21,23,39 row-size=72B cardinality=9.63K cost=42854
+|  |  |  in pipelines: 27(GETNEXT), 89(OPEN)
+|  |  |
+|  |  |--F48:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=9.21MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=2 fragment-costs=[10782]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=12 plan-id=13 cohort-id=06
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF026[bloom] <- c_customer_sk, RF027[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=10000
+|  |  |  |
+|  |  |  90:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=471.56KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=36B cardinality=10.00K cost=782
+|  |  |  |  in pipelines: 89(GETNEXT)
+|  |  |  |
+|  |  |  F30:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=11.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[303907, 391]
+|  |  |  89:AGGREGATE [FINALIZE]
+|  |  |  |  output: sum:merge(ss_quantity * ss_sales_price), max:merge(tpcds_cmax)
+|  |  |  |  group by: c_customer_sk
+|  |  |  |  having: sum(ss_quantity * ss_sales_price) > CAST(0.500000 AS DECIMAL(10,6)) * max(tpcds_cmax)
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=36B cardinality=10.00K cost=300000
+|  |  |  |  in pipelines: 89(GETNEXT), 36(OPEN)
+|  |  |  |
+|  |  |  88:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=1.61MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=36B cardinality=100.00K cost=3907
+|  |  |  |  in pipelines: 36(GETNEXT)
+|  |  |  |
+|  |  |  F23:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Instance Resources: mem-estimate=26.47MB mem-reservation=10.00MB thread-reservation=1
+|  |  |  max-parallelism=12 fragment-costs=[11555371, 3907]
+|  |  |  47:AGGREGATE [STREAMING]
+|  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales))
+|  |  |  |  group by: c_customer_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  |  tuple-ids=39 row-size=36B cardinality=100.00K cost=8641212
+|  |  |  |  in pipelines: 36(GETNEXT)
+|  |  |  |
+|  |  |  46:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
+|  |  |  |  join table id: 13
+|  |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=30,31,37 row-size=32B cardinality=2.88M cost=0
+|  |  |  |  in pipelines: 36(GETNEXT), 86(OPEN)
+|  |  |  |
+|  |  |  |--F49:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=13 plan-id=14 cohort-id=07
+|  |  |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |  |  |
+|  |  |  |  87:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=37 row-size=16B cardinality=1 cost=3
+|  |  |  |  |  in pipelines: 86(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F29:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  |  |  86:AGGREGATE [FINALIZE]
+|  |  |  |  |  output: max:merge(csales)
+|  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  |  tuple-ids=37 row-size=16B cardinality=1 cost=1
+|  |  |  |  |  in pipelines: 86(GETNEXT), 44(OPEN)
+|  |  |  |  |
+|  |  |  |  85:EXCHANGE [UNPARTITIONED]
+|  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=37 row-size=16B cardinality=1 cost=1
+|  |  |  |  |  in pipelines: 44(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F28:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[202344, 100000, 1]
+|  |  |  |  44:AGGREGATE
+|  |  |  |  |  output: max(sum(ss_quantity * ss_sales_price))
+|  |  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  |  tuple-ids=37 row-size=16B cardinality=1 cost=100000
+|  |  |  |  |  in pipelines: 44(GETNEXT), 84(OPEN)
+|  |  |  |  |
+|  |  |  |  84:AGGREGATE [FINALIZE]
+|  |  |  |  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  |  |  |  group by: c_customer_sk
+|  |  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=35 row-size=20B cardinality=100.00K cost=200000
+|  |  |  |  |  in pipelines: 84(GETNEXT), 38(OPEN)
+|  |  |  |  |
+|  |  |  |  83:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=35 row-size=20B cardinality=100.00K cost=2344
+|  |  |  |  |  in pipelines: 38(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F25:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  |  |  Per-Instance Resources: mem-estimate=26.28MB mem-reservation=6.00MB thread-reservation=1
+|  |  |  |  max-parallelism=12 fragment-costs=[9989034, 2344]
+|  |  |  |  43:AGGREGATE [STREAMING]
+|  |  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  |  |  |  group by: c_customer_sk
+|  |  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=35 row-size=20B cardinality=100.00K cost=4709082
+|  |  |  |  |  in pipelines: 38(GETNEXT)
+|  |  |  |  |
+|  |  |  |  42:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  |  hash-table-id=14
+|  |  |  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=32,34,33 row-size=28B cardinality=2.35M cost=2354541
+|  |  |  |  |  in pipelines: 38(GETNEXT), 39(OPEN)
+|  |  |  |  |
+|  |  |  |  |--F50:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  |  |  JOIN BUILD
+|  |  |  |  |  |  join-table-id=14 plan-id=15 cohort-id=08
+|  |  |  |  |  |  build expressions: c_customer_sk
+|  |  |  |  |  |  runtime filters: RF033[min_max] <- c_customer_sk
+|  |  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  |  |
+|  |  |  |  |  82:EXCHANGE [BROADCAST]
+|  |  |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  |  tuple-ids=33 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  |  |  in pipelines: 39(GETNEXT)
+|  |  |  |  |  |
+|  |  |  |  |  F27:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  |  |  39:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |  |  |     stored statistics:
+|  |  |  |  |       table: rows=100.00K size=5.49MB
+|  |  |  |  |       columns: all
+|  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |  |     tuple-ids=33 row-size=4B cardinality=100.00K cost=391
+|  |  |  |  |     in pipelines: 39(GETNEXT)
+|  |  |  |  |
+|  |  |  |  41:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  |  hash-table-id=15
+|  |  |  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  |  tuple-ids=32,34 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  |  |  in pipelines: 38(GETNEXT), 40(OPEN)
+|  |  |  |  |
+|  |  |  |  |--F51:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  |  max-parallelism=3 fragment-costs=[1545]
+|  |  |  |  |  JOIN BUILD
+|  |  |  |  |  |  join-table-id=15 plan-id=16 cohort-id=08
+|  |  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  |  runtime filters: RF034[bloom] <- d_date_sk
+|  |  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |  |
+|  |  |  |  |  81:EXCHANGE [BROADCAST]
+|  |  |  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  |  tuple-ids=34 row-size=8B cardinality=1.49K cost=54
+|  |  |  |  |  |  in pipelines: 40(GETNEXT)
+|  |  |  |  |  |
+|  |  |  |  |  F26:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  |  max-parallelism=1 fragment-costs=[73638]
+|  |  |  |  |  40:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |  |     stored statistics:
+|  |  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |  |       columns: all
+|  |  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |  |     tuple-ids=34 row-size=8B cardinality=1.49K cost=73620
+|  |  |  |  |     in pipelines: 40(GETNEXT)
+|  |  |  |  |
+|  |  |  |  38:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |  |  |     runtime filters: RF033[min_max] -> ss_customer_sk, RF034[bloom] -> ss_sold_date_sk
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=2.88M size=200.96MB
+|  |  |  |       partitions: 1824/1824 rows=2.88M
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |  |  |     tuple-ids=32 row-size=16B cardinality=2.88M cost=45007
+|  |  |  |     in pipelines: 38(GETNEXT)
+|  |  |  |
+|  |  |  45:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=16
+|  |  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=30,31 row-size=16B cardinality=2.88M cost=2880404
+|  |  |  |  in pipelines: 36(GETNEXT), 37(OPEN)
+|  |  |  |
+|  |  |  |--F52:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=16 plan-id=17 cohort-id=07
+|  |  |  |  |  build expressions: c_customer_sk
+|  |  |  |  |  runtime filters: RF031[min_max] <- c_customer_sk
+|  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  |
+|  |  |  |  80:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=31 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  |  in pipelines: 37(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F24:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  |  37:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |  |     runtime filters: RF025[min_max] -> tpcds_parquet.customer.c_customer_sk
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=100.00K size=5.49MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=31 row-size=4B cardinality=100.00K cost=391
+|  |  |  |     in pipelines: 37(GETNEXT)
+|  |  |  |
+|  |  |  36:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |  |     runtime filters: RF025[min_max] -> tpcds_parquet.store_sales.ss_customer_sk, RF031[min_max] -> ss_customer_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=2.88M size=200.96MB
+|  |  |       partitions: 1824/1824 rows=2.88M
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |  |     tuple-ids=30 row-size=12B cardinality=2.88M cost=33755
+|  |  |     in pipelines: 36(GETNEXT)
+|  |  |
+|  |  48:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=17
+|  |  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=21,23 row-size=36B cardinality=42.85K cost=719384
+|  |  |  in pipelines: 27(GETNEXT), 29(OPEN)
+|  |  |
+|  |  |--F53:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=2 fragment-costs=[112]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=17 plan-id=18 cohort-id=06
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF028[bloom] <- d_date_sk, RF029[min_max] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |  |
+|  |  |  79:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=23 row-size=12B cardinality=108 cost=4
+|  |  |  |  in pipelines: 29(GETNEXT)
+|  |  |  |
+|  |  |  F22:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[146957]
+|  |  |  29:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=23 row-size=12B cardinality=108 cost=146955
+|  |  |     in pipelines: 29(GETNEXT)
+|  |  |
+|  |  27:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |     runtime filters: RF029[min_max] -> ws_sold_date_sk, RF027[min_max] -> ws_bill_customer_sk, RF025[min_max] -> ws_bill_customer_sk, RF028[bloom] -> ws_sold_date_sk, RF026[bloom] -> ws_bill_customer_sk
+|  |     stored statistics:
+|  |       table: rows=719.38K size=45.09MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
+|  |     tuple-ids=21 row-size=24B cardinality=719.38K cost=16861
+|  |     in pipelines: 27(GETNEXT)
+|  |
+|  78:AGGREGATE [FINALIZE]
+|  |  output: count:merge(*)
+|  |  group by: itemdesc, i_item_sk, d_date
+|  |  having: count(*) > CAST(4 AS BIGINT)
+|  |  mem-estimate=46.41MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=28 row-size=50B cardinality=235.45K cost=9418164
+|  |  in pipelines: 78(GETNEXT), 30(OPEN)
+|  |
+|  77:EXCHANGE [HASH(itemdesc,i_item_sk,d_date)]
+|  |  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=28 row-size=50B cardinality=2.35M cost=124166
+|  |  in pipelines: 30(GETNEXT)
+|  |
+|  F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=33.63MB mem-reservation=17.50MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[14686864, 124166]
+|  35:AGGREGATE [STREAMING]
+|  |  output: count(*)
+|  |  group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_parquet.item.i_item_sk, d_date
+|  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  tuple-ids=28 row-size=50B cardinality=2.35M cost=9418164
+|  |  in pipelines: 30(GETNEXT)
+|  |
+|  34:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=18
+|  |  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  |  fk/pk conjuncts: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=24,25,26 row-size=162B cardinality=2.35M cost=2354541
+|  |  in pipelines: 30(GETNEXT), 32(OPEN)
+|  |
+|  |--F54:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[24564]
+|  |  JOIN BUILD
+|  |  |  join-table-id=18 plan-id=19 cohort-id=05
+|  |  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  |  runtime filters: RF021[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  76:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=26 row-size=120B cardinality=18.00K cost=6564
+|  |  |  in pipelines: 32(GETNEXT)
+|  |  |
+|  |  F19:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[4306]
+|  |  32:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     runtime filters: RF019[min_max] -> tpcds_parquet.item.i_item_sk, RF018[bloom] -> tpcds_parquet.item.i_item_sk
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |     tuple-ids=26 row-size=120B cardinality=18.00K cost=2118
+|  |     in pipelines: 32(GETNEXT)
+|  |
+|  33:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=19
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=24,25 row-size=42B cardinality=2.35M cost=2880404
+|  |  in pipelines: 30(GETNEXT), 31(OPEN)
+|  |
+|  |--F55:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[1641]
+|  |  JOIN BUILD
+|  |  |  join-table-id=19 plan-id=20 cohort-id=05
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF022[bloom] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |
+|  |  75:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=25 row-size=30B cardinality=1.49K cost=150
+|  |  |  in pipelines: 31(GETNEXT)
+|  |  |
+|  |  F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[75240]
+|  |  31:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|  |     tuple-ids=25 row-size=30B cardinality=1.49K cost=75190
+|  |     in pipelines: 31(GETNEXT)
+|  |
+|  30:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|     runtime filters: RF019[min_max] -> tpcds_parquet.store_sales.ss_item_sk, RF021[min_max] -> ss_item_sk, RF018[bloom] -> tpcds_parquet.store_sales.ss_item_sk, RF022[bloom] -> ss_sold_date_sk
+|     stored statistics:
+|       table: rows=2.88M size=200.96MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=24 row-size=12B cardinality=2.88M cost=33755
+|     in pipelines: 30(GETNEXT)
+|
+74:AGGREGATE [FINALIZE]
+|  output: sum:merge(cs_quantity * cs_list_price)
+|  group by: c_last_name, c_first_name
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=20 row-size=52B cardinality=10.53K cost=31578
+|  in pipelines: 74(GETNEXT), 57(OPEN)
+|
+73:EXCHANGE [HASH(c_last_name,c_first_name)]
+|  mem-estimate=345.95KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=20 row-size=52B cardinality=10.53K cost=576
+|  in pipelines: 57(GETNEXT)
+|
+F03:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=67.04MB mem-reservation=36.00MB thread-reservation=1
+max-parallelism=3 fragment-costs=[9542330, 267032, 576]
+26:AGGREGATE [STREAMING]
+|  output: sum(CAST(cs_quantity AS DECIMAL(10,0)) * cs_list_price)
+|  group by: c_last_name, c_first_name
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=20 row-size=52B cardinality=10.53K cost=31578
+|  in pipelines: 57(GETNEXT)
+|
+25:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=00
+|  hash predicates: i_item_sk = cs_item_sk
+|  fk/pk conjuncts: i_item_sk = cs_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  tuple-ids=7,0,2,18,1 row-size=162B cardinality=10.53K cost=235454
+|  in pipelines: 57(GETNEXT), 01(OPEN)
+|
+|--F36:PLAN FRAGMENT [HASH(itemdesc,i_item_sk,d_date)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=14.00MB mem-reservation=12.50MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[14474]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: cs_item_sk
+|  |  runtime filters: RF000[bloom] <- cs_item_sk, RF001[min_max] <- cs_item_sk
+|  |  mem-estimate=11.50MB mem-reservation=11.50MB spill-buffer=128.00KB thread-reservation=0 cost=10526
+|  |
+|  72:EXCHANGE [BROADCAST]
+|  |  mem-estimate=1.50MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,18,1 row-size=112B cardinality=10.53K cost=3948
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F15:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+|  Per-Instance Resources: mem-estimate=1010.56KB mem-reservation=0B thread-reservation=1
+|  max-parallelism=3 fragment-costs=[12706]
+|  24:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=01
+|  |  hash predicates: cs_bill_customer_sk = customer.c_customer_sk
+|  |  fk/pk conjuncts: cs_bill_customer_sk = customer.c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=0,2,18,1 row-size=112B cardinality=10.53K cost=10526
+|  |  in pipelines: 01(GETNEXT), 02(OPEN)
+|  |
+|  |--F37:PLAN FRAGMENT [HASH(cs_bill_customer_sk)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.60MB mem-reservation=4.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[104294]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  build expressions: customer.c_customer_sk
+|  |  |  runtime filters: RF007[min_max] <- customer.c_customer_sk
+|  |  |  mem-estimate=4.75MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |
+|  |  71:EXCHANGE [HASH(customer.c_customer_sk)]
+|  |  |  mem-estimate=3.85MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=1 row-size=40B cardinality=100.00K cost=4294
+|  |  |  in pipelines: 02(GETNEXT)
+|  |  |
+|  |  F14:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.52MB mem-reservation=1.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[8197]
+|  |  02:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |     stored statistics:
+|  |       table: rows=100.00K size=5.49MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=1 row-size=40B cardinality=100.00K cost=3903
+|  |     in pipelines: 02(GETNEXT)
+|  |
+|  70:EXCHANGE [HASH(cs_bill_customer_sk)]
+|  |  mem-estimate=498.70KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=0,2,18 row-size=72B cardinality=10.53K cost=864
+|  |  in pipelines: 01(GETNEXT)
+|  |
+|  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=48.98MB mem-reservation=8.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[1561508]
+|  23:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=02
+|  |  hash predicates: cs_bill_customer_sk = c_customer_sk
+|  |  fk/pk conjuncts: cs_bill_customer_sk = c_customer_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=0,2,18 row-size=72B cardinality=10.53K cost=85309
+|  |  in pipelines: 01(GETNEXT), 68(OPEN)
+|  |
+|  |--F38:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=9.21MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[11173]
+|  |  JOIN BUILD
+|  |  |  join-table-id=02 plan-id=03 cohort-id=02
+|  |  |  build expressions: c_customer_sk
+|  |  |  runtime filters: RF008[bloom] <- c_customer_sk, RF009[min_max] <- c_customer_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=10000
+|  |  |
+|  |  69:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=471.56KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=36B cardinality=10.00K cost=1173
+|  |  |  in pipelines: 68(GETNEXT)
+|  |  |
+|  |  F13:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=11.61MB mem-reservation=2.88MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[303907, 391]
+|  |  68:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge(ss_quantity * ss_sales_price), max:merge(tpcds_cmax)
+|  |  |  group by: c_customer_sk
+|  |  |  having: sum(ss_quantity * ss_sales_price) > CAST(0.500000 AS DECIMAL(10,6)) * max(tpcds_cmax)
+|  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  tuple-ids=18 row-size=36B cardinality=10.00K cost=300000
+|  |  |  in pipelines: 68(GETNEXT), 10(OPEN)
+|  |  |
+|  |  67:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  mem-estimate=1.61MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=36B cardinality=100.00K cost=3907
+|  |  |  in pipelines: 10(GETNEXT)
+|  |  |
+|  |  F06:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=26.47MB mem-reservation=10.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[11555371, 3907]
+|  |  21:AGGREGATE [STREAMING]
+|  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price), max(max(csales))
+|  |  |  group by: c_customer_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
+|  |  |  tuple-ids=18 row-size=36B cardinality=100.00K cost=8641212
+|  |  |  in pipelines: 10(GETNEXT)
+|  |  |
+|  |  20:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
+|  |  |  join table id: 03
+|  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=9,10,16 row-size=32B cardinality=2.88M cost=0
+|  |  |  in pipelines: 10(GETNEXT), 65(OPEN)
+|  |  |
+|  |  |--F39:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=16.02KB mem-reservation=0B thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[3]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
+|  |  |  |  mem-estimate=16B mem-reservation=0B thread-reservation=0 cost=0
+|  |  |  |
+|  |  |  66:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=16 row-size=16B cardinality=1 cost=3
+|  |  |  |  in pipelines: 65(GETNEXT)
+|  |  |  |
+|  |  |  F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[2, 1]
+|  |  |  65:AGGREGATE [FINALIZE]
+|  |  |  |  output: max:merge(csales)
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  tuple-ids=16 row-size=16B cardinality=1 cost=1
+|  |  |  |  in pipelines: 65(GETNEXT), 18(OPEN)
+|  |  |  |
+|  |  |  64:EXCHANGE [UNPARTITIONED]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=16 row-size=16B cardinality=1 cost=1
+|  |  |  |  in pipelines: 18(GETNEXT)
+|  |  |  |
+|  |  |  F11:PLAN FRAGMENT [HASH(c_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  |  Per-Instance Resources: mem-estimate=10.92MB mem-reservation=2.88MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[202344, 100000, 1]
+|  |  |  18:AGGREGATE
+|  |  |  |  output: max(sum(ss_quantity * ss_sales_price))
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
+|  |  |  |  tuple-ids=16 row-size=16B cardinality=1 cost=100000
+|  |  |  |  in pipelines: 18(GETNEXT), 63(OPEN)
+|  |  |  |
+|  |  |  63:AGGREGATE [FINALIZE]
+|  |  |  |  output: sum:merge(ss_quantity * ss_sales_price)
+|  |  |  |  group by: c_customer_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=2.88MB spill-buffer=128.00KB thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=100.00K cost=200000
+|  |  |  |  in pipelines: 63(GETNEXT), 12(OPEN)
+|  |  |  |
+|  |  |  62:EXCHANGE [HASH(c_customer_sk)]
+|  |  |  |  mem-estimate=939.04KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=100.00K cost=2344
+|  |  |  |  in pipelines: 12(GETNEXT)
+|  |  |  |
+|  |  |  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  |  Per-Instance Resources: mem-estimate=26.28MB mem-reservation=6.00MB thread-reservation=1
+|  |  |  max-parallelism=12 fragment-costs=[9989034, 2344]
+|  |  |  17:AGGREGATE [STREAMING]
+|  |  |  |  output: sum(CAST(ss_quantity AS DECIMAL(10,0)) * ss_sales_price)
+|  |  |  |  group by: c_customer_sk
+|  |  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=14 row-size=20B cardinality=100.00K cost=4709082
+|  |  |  |  in pipelines: 12(GETNEXT)
+|  |  |  |
+|  |  |  16:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=04
+|  |  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  |  tuple-ids=11,13,12 row-size=28B cardinality=2.35M cost=2354541
+|  |  |  |  in pipelines: 12(GETNEXT), 13(OPEN)
+|  |  |  |
+|  |  |  |--F40:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=04 plan-id=05 cohort-id=04
+|  |  |  |  |  build expressions: c_customer_sk
+|  |  |  |  |  runtime filters: RF015[min_max] <- c_customer_sk
+|  |  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |  |
+|  |  |  |  61:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=12 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  |  in pipelines: 13(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  |  13:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=100.00K size=5.49MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=12 row-size=4B cardinality=100.00K cost=391
+|  |  |  |     in pipelines: 13(GETNEXT)
+|  |  |  |
+|  |  |  15:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  |  hash-table-id=05
+|  |  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  |  tuple-ids=11,13 row-size=24B cardinality=2.35M cost=2880404
+|  |  |  |  in pipelines: 12(GETNEXT), 14(OPEN)
+|  |  |  |
+|  |  |  |--F41:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  |  max-parallelism=3 fragment-costs=[1545]
+|  |  |  |  JOIN BUILD
+|  |  |  |  |  join-table-id=05 plan-id=06 cohort-id=04
+|  |  |  |  |  build expressions: d_date_sk
+|  |  |  |  |  runtime filters: RF016[bloom] <- d_date_sk
+|  |  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |  |  |  |
+|  |  |  |  60:EXCHANGE [BROADCAST]
+|  |  |  |  |  mem-estimate=23.65KB mem-reservation=0B thread-reservation=0
+|  |  |  |  |  tuple-ids=13 row-size=8B cardinality=1.49K cost=54
+|  |  |  |  |  in pipelines: 14(GETNEXT)
+|  |  |  |  |
+|  |  |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  |  max-parallelism=1 fragment-costs=[73638]
+|  |  |  |  14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |  |     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     stored statistics:
+|  |  |  |       table: rows=73.05K size=2.15MB
+|  |  |  |       columns: all
+|  |  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |  |     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|  |  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |  |     tuple-ids=13 row-size=8B cardinality=1.49K cost=73620
+|  |  |  |     in pipelines: 14(GETNEXT)
+|  |  |  |
+|  |  |  12:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |  |     runtime filters: RF015[min_max] -> ss_customer_sk, RF016[bloom] -> ss_sold_date_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=2.88M size=200.96MB
+|  |  |       partitions: 1824/1824 rows=2.88M
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |  |     tuple-ids=11 row-size=16B cardinality=2.88M cost=45007
+|  |  |     in pipelines: 12(GETNEXT)
+|  |  |
+|  |  19:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=06
+|  |  |  hash predicates: ss_customer_sk = c_customer_sk
+|  |  |  fk/pk conjuncts: ss_customer_sk = c_customer_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=9,10 row-size=16B cardinality=2.88M cost=2880404
+|  |  |  in pipelines: 10(GETNEXT), 11(OPEN)
+|  |  |
+|  |  |--F42:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=19.39MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[102346]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=06 plan-id=07 cohort-id=03
+|  |  |  |  build expressions: c_customer_sk
+|  |  |  |  runtime filters: RF013[min_max] <- c_customer_sk
+|  |  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=100000
+|  |  |  |
+|  |  |  59:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=398.62KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=10 row-size=4B cardinality=100.00K cost=2346
+|  |  |  |  in pipelines: 11(GETNEXT)
+|  |  |  |
+|  |  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.03MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[1173]
+|  |  |  11:SCAN HDFS [tpcds_parquet.customer, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=5.49MB
+|  |  |     runtime filters: RF007[min_max] -> tpcds_parquet.customer.c_customer_sk
+|  |  |     stored statistics:
+|  |  |       table: rows=100.00K size=5.49MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=100.00K
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=10 row-size=4B cardinality=100.00K cost=391
+|  |  |     in pipelines: 11(GETNEXT)
+|  |  |
+|  |  10:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF007[min_max] -> tpcds_parquet.store_sales.ss_customer_sk, RF013[min_max] -> ss_customer_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=9 row-size=12B cardinality=2.88M cost=33755
+|  |     in pipelines: 10(GETNEXT)
+|  |
+|  22:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=07
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=0,2 row-size=36B cardinality=85.31K cost=1441548
+|  |  in pipelines: 01(GETNEXT), 03(OPEN)
+|  |
+|  |--F43:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=07 plan-id=08 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  58:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=2 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 03(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2000 AS INT), d_moy = CAST(2 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=2 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 03(GETNEXT)
+|  |
+|  01:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     runtime filters: RF011[min_max] -> cs_sold_date_sk, RF009[min_max] -> cs_bill_customer_sk, RF007[min_max] -> cs_bill_customer_sk, RF010[bloom] -> cs_sold_date_sk, RF008[bloom] -> cs_bill_customer_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
+|     tuple-ids=0 row-size=24B cardinality=1.44M cost=33787
+|     in pipelines: 01(GETNEXT)
+|
+57:AGGREGATE [FINALIZE]
+|  output: count:merge(*)
+|  group by: itemdesc, i_item_sk, d_date
+|  having: count(*) > CAST(4 AS BIGINT)
+|  mem-estimate=46.41MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=7 row-size=50B cardinality=235.45K cost=9418164
+|  in pipelines: 57(GETNEXT), 04(OPEN)
+|
+56:EXCHANGE [HASH(itemdesc,i_item_sk,d_date)]
+|  mem-estimate=10.63MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=7 row-size=50B cardinality=2.35M cost=124166
+|  in pipelines: 04(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=33.63MB mem-reservation=17.50MB thread-reservation=1
+max-parallelism=12 fragment-costs=[14686864, 124166]
+09:AGGREGATE [STREAMING]
+|  output: count(*)
+|  group by: substring(i_item_desc, CAST(1 AS BIGINT), CAST(30 AS BIGINT)), tpcds_parquet.item.i_item_sk, d_date
+|  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  tuple-ids=7 row-size=50B cardinality=2.35M cost=9418164
+|  in pipelines: 04(GETNEXT)
+|
+08:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=08
+|  hash predicates: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  fk/pk conjuncts: ss_item_sk = tpcds_parquet.item.i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=3,4,5 row-size=162B cardinality=2.35M cost=2354541
+|  in pipelines: 04(GETNEXT), 06(OPEN)
+|
+|--F44:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=21.19MB mem-reservation=19.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[24564]
+|  JOIN BUILD
+|  |  join-table-id=08 plan-id=09 cohort-id=01
+|  |  build expressions: tpcds_parquet.item.i_item_sk
+|  |  runtime filters: RF003[min_max] <- tpcds_parquet.item.i_item_sk
+|  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=18000
+|  |
+|  55:EXCHANGE [BROADCAST]
+|  |  mem-estimate=2.19MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=5 row-size=120B cardinality=18.00K cost=6564
+|  |  in pipelines: 06(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.49MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[4306]
+|  06:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     runtime filters: RF001[min_max] -> tpcds_parquet.item.i_item_sk, RF000[bloom] -> tpcds_parquet.item.i_item_sk
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|     tuple-ids=5 row-size=120B cardinality=18.00K cost=2118
+|     in pipelines: 06(GETNEXT)
+|
+07:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=09
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3,4 row-size=42B cardinality=2.35M cost=2880404
+|  in pipelines: 04(GETNEXT), 05(OPEN)
+|
+|--F45:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.83MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[1641]
+|  JOIN BUILD
+|  |  join-table-id=09 plan-id=10 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF004[bloom] <- d_date_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=1491
+|  |
+|  54:EXCHANGE [BROADCAST]
+|  |  mem-estimate=77.68KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=4 row-size=30B cardinality=1.49K cost=150
+|  |  in pipelines: 05(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.13MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[75240]
+|  05:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|     parquet dictionary predicates: d_year IN (CAST(2000 AS INT), CAST(2001 AS INT), CAST(2002 AS INT), CAST(2003 AS INT))
+|     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|     tuple-ids=4 row-size=30B cardinality=1.49K cost=75190
+|     in pipelines: 05(GETNEXT)
+|
+04:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=200.96MB
+   runtime filters: RF001[min_max] -> tpcds_parquet.store_sales.ss_item_sk, RF003[min_max] -> ss_item_sk, RF000[bloom] -> tpcds_parquet.store_sales.ss_item_sk, RF004[bloom] -> ss_sold_date_sk
+   stored statistics:
+     table: rows=2.88M size=200.96MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+   tuple-ids=3 row-size=12B cardinality=2.88M cost=33755
+   in pipelines: 04(GETNEXT)
+====
+# TPCDS-Q49
+SELECT channel,
+       item,
+       return_ratio,
+       return_rank,
+       currency_rank
+FROM
+  (SELECT 'web' AS channel,
+          web.item,
+          web.return_ratio,
+          web.return_rank,
+          web.currency_rank
+   FROM
+     (SELECT item,
+             return_ratio,
+             currency_ratio,
+             rank() OVER (
+                          ORDER BY return_ratio) AS return_rank,
+                         rank() OVER (
+                                      ORDER BY currency_ratio) AS currency_rank
+      FROM
+        (SELECT ws.ws_item_sk AS item,
+                (cast(sum(coalesce(wr.wr_return_quantity,0)) AS decimal(15,4))/ cast(sum(coalesce(ws.ws_quantity,0)) AS decimal(15,4))) AS return_ratio,
+                (cast(sum(coalesce(wr.wr_return_amt,0)) AS decimal(15,4))/ cast(sum(coalesce(ws.ws_net_paid,0)) AS decimal(15,4))) AS currency_ratio
+         FROM web_sales ws
+         LEFT OUTER JOIN web_returns wr ON (ws.ws_order_number = wr.wr_order_number
+                                            AND ws.ws_item_sk = wr.wr_item_sk) ,date_dim
+         WHERE wr.wr_return_amt > 10000
+           AND ws.ws_net_profit > 1
+           AND ws.ws_net_paid > 0
+           AND ws.ws_quantity > 0
+           AND ws_sold_date_sk = d_date_sk
+           AND d_year = 2001
+           AND d_moy = 12
+         GROUP BY ws.ws_item_sk) in_web) web
+   WHERE (web.return_rank <= 10
+          OR web.currency_rank <= 10)
+   UNION SELECT 'catalog' AS channel,
+                catalog.item,
+                catalog.return_ratio,
+                catalog.return_rank,
+                catalog.currency_rank
+   FROM
+     (SELECT item,
+             return_ratio,
+             currency_ratio,
+             rank() OVER (
+                          ORDER BY return_ratio) AS return_rank,
+                         rank() OVER (
+                                      ORDER BY currency_ratio) AS currency_rank
+      FROM
+        (SELECT cs.cs_item_sk AS item,
+                (cast(sum(coalesce(cr.cr_return_quantity,0)) AS decimal(15,4))/ cast(sum(coalesce(cs.cs_quantity,0)) AS decimal(15,4))) AS return_ratio,
+                (cast(sum(coalesce(cr.cr_return_amount,0)) AS decimal(15,4))/ cast(sum(coalesce(cs.cs_net_paid,0)) AS decimal(15,4))) AS currency_ratio
+         FROM catalog_sales cs
+         LEFT OUTER JOIN catalog_returns cr ON (cs.cs_order_number = cr.cr_order_number
+                                                AND cs.cs_item_sk = cr.cr_item_sk) ,date_dim
+         WHERE cr.cr_return_amount > 10000
+           AND cs.cs_net_profit > 1
+           AND cs.cs_net_paid > 0
+           AND cs.cs_quantity > 0
+           AND cs_sold_date_sk = d_date_sk
+           AND d_year = 2001
+           AND d_moy = 12
+         GROUP BY cs.cs_item_sk) in_cat) CATALOG
+   WHERE (catalog.return_rank <= 10
+          OR catalog.currency_rank <=10)
+   UNION SELECT 'store' AS channel,
+                store.item,
+                store.return_ratio,
+                store.return_rank,
+                store.currency_rank
+   FROM
+     (SELECT item,
+             return_ratio,
+             currency_ratio,
+             rank() OVER (
+                          ORDER BY return_ratio) AS return_rank,
+                         rank() OVER (
+                                      ORDER BY currency_ratio) AS currency_rank
+      FROM
+        (SELECT sts.ss_item_sk AS item,
+                (cast(sum(coalesce(sr.sr_return_quantity,0)) AS decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) AS decimal(15,4))) AS return_ratio,
+                (cast(sum(coalesce(sr.sr_return_amt,0)) AS decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) AS decimal(15,4))) AS currency_ratio
+         FROM store_sales sts
+         LEFT OUTER JOIN store_returns sr ON (sts.ss_ticket_number = sr.sr_ticket_number
+                                              AND sts.ss_item_sk = sr.sr_item_sk) ,date_dim
+         WHERE sr.sr_return_amt > 10000
+           AND sts.ss_net_profit > 1
+           AND sts.ss_net_paid > 0
+           AND sts.ss_quantity > 0
+           AND ss_sold_date_sk = d_date_sk
+           AND d_year = 2001
+           AND d_moy = 12
+         GROUP BY sts.ss_item_sk) in_store) store
+   WHERE (store.return_rank <= 10
+          OR store.currency_rank <= 10) ) sq1
+ORDER BY 1,
+         4,
+         5,
+         2
+LIMIT 100;
+
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=167.50MB Threads=22
+Per-Host Resource Estimates: Memory=451MB
+F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=24.61MB mem-reservation=15.94MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[8891, 17696, 35376, 119504, 400, 500]
+PLAN-ROOT SINK
+|  output exprs: channel, item, return_ratio, return_rank, currency_rank
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=500
+|
+35:TOP-N [LIMIT=100]
+|  order by: channel ASC, return_rank ASC, currency_rank ASC, item ASC
+|  mem-estimate=5.08KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=26 row-size=52B cardinality=100 cost=400
+|  in pipelines: 35(GETNEXT), 34(OPEN)
+|
+34:AGGREGATE [FINALIZE]
+|  group by: channel, item, return_ratio, return_rank, currency_rank
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=24 row-size=52B cardinality=2.99K cost=14940
+|  in pipelines: 34(GETNEXT), 09(OPEN), 20(OPEN), 31(OPEN)
+|
+00:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=24 row-size=52B cardinality=2.99K cost=44820
+|  in pipelines: 09(GETNEXT), 20(GETNEXT), 31(GETNEXT)
+|
+|--33:SELECT
+|  |  predicates: (rank() <= CAST(10 AS BIGINT) OR rank() <= CAST(10 AS BIGINT))
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=58,55 row-size=104B cardinality=1.71K cost=17055
+|  |  in pipelines: 31(GETNEXT)
+|  |
+|  32:ANALYTIC
+|  |  functions: rank()
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=58,55 row-size=104B cardinality=17.05K cost=17055
+|  |  in pipelines: 31(GETNEXT)
+|  |
+|  31:SORT
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  |  materialized: (CAST(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15,4)))
+|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=58 row-size=96B cardinality=17.05K cost=17055
+|  |  in pipelines: 31(GETNEXT), 29(OPEN)
+|  |
+|  30:ANALYTIC
+|  |  functions: rank()
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=56,53 row-size=80B cardinality=17.05K cost=17055
+|  |  in pipelines: 29(GETNEXT)
+|  |
+|  50:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  mem-estimate=627.73KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=56 row-size=72B cardinality=17.05K cost=1266
+|  |  in pipelines: 29(GETNEXT)
+|  |
+|  F13:PLAN FRAGMENT [HASH(sts.ss_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[86275, 17055, 1266]
+|  29:SORT
+|  |  order by: (CAST(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  materialized: (CAST(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15,4)))
+|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=56 row-size=72B cardinality=17.05K cost=17055
+|  |  in pipelines: 29(GETNEXT), 49(OPEN)
+|  |
+|  49:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(coalesce(sr.sr_return_quantity, 0)), sum:merge(coalesce(sts.ss_quantity, 0)), sum:merge(coalesce(sr.sr_return_amt, 0)), sum:merge(coalesce(sts.ss_net_paid, 0))
+|  |  group by: sts.ss_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=19 row-size=56B cardinality=17.05K cost=85275
+|  |  in pipelines: 49(GETNEXT), 23(OPEN)
+|  |
+|  48:EXCHANGE [HASH(sts.ss_item_sk)]
+|  |  mem-estimate=1.01MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=19 row-size=56B cardinality=17.05K cost=1000
+|  |  in pipelines: 23(GETNEXT)
+|  |
+|  F10:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=26.70MB mem-reservation=6.00MB thread-reservation=1
+|  max-parallelism=12 fragment-costs=[9680620, 1000]
+|  28:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(coalesce(sr.sr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(sts.ss_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(sr.sr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(sts.ss_net_paid, CAST(0 AS DECIMAL(7,2))))
+|  |  group by: sts.ss_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=19 row-size=56B cardinality=17.05K cost=85275
+|  |  in pipelines: 23(GETNEXT)
+|  |
+|  27:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=04
+|  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=16,17N,18 row-size=68B cardinality=17.05K cost=288040
+|  |  in pipelines: 23(GETNEXT), 25(OPEN)
+|  |
+|  |--F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=03
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF004[bloom] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  47:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=18 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 25(GETNEXT)
+|  |  |
+|  |  F12:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  25:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=18 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 25(GETNEXT)
+|  |
+|  26:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  |  hash-table-id=05
+|  |  hash predicates: sts.ss_item_sk = sr.sr_item_sk, sts.ss_ticket_number = sr.sr_ticket_number
+|  |  fk/pk conjuncts: sts.ss_item_sk = sr.sr_item_sk, sts.ss_ticket_number = sr.sr_ticket_number
+|  |  other predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=128.00KB thread-reservation=0
+|  |  tuple-ids=16,17N row-size=56B cardinality=288.04K cost=576080
+|  |  in pipelines: 23(GETNEXT), 24(OPEN)
+|  |
+|  |--F21:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=12.19MB mem-reservation=11.50MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[59863]
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=03
+|  |  |  build expressions: sr.sr_item_sk, sr.sr_ticket_number
+|  |  |  mem-estimate=11.50MB mem-reservation=11.50MB spill-buffer=128.00KB thread-reservation=0 cost=57502
+|  |  |
+|  |  46:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=701.85KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=17 row-size=24B cardinality=28.75K cost=2361
+|  |  |  in pipelines: 24(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[295040]
+|  |  24:SCAN HDFS [tpcds_parquet.store_returns sr, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=15.43MB
+|  |     predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  |     stored statistics:
+|  |       table: rows=287.51K size=15.43MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|  |     parquet statistics predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  |     parquet dictionary predicates: sr.sr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  |     mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=17 row-size=24B cardinality=28.75K cost=294253
+|  |     in pipelines: 24(GETNEXT)
+|  |
+|  23:SCAN HDFS [tpcds_parquet.store_sales sts, RANDOM]
+|     HDFS partitions=1824/1824 files=1824 size=199.44MB
+|     predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
+|     runtime filters: RF004[bloom] -> ss_sold_date_sk
+|     stored statistics:
+|       table: rows=2.88M size=199.44MB
+|       partitions: 1824/1824 rows=2.88M
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|     parquet statistics predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
+|     parquet dictionary predicates: sts.ss_net_paid > CAST(0 AS DECIMAL(3,0)), sts.ss_net_profit > CAST(1 AS DECIMAL(3,0)), sts.ss_quantity > CAST(0 AS INT)
+|     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
+|     tuple-ids=16 row-size=32B cardinality=288.04K cost=8731225
+|     in pipelines: 23(GETNEXT)
+|
+|--22:SELECT
+|  |  predicates: (rank() <= CAST(10 AS BIGINT) OR rank() <= CAST(10 AS BIGINT))
+|  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=47,44 row-size=104B cardinality=853 cost=8531
+|  |  in pipelines: 20(GETNEXT)
+|  |
+|  21:ANALYTIC
+|  |  functions: rank()
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=47,44 row-size=104B cardinality=8.53K cost=8531
+|  |  in pipelines: 20(GETNEXT)
+|  |
+|  20:SORT
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  |  materialized: (CAST(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15,4)))
+|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=47 row-size=96B cardinality=8.53K cost=8531
+|  |  in pipelines: 20(GETNEXT), 18(OPEN)
+|  |
+|  19:ANALYTIC
+|  |  functions: rank()
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=45,42 row-size=80B cardinality=8.53K cost=8531
+|  |  in pipelines: 18(GETNEXT)
+|  |
+|  45:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  mem-estimate=427.95KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=45 row-size=72B cardinality=8.53K cost=634
+|  |  in pipelines: 18(GETNEXT)
+|  |
+|  F08:PLAN FRAGMENT [HASH(cs.cs_item_sk)] hosts=3 instances=3
+|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[43155, 8531, 634]
+|  18:SORT
+|  |  order by: (CAST(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  |  materialized: (CAST(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15,4)))
+|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  |  tuple-ids=45 row-size=72B cardinality=8.53K cost=8531
+|  |  in pipelines: 18(GETNEXT), 44(OPEN)
+|  |
+|  44:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(coalesce(cr.cr_return_quantity, 0)), sum:merge(coalesce(cs.cs_quantity, 0)), sum:merge(coalesce(cr.cr_return_amount, 0)), sum:merge(coalesce(cs.cs_net_paid, 0))
+|  |  group by: cs.cs_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=11 row-size=56B cardinality=8.53K cost=42655
+|  |  in pipelines: 44(GETNEXT), 12(OPEN)
+|  |
+|  43:EXCHANGE [HASH(cs.cs_item_sk)]
+|  |  mem-estimate=335.51KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=11 row-size=56B cardinality=8.53K cost=500
+|  |  in pipelines: 12(GETNEXT)
+|  |
+|  F05:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=58.70MB mem-reservation=18.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[4844813, 500]
+|  17:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(coalesce(cr.cr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(cs.cs_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(cr.cr_return_amount, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(cs.cs_net_paid, CAST(0 AS DECIMAL(7,2))))
+|  |  group by: cs.cs_item_sk
+|  |  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=11 row-size=56B cardinality=8.53K cost=42655
+|  |  in pipelines: 12(GETNEXT)
+|  |
+|  16:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=02
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=8,9N,10 row-size=68B cardinality=8.53K cost=144155
+|  |  in pipelines: 12(GETNEXT), 14(OPEN)
+|  |
+|  |--F18:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[114]
+|  |  JOIN BUILD
+|  |  |  join-table-id=02 plan-id=03 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF002[bloom] <- d_date_sk, RF003[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |  |
+|  |  42:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=10 row-size=12B cardinality=108 cost=6
+|  |  |  in pipelines: 14(GETNEXT)
+|  |  |
+|  |  F07:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[146957]
+|  |  14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=10 row-size=12B cardinality=108 cost=146955
+|  |     in pipelines: 14(GETNEXT)
+|  |
+|  15:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  |  hash-table-id=03
+|  |  hash predicates: cs.cs_item_sk = cr.cr_item_sk, cs.cs_order_number = cr.cr_order_number
+|  |  fk/pk conjuncts: cs.cs_item_sk = cr.cr_item_sk, cs.cs_order_number = cr.cr_order_number
+|  |  other predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=8,9N row-size=56B cardinality=144.16K cost=288310
+|  |  in pipelines: 12(GETNEXT), 13(OPEN)
+|  |
+|  |--F19:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.11MB mem-reservation=7.75MB thread-reservation=1
+|  |  |  max-parallelism=3 fragment-costs=[29996]
+|  |  JOIN BUILD
+|  |  |  join-table-id=03 plan-id=04 cohort-id=02
+|  |  |  build expressions: cr.cr_item_sk, cr.cr_order_number
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=28814
+|  |  |
+|  |  41:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=365.66KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=9 row-size=24B cardinality=14.41K cost=1182
+|  |  |  in pipelines: 13(GETNEXT)
+|  |  |
+|  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=24.11MB mem-reservation=4.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[147838]
+|  |  13:SCAN HDFS [tpcds_parquet.catalog_returns cr, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=10.62MB
+|  |     predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
+|  |     stored statistics:
+|  |       table: rows=144.07K size=10.62MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=144.07K
+|  |     parquet statistics predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
+|  |     parquet dictionary predicates: cr.cr_return_amount > CAST(10000 AS DECIMAL(5,0))
+|  |     mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=9 row-size=24B cardinality=14.41K cost=147444
+|  |     in pipelines: 13(GETNEXT)
+|  |
+|  12:SCAN HDFS [tpcds_parquet.catalog_sales cs, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
+|     runtime filters: RF003[min_max] -> cs_sold_date_sk, RF002[bloom] -> cs_sold_date_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     parquet statistics predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
+|     parquet dictionary predicates: cs.cs_net_paid > CAST(0 AS DECIMAL(3,0)), cs.cs_net_profit > CAST(1 AS DECIMAL(3,0)), cs.cs_quantity > CAST(0 AS INT)
+|     mem-estimate=48.00MB mem-reservation=16.00MB thread-reservation=0
+|     tuple-ids=8 row-size=32B cardinality=144.16K cost=4369693
+|     in pipelines: 12(GETNEXT)
+|
+11:SELECT
+|  predicates: (rank() <= CAST(10 AS BIGINT) OR rank() <= CAST(10 AS BIGINT))
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=36,33 row-size=104B cardinality=429 cost=4286
+|  in pipelines: 09(GETNEXT)
+|
+10:ANALYTIC
+|  functions: rank()
+|  order by: (CAST(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=36,33 row-size=104B cardinality=4.29K cost=4286
+|  in pipelines: 09(GETNEXT)
+|
+09:SORT
+|  order by: (CAST(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15,4))) ASC
+|  materialized: (CAST(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15,4)))
+|  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=36 row-size=96B cardinality=4.29K cost=4286
+|  in pipelines: 09(GETNEXT), 07(OPEN)
+|
+08:ANALYTIC
+|  functions: rank()
+|  order by: (CAST(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=34,31 row-size=80B cardinality=4.29K cost=4286
+|  in pipelines: 07(GETNEXT)
+|
+40:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: (CAST(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  mem-estimate=302.68KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=34 row-size=72B cardinality=4.29K cost=319
+|  in pipelines: 07(GETNEXT)
+|
+F03:PLAN FRAGMENT [HASH(ws.ws_item_sk)] hosts=2 instances=2
+Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservation=1
+max-parallelism=2 fragment-costs=[21682, 4286, 319]
+07:SORT
+|  order by: (CAST(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15,4))) ASC
+|  materialized: (CAST(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15,4)) / CAST(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15,4)))
+|  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=34 row-size=72B cardinality=4.29K cost=4286
+|  in pipelines: 07(GETNEXT), 39(OPEN)
+|
+39:AGGREGATE [FINALIZE]
+|  output: sum:merge(coalesce(wr.wr_return_quantity, 0)), sum:merge(coalesce(ws.ws_quantity, 0)), sum:merge(coalesce(wr.wr_return_amt, 0)), sum:merge(coalesce(ws.ws_net_paid, 0))
+|  group by: ws.ws_item_sk
+|  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=4.29K cost=21430
+|  in pipelines: 39(GETNEXT), 01(OPEN)
+|
+38:EXCHANGE [HASH(ws.ws_item_sk)]
+|  mem-estimate=237.20KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=4.29K cost=252
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=42.47MB mem-reservation=18.00MB thread-reservation=1
+max-parallelism=2 fragment-costs=[2417877, 252]
+06:AGGREGATE [STREAMING]
+|  output: sum(CAST(coalesce(wr.wr_return_quantity, CAST(0 AS INT)) AS BIGINT)), sum(CAST(coalesce(ws.ws_quantity, CAST(0 AS INT)) AS BIGINT)), sum(coalesce(wr.wr_return_amt, CAST(0 AS DECIMAL(7,2)))), sum(coalesce(ws.ws_net_paid, CAST(0 AS DECIMAL(7,2))))
+|  group by: ws.ws_item_sk
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=3 row-size=56B cardinality=4.29K cost=21430
+|  in pipelines: 01(GETNEXT)
+|
+05:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=00
+|  hash predicates: ws_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N,2 row-size=68B cardinality=4.29K cost=71938
+|  in pipelines: 01(GETNEXT), 03(OPEN)
+|
+|--F16:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=2 fragment-costs=[112]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF000[bloom] <- d_date_sk, RF001[min_max] <- d_date_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=108
+|  |
+|  37:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=12B cardinality=108 cost=4
+|  |  in pipelines: 03(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.06MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[146957]
+|  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|     parquet dictionary predicates: d_year = CAST(2001 AS INT), d_moy = CAST(12 AS INT)
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=2 row-size=12B cardinality=108 cost=146955
+|     in pipelines: 03(GETNEXT)
+|
+04:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  hash-table-id=01
+|  hash predicates: ws.ws_item_sk = wr.wr_item_sk, ws.ws_order_number = wr.wr_order_number
+|  fk/pk conjuncts: ws.ws_item_sk = wr.wr_item_sk, ws.ws_order_number = wr.wr_order_number
+|  other predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1N row-size=56B cardinality=71.94K cost=143876
+|  in pipelines: 01(GETNEXT), 02(OPEN)
+|
+|--F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  Per-Instance Resources: mem-estimate=7.94MB mem-reservation=7.75MB thread-reservation=1
+|  |  max-parallelism=2 fragment-costs=[14746]
+|  JOIN BUILD
+|  |  join-table-id=01 plan-id=02 cohort-id=01
+|  |  build expressions: wr.wr_item_sk, wr.wr_order_number
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=14352
+|  |
+|  36:EXCHANGE [BROADCAST]
+|  |  mem-estimate=196.19KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1 row-size=24B cardinality=7.18K cost=394
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.11MB mem-reservation=2.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[73642]
+|  02:SCAN HDFS [tpcds_parquet.web_returns wr, RANDOM]
+|     HDFS partitions=1/1 files=1 size=5.66MB
+|     predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|     stored statistics:
+|       table: rows=71.76K size=5.66MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=71.76K
+|     parquet statistics predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|     parquet dictionary predicates: wr.wr_return_amt > CAST(10000 AS DECIMAL(5,0))
+|     mem-estimate=16.00MB mem-reservation=2.00MB thread-reservation=0
+|     tuple-ids=1 row-size=24B cardinality=7.18K cost=73445
+|     in pipelines: 02(GETNEXT)
+|
+01:SCAN HDFS [tpcds_parquet.web_sales ws, RANDOM]
+   HDFS partitions=1/1 files=2 size=45.09MB
+   predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
+   runtime filters: RF001[min_max] -> ws_sold_date_sk, RF000[bloom] -> ws_sold_date_sk
+   stored statistics:
+     table: rows=719.38K size=45.09MB
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=644.77K
+   parquet statistics predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
+   parquet dictionary predicates: ws.ws_net_paid > CAST(0 AS DECIMAL(3,0)), ws.ws_net_profit > CAST(1 AS DECIMAL(3,0)), ws.ws_quantity > CAST(0 AS INT)
+   mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
+   tuple-ids=0 row-size=32B cardinality=71.94K cost=2180633
+   in pipelines: 01(GETNEXT)
+====
+# TPCDS-Q76
+SELECT channel,
+       col_name,
+       d_year,
+       d_qoy,
+       i_category,
+       COUNT(*) sales_cnt,
+       SUM(ext_sales_price) sales_amt
+FROM
+  ( SELECT 'store' AS channel,
+           'ss_store_sk' col_name,
+                         d_year,
+                         d_qoy,
+                         i_category,
+                         ss_ext_sales_price ext_sales_price
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_store_sk IS NULL
+     AND ss_sold_date_sk=d_date_sk
+     AND ss_item_sk=i_item_sk
+   UNION ALL SELECT 'web' AS channel,
+                    'ws_ship_customer_sk' col_name,
+                                          d_year,
+                                          d_qoy,
+                                          i_category,
+                                          ws_ext_sales_price ext_sales_price
+   FROM web_sales,
+        item,
+        date_dim
+   WHERE ws_ship_customer_sk IS NULL
+     AND ws_sold_date_sk=d_date_sk
+     AND ws_item_sk=i_item_sk
+   UNION ALL SELECT 'catalog' AS channel,
+                    'cs_ship_addr_sk' col_name,
+                                      d_year,
+                                      d_qoy,
+                                      i_category,
+                                      cs_ext_sales_price ext_sales_price
+   FROM catalog_sales,
+        item,
+        date_dim
+   WHERE cs_ship_addr_sk IS NULL
+     AND cs_sold_date_sk=d_date_sk
+     AND cs_item_sk=i_item_sk) foo
+GROUP BY channel,
+         col_name,
+         d_year,
+         d_qoy,
+         i_category
+ORDER BY channel,
+         col_name,
+         d_year,
+         d_qoy,
+         i_category
+LIMIT 100;
+
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=92.62MB Threads=31
+Per-Host Resource Estimates: Memory=419MB
+F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[708]
+PLAN-ROOT SINK
+|  output exprs: channel, col_name, d_year, d_qoy, i_category, count(*), sum(ext_sales_price)
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=700
+|
+30:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: channel ASC, col_name ASC, d_year ASC, d_qoy ASC, i_category ASC
+|  limit: 100
+|  mem-estimate=25.23KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=12 row-size=74B cardinality=100 cost=8
+|  in pipelines: 17(GETNEXT)
+|
+F14:PLAN FRAGMENT [HASH(channel,col_name,d_year,d_qoy,i_category)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=14.14MB mem-reservation=4.75MB thread-reservation=1
+max-parallelism=3 fragment-costs=[972054, 500, 8]
+17:TOP-N [LIMIT=100]
+|  order by: channel ASC, col_name ASC, d_year ASC, d_qoy ASC, i_category ASC
+|  mem-estimate=7.22KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=12 row-size=74B cardinality=100 cost=500
+|  in pipelines: 17(GETNEXT), 29(OPEN)
+|
+29:AGGREGATE [FINALIZE]
+|  output: count:merge(*), sum:merge(ext_sales_price)
+|  group by: channel, col_name, d_year, d_qoy, i_category
+|  mem-estimate=10.00MB mem-reservation=4.75MB spill-buffer=256.00KB thread-reservation=0
+|  tuple-ids=11 row-size=74B cardinality=137.37K cost=961604
+|  in pipelines: 29(GETNEXT), 01(OPEN), 07(OPEN), 11(OPEN)
+|
+28:EXCHANGE [HASH(channel,col_name,d_year,d_qoy,i_category)]
+|  mem-estimate=4.14MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=11 row-size=74B cardinality=137.37K cost=10450
+|  in pipelines: 01(GETNEXT), 07(GETNEXT), 11(GETNEXT)
+|
+F13:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+Per-Instance Resources: mem-estimate=26.91MB mem-reservation=2.25MB thread-reservation=1
+max-parallelism=12 fragment-costs=[3597079, 10450]
+16:AGGREGATE [STREAMING]
+|  output: count(*), sum(ext_sales_price)
+|  group by: channel, col_name, d_year, d_qoy, i_category
+|  mem-estimate=10.00MB mem-reservation=2.00MB spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=11 row-size=74B cardinality=137.37K cost=961604
+|  in pipelines: 01(GETNEXT), 07(GETNEXT), 11(GETNEXT)
+|
+00:UNION
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=9 row-size=54B cardinality=137.37K cost=2472696
+|  in pipelines: 01(GETNEXT), 07(GETNEXT), 11(GETNEXT)
+|
+|--15:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=03
+|  |  hash predicates: cs_item_sk = i_item_sk
+|  |  fk/pk conjuncts: cs_item_sk = i_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=6,8,7 row-size=58B cardinality=7.17K cost=7165
+|  |  in pipelines: 11(GETNEXT), 12(OPEN)
+|  |
+|  |--F19:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  |  Per-Instance Resources: mem-estimate=3.41MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=12 fragment-costs=[18526]
+|  |  JOIN BUILD
+|  |  |  join-table-id=03 plan-id=04 cohort-id=01
+|  |  |  build expressions: i_item_sk
+|  |  |  runtime filters: RF008[bloom] <- i_item_sk, RF009[min_max] <- i_item_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |  |
+|  |  27:EXCHANGE [HASH(i_item_sk)]
+|  |  |  mem-estimate=485.07KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=7 row-size=26B cardinality=18.00K cost=526
+|  |  |  in pipelines: 12(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=17.40MB mem-reservation=256.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[982]
+|  |  12:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=1.73MB
+|  |     stored statistics:
+|  |       table: rows=18.00K size=1.73MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|  |     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|  |     tuple-ids=7 row-size=26B cardinality=18.00K cost=456
+|  |     in pipelines: 12(GETNEXT)
+|  |
+|  26:EXCHANGE [HASH(cs_item_sk)]
+|  |  mem-estimate=194.63KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=6,8 row-size=32B cardinality=7.17K cost=280
+|  |  in pipelines: 11(GETNEXT)
+|  |
+|  F10:PLAN FRAGMENT [HASH(cs_sold_date_sk)] hosts=3 instances=3
+|  Per-Instance Resources: mem-estimate=1.99MB mem-reservation=0B thread-reservation=1
+|  max-parallelism=3 fragment-costs=[7613]
+|  14:HASH JOIN [INNER JOIN, PARTITIONED]
+|  |  hash-table-id=05
+|  |  hash predicates: cs_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: none
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=6,8 row-size=32B cardinality=7.17K cost=7165
+|  |  in pipelines: 11(GETNEXT), 13(OPEN)
+|  |
+|  |--F21:PLAN FRAGMENT [HASH(cs_sold_date_sk)] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=3.79MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[74191]
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=01
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF010[bloom] <- d_date_sk, RF011[min_max] <- d_date_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=73049
+|  |  |
+|  |  25:EXCHANGE [HASH(d_date_sk)]
+|  |  |  mem-estimate=872.04KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=8 row-size=12B cardinality=73.05K cost=1142
+|  |  |  in pipelines: 13(GETNEXT)
+|  |  |
+|  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Instance Resources: mem-estimate=16.19MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[1999]
+|  |  13:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=8 row-size=12B cardinality=73.05K cost=857
+|  |     in pipelines: 13(GETNEXT)
+|  |
+|  24:EXCHANGE [HASH(cs_sold_date_sk)]
+|  |  mem-estimate=118.65KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=6 row-size=20B cardinality=7.17K cost=168
+|  |  in pipelines: 11(GETNEXT)
+|  |
+|  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+|  Per-Instance Resources: mem-estimate=48.28MB mem-reservation=8.00MB thread-reservation=1
+|  max-parallelism=3 fragment-costs=[1469872]
+|  11:SCAN HDFS [tpcds_parquet.catalog_sales, RANDOM]
+|     HDFS partitions=1/1 files=3 size=96.62MB
+|     predicates: cs_ship_addr_sk IS NULL
+|     runtime filters: RF009[min_max] -> cs_item_sk, RF011[min_max] -> cs_sold_date_sk, RF008[bloom] -> cs_item_sk, RF010[bloom] -> cs_sold_date_sk
+|     stored statistics:
+|       table: rows=1.44M size=96.62MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=650.14K
+|     mem-estimate=48.00MB mem-reservation=8.00MB thread-reservation=0
+|     tuple-ids=6 row-size=20B cardinality=7.17K cost=1469704
+|     in pipelines: 11(GETNEXT)
+|
+|--10:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=01
+|  |  hash predicates: i_item_sk = ws_item_sk
+|  |  fk/pk conjuncts: i_item_sk = ws_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=4,5,3 row-size=58B cardinality=173 cost=18000
+|  |  in pipelines: 07(GETNEXT), 08(OPEN)
+|  |
+|  |--F17:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=3 fragment-costs=[194]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=01
+|  |  |  build expressions: ws_item_sk
+|  |  |  runtime filters: RF004[bloom] <- ws_item_sk, RF005[min_max] <- ws_item_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=173
+|  |  |
+|  |  23:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=5,3 row-size=32B cardinality=173 cost=21
+|  |  |  in pipelines: 08(GETNEXT)
+|  |  |
+|  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.16MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[73913]
+|  |  09:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: d_date_sk = ws_sold_date_sk
+|  |  |  fk/pk conjuncts: none
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=5,3 row-size=32B cardinality=173 cost=73049
+|  |  |  in pipelines: 08(GETNEXT), 06(OPEN)
+|  |  |
+|  |  |--F18:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=1 fragment-costs=[178]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=02
+|  |  |  |  build expressions: ws_sold_date_sk
+|  |  |  |  runtime filters: RF006[bloom] <- ws_sold_date_sk, RF007[min_max] <- ws_sold_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=173
+|  |  |  |
+|  |  |  22:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=3 row-size=20B cardinality=173 cost=5
+|  |  |  |  in pipelines: 06(GETNEXT)
+|  |  |  |
+|  |  |  F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=32.09MB mem-reservation=8.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[733440]
+|  |  |  06:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|  |  |     HDFS partitions=1/1 files=2 size=45.09MB
+|  |  |     predicates: ws_ship_customer_sk IS NULL
+|  |  |     stored statistics:
+|  |  |       table: rows=719.38K size=45.09MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|  |  |     mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=0
+|  |  |     tuple-ids=3 row-size=20B cardinality=173 cost=733435
+|  |  |     in pipelines: 06(GETNEXT)
+|  |  |
+|  |  08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     runtime filters: RF007[min_max] -> d_date_sk, RF006[bloom] -> d_date_sk
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=5 row-size=12B cardinality=73.05K cost=857
+|  |     in pipelines: 08(GETNEXT)
+|  |
+|  07:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     runtime filters: RF005[min_max] -> i_item_sk, RF004[bloom] -> i_item_sk
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=4 row-size=26B cardinality=18.00K cost=456
+|     in pipelines: 07(GETNEXT)
+|
+05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: ss_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1,2 row-size=58B cardinality=130.03K cost=130034
+|  in pipelines: 01(GETNEXT), 03(OPEN)
+|
+|--F16:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Instance Resources: mem-estimate=3.79MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=12 fragment-costs=[74191]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF000[bloom] <- d_date_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=73049
+|  |
+|  21:EXCHANGE [HASH(d_date_sk)]
+|  |  mem-estimate=872.04KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=2 row-size=12B cardinality=73.05K cost=1142
+|  |  in pipelines: 03(GETNEXT)
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.75MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1999]
+|  03:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=2 row-size=12B cardinality=73.05K cost=857
+|     in pipelines: 03(GETNEXT)
+|
+20:EXCHANGE [HASH(ss_sold_date_sk)]
+|  mem-estimate=2.06MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0,1 row-size=46B cardinality=130.03K cost=6844
+|  in pipelines: 01(GETNEXT)
+|
+F02:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 (adjusted from 12)
+Per-Instance Resources: mem-estimate=3.63MB mem-reservation=0B thread-reservation=1
+max-parallelism=3 fragment-costs=[139926]
+04:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=04
+|  hash predicates: ss_item_sk = i_item_sk
+|  fk/pk conjuncts: ss_item_sk = i_item_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=0,1 row-size=46B cardinality=130.03K cost=130034
+|  in pipelines: 01(GETNEXT), 02(OPEN)
+|
+|--F20:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=3.41MB mem-reservation=2.94MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[18526]
+|  JOIN BUILD
+|  |  join-table-id=04 plan-id=05 cohort-id=01
+|  |  build expressions: i_item_sk
+|  |  runtime filters: RF002[bloom] <- i_item_sk, RF003[min_max] <- i_item_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=18000
+|  |
+|  19:EXCHANGE [HASH(i_item_sk)]
+|  |  mem-estimate=485.07KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=1 row-size=26B cardinality=18.00K cost=526
+|  |  in pipelines: 02(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=16.35MB mem-reservation=256.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[982]
+|  02:SCAN HDFS [tpcds_parquet.item, RANDOM]
+|     HDFS partitions=1/1 files=1 size=1.73MB
+|     stored statistics:
+|       table: rows=18.00K size=1.73MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=18.00K
+|     mem-estimate=16.00MB mem-reservation=256.00KB thread-reservation=0
+|     tuple-ids=1 row-size=26B cardinality=18.00K cost=456
+|     in pipelines: 02(GETNEXT)
+|
+18:EXCHANGE [HASH(ss_item_sk)]
+|  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=0 row-size=20B cardinality=130.03K cost=3048
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
+Per-Instance Resources: mem-estimate=16.28MB mem-reservation=1.00MB thread-reservation=1
+max-parallelism=12 fragment-costs=[2939710]
+01:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+   HDFS partitions=1824/1824 files=1824 size=199.44MB
+   predicates: ss_store_sk IS NULL
+   runtime filters: RF003[min_max] -> ss_item_sk, RF000[bloom] -> ss_sold_date_sk, RF002[bloom] -> ss_item_sk
+   stored statistics:
+     table: rows=2.88M size=199.44MB
+     partitions: 1824/1824 rows=2.88M
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=130.09K
+   mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+   tuple-ids=0 row-size=20B cardinality=130.03K cost=2936662
+   in pipelines: 01(GETNEXT)
+====
+# TPCDS-Q78
+WITH ws AS
+  (SELECT d_year AS ws_sold_year,
+          ws_item_sk,
+          ws_bill_customer_sk ws_customer_sk,
+          sum(ws_quantity) ws_qty,
+          sum(ws_wholesale_cost) ws_wc,
+          sum(ws_sales_price) ws_sp
+   FROM web_sales
+   LEFT JOIN web_returns ON wr_order_number=ws_order_number
+   AND ws_item_sk=wr_item_sk
+   JOIN date_dim ON ws_sold_date_sk = d_date_sk
+   WHERE wr_order_number IS NULL
+   GROUP BY d_year,
+            ws_item_sk,
+            ws_bill_customer_sk ),
+     cs AS
+  (SELECT d_year AS cs_sold_year,
+          cs_item_sk,
+          cs_bill_customer_sk cs_customer_sk,
+          sum(cs_quantity) cs_qty,
+          sum(cs_wholesale_cost) cs_wc,
+          sum(cs_sales_price) cs_sp
+   FROM catalog_sales
+   LEFT JOIN catalog_returns ON cr_order_number=cs_order_number
+   AND cs_item_sk=cr_item_sk
+   JOIN date_dim ON cs_sold_date_sk = d_date_sk
+   WHERE cr_order_number IS NULL
+   GROUP BY d_year,
+            cs_item_sk,
+            cs_bill_customer_sk ),
+     ss AS
+  (SELECT d_year AS ss_sold_year,
+          ss_item_sk,
+          ss_customer_sk,
+          sum(ss_quantity) ss_qty,
+          sum(ss_wholesale_cost) ss_wc,
+          sum(ss_sales_price) ss_sp
+   FROM store_sales
+   LEFT JOIN store_returns ON sr_ticket_number=ss_ticket_number
+   AND ss_item_sk=sr_item_sk
+   JOIN date_dim ON ss_sold_date_sk = d_date_sk
+   WHERE sr_ticket_number IS NULL
+   GROUP BY d_year,
+            ss_item_sk,
+            ss_customer_sk )
+SELECT ss_sold_year,
+       ss_item_sk,
+       ss_customer_sk,
+       round((ss_qty*1.00)/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio,
+       ss_qty store_qty,
+       ss_wc store_wholesale_cost,
+       ss_sp store_sales_price,
+       coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty,
+       coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost,
+       coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price
+FROM ss
+LEFT JOIN ws ON (ws_sold_year=ss_sold_year
+                 AND ws_item_sk=ss_item_sk
+                 AND ws_customer_sk=ss_customer_sk)
+LEFT JOIN cs ON (cs_sold_year=ss_sold_year
+                 AND cs_item_sk=ss_item_sk
+                 AND cs_customer_sk=ss_customer_sk)
+WHERE (coalesce(ws_qty,0)>0
+       OR coalesce(cs_qty, 0)>0)
+  AND ss_sold_year=2000
+ORDER BY ss_sold_year,
+         ss_item_sk,
+         ss_customer_sk,
+         ss_qty DESC,
+         ss_wc DESC,
+         ss_sp DESC,
+         other_chan_qty,
+         other_chan_wholesale_cost,
+         other_chan_sales_price,
+         ratio
+LIMIT 100;
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=296.62MB Threads=24
+Per-Host Resource Estimates: Memory=558MB
+F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=4.04MB mem-reservation=4.00MB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[1012]
+PLAN-ROOT SINK
+|  output exprs: ss_sold_year, ss_item_sk, ss_customer_sk, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2), ss_qty, ss_wc, ss_sp, coalesce(ws_qty, 0) + coalesce(cs_qty, 0), coalesce(ws_wc, 0) + coalesce(cs_wc, 0), coalesce(ws_sp, 0) + coalesce(cs_sp, 0)
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=1000
+|
+35:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: ss_sold_year ASC, ss_item_sk ASC, ss_customer_sk ASC, ss_qty DESC, ss_wc DESC, ss_sp DESC, coalesce(ws_qty, 0) + coalesce(cs_qty, 0) ASC, coalesce(ws_wc, 0) + coalesce(cs_wc, 0) ASC, coalesce(ws_sp, 0) + coalesce(cs_sp, 0) ASC, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2) ASC
+|  limit: 100
+|  mem-estimate=37.63KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=15 row-size=112B cardinality=100 cost=12
+|  in pipelines: 20(GETNEXT)
+|
+F03:PLAN FRAGMENT [HASH(d_year,cs_item_sk,cs_bill_customer_sk)] hosts=3 instances=3
+Per-Instance Resources: mem-estimate=15.42MB mem-reservation=8.50MB thread-reservation=1
+max-parallelism=3 fragment-costs=[1785038, 884887, 12]
+20:TOP-N [LIMIT=100]
+|  order by: ss_sold_year ASC, ss_item_sk ASC, ss_customer_sk ASC, ss_qty DESC, ss_wc DESC, ss_sp DESC, coalesce(ws_qty, 0) + coalesce(cs_qty, 0) ASC, coalesce(ws_wc, 0) + coalesce(cs_wc, 0) ASC, coalesce(ws_sp, 0) + coalesce(cs_sp, 0) ASC, round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2) ASC
+|  materialized: coalesce(ws_qty, 0) + coalesce(cs_qty, 0), coalesce(ws_wc, 0) + coalesce(cs_wc, 0), coalesce(ws_sp, 0) + coalesce(cs_sp, 0), round((ss_qty * 1.00) / (coalesce(ws_qty, 0) + coalesce(cs_qty, 0)), 2)
+|  mem-estimate=10.94KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=15 row-size=112B cardinality=100 cost=1000
+|  in pipelines: 20(GETNEXT), 24(OPEN)
+|
+19:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: d_year = d_year, cs_bill_customer_sk = ss_customer_sk, cs_item_sk = ss_item_sk
+|  fk/pk conjuncts: cs_bill_customer_sk = ss_customer_sk, cs_item_sk = ss_item_sk
+|  other predicates: (coalesce(sum(ws_quantity), CAST(0 AS BIGINT)) > CAST(0 AS BIGINT) OR coalesce(sum(cs_quantity), CAST(0 AS BIGINT)) > CAST(0 AS BIGINT))
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=13N,8N,3 row-size=168B cardinality=3.00K cost=883887
+|  in pipelines: 24(GETNEXT), 28(OPEN)
+|
+|--F13:PLAN FRAGMENT [HASH(d_year,cs_item_sk,cs_bill_customer_sk)] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=5.33MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=3.00MB
+|  |  max-parallelism=3 fragment-costs=[9368]
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: d_year, ss_customer_sk, ss_item_sk
+|  |  runtime filters: RF000[bloom] <- d_year, RF001[bloom] <- ss_customer_sk, RF002[bloom] <- ss_item_sk, RF004[min_max] <- ss_customer_sk, RF005[min_max] <- ss_item_sk
+|  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=9015
+|  |
+|  34:EXCHANGE [HASH(d_year,ss_item_sk,ss_customer_sk)]
+|  |  mem-estimate=404.34KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8N,3 row-size=112B cardinality=3.00K cost=353
+|  |  in pipelines: 28(GETNEXT)
+|  |
+|  F07:PLAN FRAGMENT [HASH(d_year,ws_item_sk,ws_bill_customer_sk)] hosts=2 instances=2
+|  Per-Instance Resources: mem-estimate=14.07MB mem-reservation=8.50MB thread-reservation=1
+|  max-parallelism=2 fragment-costs=[896697, 444365]
+|  18:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
+|  |  hash-table-id=01
+|  |  hash predicates: d_year = d_year, ws_bill_customer_sk = ss_customer_sk, ws_item_sk = ss_item_sk
+|  |  fk/pk conjuncts: ws_bill_customer_sk = ss_customer_sk, ws_item_sk = ss_item_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=8N,3 row-size=112B cardinality=3.00K cost=444012
+|  |  in pipelines: 28(GETNEXT), 32(OPEN)
+|  |
+|  |--F14:PLAN FRAGMENT [HASH(d_year,ws_item_sk,ws_bill_customer_sk)] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=5.17MB mem-reservation=4.94MB thread-reservation=1 runtime-filters-memory=3.00MB
+|  |  |  max-parallelism=2 fragment-costs=[9192]
+|  |  JOIN BUILD
+|  |  |  join-table-id=01 plan-id=02 cohort-id=02
+|  |  |  build expressions: d_year, ss_customer_sk, ss_item_sk
+|  |  |  runtime filters: RF008[bloom] <- d_year, RF009[bloom] <- ss_customer_sk, RF010[bloom] <- ss_item_sk, RF012[min_max] <- ss_customer_sk, RF013[min_max] <- ss_item_sk
+|  |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0 cost=9015
+|  |  |
+|  |  33:EXCHANGE [HASH(d_year,ss_item_sk,ss_customer_sk)]
+|  |  |  mem-estimate=234.78KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=56B cardinality=3.00K cost=177
+|  |  |  in pipelines: 32(GETNEXT)
+|  |  |
+|  |  F11:PLAN FRAGMENT [HASH(d_year,ss_item_sk,ss_customer_sk)] hosts=3 instances=3 (adjusted from 12)
+|  |  Per-Instance Resources: mem-estimate=27.70MB mem-reservation=17.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[3568694, 177]
+|  |  32:AGGREGATE [FINALIZE]
+|  |  |  output: sum:merge(ss_quantity), sum:merge(ss_wholesale_cost), sum:merge(ss_sales_price)
+|  |  |  group by: d_year, ss_item_sk, ss_customer_sk
+|  |  |  having: d_year = CAST(2000 AS INT)
+|  |  |  mem-estimate=17.00MB mem-reservation=17.00MB spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=56B cardinality=3.00K cost=3534180
+|  |  |  in pipelines: 32(GETNEXT), 00(OPEN)
+|  |  |
+|  |  31:EXCHANGE [HASH(d_year,ss_item_sk,ss_customer_sk)]
+|  |  |  mem-estimate=10.70MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=3 row-size=56B cardinality=589.03K cost=34514
+|  |  |  in pipelines: 00(GETNEXT)
+|  |  |
+|  |  F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=12
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=26.70MB mem-reservation=9.00MB thread-reservation=1
+|  |  max-parallelism=12 fragment-costs=[12276657, 34514]
+|  |  05:AGGREGATE [STREAMING]
+|  |  |  output: sum(CAST(ss_quantity AS BIGINT)), sum(ss_wholesale_cost), sum(ss_sales_price)
+|  |  |  group by: d_year, ss_item_sk, ss_customer_sk
+|  |  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  |  tuple-ids=3 row-size=56B cardinality=589.03K cost=3534180
+|  |  |  in pipelines: 00(GETNEXT)
+|  |  |
+|  |  04:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  |  hash-table-id=02
+|  |  |  hash predicates: ss_sold_date_sk = d_date_sk
+|  |  |  fk/pk conjuncts: ss_sold_date_sk = d_date_sk
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  |  tuple-ids=0,1N,2 row-size=60B cardinality=589.03K cost=2880404
+|  |  |  in pipelines: 00(GETNEXT), 02(OPEN)
+|  |  |
+|  |  |--F15:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  |  max-parallelism=3 fragment-costs=[388]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=02 plan-id=03 cohort-id=03
+|  |  |  |  build expressions: d_date_sk
+|  |  |  |  runtime filters: RF016[bloom] <- d_date_sk
+|  |  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |  |  |
+|  |  |  30:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=2 row-size=8B cardinality=373 cost=15
+|  |  |  |  in pipelines: 02(GETNEXT)
+|  |  |  |
+|  |  |  F10:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[73625]
+|  |  |  02:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |  |     stored statistics:
+|  |  |       table: rows=73.05K size=2.15MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |  |     tuple-ids=2 row-size=8B cardinality=373 cost=73620
+|  |  |     in pipelines: 02(GETNEXT)
+|  |  |
+|  |  03:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  |  |  hash-table-id=03
+|  |  |  hash predicates: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
+|  |  |  fk/pk conjuncts: ss_item_sk = sr_item_sk, ss_ticket_number = sr_ticket_number
+|  |  |  other predicates: sr_ticket_number IS NULL
+|  |  |  mem-estimate=0B mem-reservation=0B spill-buffer=1.00MB thread-reservation=0
+|  |  |  tuple-ids=0,1N row-size=52B cardinality=2.88M cost=5760808
+|  |  |  in pipelines: 00(GETNEXT), 01(OPEN)
+|  |  |
+|  |  |--F16:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  |  |  Per-Instance Resources: mem-estimate=72.41MB mem-reservation=68.00MB thread-reservation=1
+|  |  |  |  max-parallelism=3 fragment-costs=[591876]
+|  |  |  JOIN BUILD
+|  |  |  |  join-table-id=03 plan-id=04 cohort-id=03
+|  |  |  |  build expressions: sr_item_sk, sr_ticket_number
+|  |  |  |  mem-estimate=68.00MB mem-reservation=68.00MB spill-buffer=1.00MB thread-reservation=0 cost=575028
+|  |  |  |
+|  |  |  29:EXCHANGE [BROADCAST]
+|  |  |  |  mem-estimate=4.41MB mem-reservation=0B thread-reservation=0
+|  |  |  |  tuple-ids=1 row-size=16B cardinality=287.51K cost=16848
+|  |  |  |  in pipelines: 01(GETNEXT)
+|  |  |  |
+|  |  |  F09:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  |  Per-Instance Resources: mem-estimate=24.08MB mem-reservation=4.00MB thread-reservation=1
+|  |  |  max-parallelism=1 fragment-costs=[10109]
+|  |  |  01:SCAN HDFS [tpcds_parquet.store_returns, RANDOM]
+|  |  |     HDFS partitions=1/1 files=1 size=15.43MB
+|  |  |     stored statistics:
+|  |  |       table: rows=287.51K size=15.43MB
+|  |  |       columns: all
+|  |  |     extrapolated-rows=disabled max-scan-range-rows=287.51K
+|  |  |     mem-estimate=24.00MB mem-reservation=4.00MB thread-reservation=0
+|  |  |     tuple-ids=1 row-size=16B cardinality=287.51K cost=4493
+|  |  |     in pipelines: 01(GETNEXT)
+|  |  |
+|  |  00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
+|  |     HDFS partitions=1824/1824 files=1824 size=200.96MB
+|  |     runtime filters: RF016[bloom] -> ss_sold_date_sk
+|  |     stored statistics:
+|  |       table: rows=2.88M size=200.96MB
+|  |       partitions: 1824/1824 rows=2.88M
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
+|  |     mem-estimate=16.00MB mem-reservation=4.00MB thread-reservation=0
+|  |     tuple-ids=0 row-size=36B cardinality=2.88M cost=101265
+|  |     in pipelines: 00(GETNEXT)
+|  |
+|  28:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(ws_quantity), sum:merge(ws_wholesale_cost), sum:merge(ws_sales_price)
+|  |  group by: d_year, ws_item_sk, ws_bill_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  |  tuple-ids=8 row-size=56B cardinality=148.00K cost=888024
+|  |  in pipelines: 28(GETNEXT), 06(OPEN)
+|  |
+|  27:EXCHANGE [HASH(d_year,ws_item_sk,ws_bill_customer_sk)]
+|  |  mem-estimate=4.07MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=8 row-size=56B cardinality=148.00K cost=8673
+|  |  in pipelines: 06(GETNEXT)
+|  |
+|  F04:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+|  Per-Instance Resources: mem-estimate=42.47MB mem-reservation=21.00MB thread-reservation=1
+|  max-parallelism=2 fragment-costs=[3071467, 8673]
+|  11:AGGREGATE [STREAMING]
+|  |  output: sum(CAST(ws_quantity AS BIGINT)), sum(ws_wholesale_cost), sum(ws_sales_price)
+|  |  group by: d_year, ws_item_sk, ws_bill_customer_sk
+|  |  mem-estimate=10.00MB mem-reservation=5.00MB spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=8 row-size=56B cardinality=148.00K cost=888024
+|  |  in pipelines: 06(GETNEXT)
+|  |
+|  10:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash-table-id=04
+|  |  hash predicates: ws_sold_date_sk = d_date_sk
+|  |  fk/pk conjuncts: ws_sold_date_sk = d_date_sk
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  |  tuple-ids=5,6N,7 row-size=60B cardinality=148.00K cost=719384
+|  |  in pipelines: 06(GETNEXT), 08(OPEN)
+|  |
+|  |--F17:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  |  max-parallelism=2 fragment-costs=[383]
+|  |  JOIN BUILD
+|  |  |  join-table-id=04 plan-id=05 cohort-id=02
+|  |  |  build expressions: d_date_sk
+|  |  |  runtime filters: RF014[bloom] <- d_date_sk, RF015[min_max] <- d_date_sk
+|  |  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |  |
+|  |  26:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=7 row-size=8B cardinality=373 cost=10
+|  |  |  in pipelines: 08(GETNEXT)
+|  |  |
+|  |  F06:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[73625]
+|  |  08:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=2.15MB
+|  |     predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |     runtime filters: RF008[bloom] -> tpcds_parquet.date_dim.d_year
+|  |     stored statistics:
+|  |       table: rows=73.05K size=2.15MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|  |     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|  |     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|  |     tuple-ids=7 row-size=8B cardinality=373 cost=73620
+|  |     in pipelines: 08(GETNEXT)
+|  |
+|  09:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  |  hash-table-id=05
+|  |  hash predicates: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
+|  |  fk/pk conjuncts: ws_item_sk = wr_item_sk, ws_order_number = wr_order_number
+|  |  other predicates: wr_order_number IS NULL
+|  |  mem-estimate=0B mem-reservation=0B spill-buffer=256.00KB thread-reservation=0
+|  |  tuple-ids=5,6N row-size=52B cardinality=719.38K cost=1438768
+|  |  in pipelines: 06(GETNEXT), 07(OPEN)
+|  |
+|  |--F18:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  |  |  Per-Instance Resources: mem-estimate=20.11MB mem-reservation=19.00MB thread-reservation=1
+|  |  |  max-parallelism=2 fragment-costs=[146330]
+|  |  JOIN BUILD
+|  |  |  join-table-id=05 plan-id=06 cohort-id=02
+|  |  |  build expressions: wr_item_sk, wr_order_number
+|  |  |  mem-estimate=19.00MB mem-reservation=19.00MB spill-buffer=256.00KB thread-reservation=0 cost=143526
+|  |  |
+|  |  25:EXCHANGE [BROADCAST]
+|  |  |  mem-estimate=1.11MB mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=6 row-size=16B cardinality=71.76K cost=2804
+|  |  |  in pipelines: 07(GETNEXT)
+|  |  |
+|  |  F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  |  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  |  Per-Instance Resources: mem-estimate=16.08MB mem-reservation=1.00MB thread-reservation=1
+|  |  max-parallelism=1 fragment-costs=[2524]
+|  |  07:SCAN HDFS [tpcds_parquet.web_returns, RANDOM]
+|  |     HDFS partitions=1/1 files=1 size=5.66MB
+|  |     runtime filters: RF013[min_max] -> tpcds_parquet.web_returns.wr_item_sk, RF010[bloom] -> tpcds_parquet.web_returns.wr_item_sk
+|  |     stored statistics:
+|  |       table: rows=71.76K size=5.66MB
+|  |       columns: all
+|  |     extrapolated-rows=disabled max-scan-range-rows=71.76K
+|  |     mem-estimate=16.00MB mem-reservation=1.00MB thread-reservation=0
+|  |     tuple-ids=6 row-size=16B cardinality=71.76K cost=1122
+|  |     in pipelines: 07(GETNEXT)
+|  |
+|  06:SCAN HDFS [tpcds_parquet.web_sales, RANDOM]
+|     HDFS partitions=1/1 files=2 size=45.09MB
+|     runtime filters: RF012[min_max] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF013[min_max] -> tpcds_parquet.web_sales.ws_item_sk, RF015[min_max] -> ws_sold_date_sk, RF009[bloom] -> tpcds_parquet.web_sales.ws_bill_customer_sk, RF010[bloom] -> tpcds_parquet.web_sales.ws_item_sk, RF014[bloom] -> ws_sold_date_sk
+|     stored statistics:
+|       table: rows=719.38K size=45.09MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=644.77K
+|     mem-estimate=32.00MB mem-reservation=16.00MB thread-reservation=0
+|     tuple-ids=5 row-size=36B cardinality=719.38K cost=25291
+|     in pipelines: 06(GETNEXT)
+|
+24:AGGREGATE [FINALIZE]
+|  output: sum:merge(cs_quantity), sum:merge(cs_wholesale_cost), sum:merge(cs_sales_price)
+|  group by: d_year, cs_item_sk, cs_bill_customer_sk
+|  mem-estimate=10.00MB mem-reservation=8.50MB spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=13 row-size=56B cardinality=294.63K cost=1767774
+|  in pipelines: 24(GETNEXT), 12(OPEN)
+|
+23:EXCHANGE [HASH(d_year,cs_item_sk,cs_bill_customer_sk)]
+|  mem-estimate=5.42MB mem-reservation=0B thread-reservation=0
+|  tuple-ids=13 row-size=56B cardinality=294.63K cost=17264
+|  in pipelines: 12(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
+Per-Instance Resources: mem-estimate=58.70MB mem-reservation=25.00MB thread-reservation=1
+max-parallelism=3 fragment-costs=[6143098, 17264]
+17:AGGREGATE [STREAMING]
+|  output: sum(CAST(cs_quantity AS BIGINT)), sum(cs_wholesale_cost), sum(cs_sales_price)
+|  group by: d_year, cs_item_sk, cs_bill_customer_sk
+|  mem-estimate=10.00MB mem-reservation=9.00MB spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=13 row-size=56B cardinality=294.63K cost=1767774
+|  in pipelines: 12(GETNEXT)
+|
+16:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=06
+|  hash predicates: cs_sold_date_sk = d_date_sk
+|  fk/pk conjuncts: cs_sold_date_sk = d_date_sk
+|  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
+|  tuple-ids=10,11N,12 row-size=60B cardinality=294.63K cost=1441548
+|  in pipelines: 12(GETNEXT), 14(OPEN)
+|
+|--F19:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=8.77MB mem-reservation=8.75MB thread-reservation=1 runtime-filters-memory=1.00MB
+|  |  max-parallelism=3 fragment-costs=[388]
+|  JOIN BUILD
+|  |  join-table-id=06 plan-id=07 cohort-id=01
+|  |  build expressions: d_date_sk
+|  |  runtime filters: RF006[bloom] <- d_date_sk, RF007[min_max] <- d_date_sk
+|  |  mem-estimate=7.75MB mem-reservation=7.75MB spill-buffer=64.00KB thread-reservation=0 cost=373
+|  |
+|  22:EXCHANGE [BROADCAST]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=12 row-size=8B cardinality=373 cost=15
+|  |  in pipelines: 14(GETNEXT)
+|  |
+|  F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  Per-Host Shared Resources: mem-estimate=1.00MB mem-reservation=1.00MB thread-reservation=0 runtime-filters-memory=1.00MB
+|  Per-Instance Resources: mem-estimate=16.05MB mem-reservation=512.00KB thread-reservation=1
+|  max-parallelism=1 fragment-costs=[73625]
+|  14:SCAN HDFS [tpcds_parquet.date_dim, RANDOM]
+|     HDFS partitions=1/1 files=1 size=2.15MB
+|     predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|     runtime filters: RF000[bloom] -> tpcds_parquet.date_dim.d_year
+|     stored statistics:
+|       table: rows=73.05K size=2.15MB
+|       columns: all
+|     extrapolated-rows=disabled max-scan-range-rows=73.05K
+|     parquet statistics predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|     parquet dictionary predicates: tpcds_parquet.date_dim.d_year = CAST(2000 AS INT)
+|     mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=0
+|     tuple-ids=12 row-size=8B cardinality=373 cost=73620
+|     in pipelines: 14(GETNEXT)
+|
+15:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
+|  hash-table-id=07
+|  hash predicates: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
+|  fk/pk conjuncts: cs_item_sk = cr_item_sk, cs_order_number = cr_order_number
+|  other predicates: cr_order_number IS NULL
+|  mem-estimate=0B mem-reservation=0B spill-buffer=512.00KB thread-reservation=0
+|  tuple-ids=10,11N row-size=52B cardinality=1.44M cost=2883096
+|  in pipelines: 12(GETNEXT), 13(OPEN)
+|
+|--F20:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  |  Per-Instance Resources: mem-estimate=36.22MB mem-reservation=34.00MB thread-reservation=1
+|  |  max-parallelism=3 fragment-costs=[296576]
+|  JOIN BUILD
+|  |  join-table-id=07 plan-id=08 cohort-id=01
+|  |  build expressions: cr_item_sk, cr_order_number
+|  |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=512.00KB thread-reservation=0 cost=288134
+|  |
+|  21:EXCHANGE [BROADCAST]
+|  |  mem-estimate=2.22MB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=11 row-size=16B cardinality=144.07K cost=8442
+|  |  in pipelines: 13(GETNEXT)
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
... 984 lines suppressed ...


[impala] 05/07: IMPALA-11854: ImpalaStringWritable's underlying array can't be changed in UDFs

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit afe59f7f0d5658a6639e3e6f1f9ff9ed2f76d658
Author: Peter Rozsa <pr...@cloudera.com>
AuthorDate: Wed Feb 1 08:55:39 2023 +0100

    IMPALA-11854: ImpalaStringWritable's underlying array can't be changed in UDFs
    
    This change fixes the behavior of BytesWritable and TextWritable's
    getBytes() method. Now the returned byte array could be handled as
    the underlying buffer as it gets loaded before the UDF's evaluation,
    and tracks the changes as a regular Java byte array; the resizing
    operation still resets the reference. The operations that wrote back
    to the native heap were also removed as these operations are now
    handled in the byte array. ImpalaStringWritable class is also removed,
    writables that used it before now store the data directly.
    
    Tests:
     - Test UDFs added as BufferAlteringUdf and GenericBufferAlteringUdf
     - E2E test ran for UDFs
    
    Change-Id: Ifb28bd0dce7b0482c7abe1f61f245691fcbfe212
    Reviewed-on: http://gerrit.cloudera.org:8080/19507
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../impala/compat/HiveEsriGeospatialBuiltins.java  |  37 +--
 .../impala/hive/executor/HiveUdfExecutor.java      |   5 +-
 .../hive/executor/HiveUdfExecutorGeneric.java      |  64 ++---
 .../hive/executor/HiveUdfExecutorLegacy.java       |  11 +-
 .../impala/hive/executor/ImpalaBytesWritable.java  |  34 +--
 .../impala/hive/executor/ImpalaStringWritable.java | 117 --------
 .../impala/hive/executor/ImpalaTextWritable.java   |  22 +-
 .../impala/hive/executor/JavaUdfDataType.java      | 315 ++++++++++-----------
 .../{ImpalaTextWritable.java => Reloadable.java}   |  27 +-
 .../impala/hive/executor/UdfExecutorTest.java      |  45 +--
 .../java/org/apache/impala/BufferAlteringUdf.java  |  85 ++++++
 .../apache/impala/GenericBufferAlteringUdf.java    | 105 +++++++
 .../queries/QueryTest/generic-java-udf.test        |  28 ++
 .../queries/QueryTest/java-udf.test                |  35 +++
 .../queries/QueryTest/load-generic-java-udfs.test  |   8 +
 .../queries/QueryTest/load-java-udfs.test          |  12 +
 16 files changed, 513 insertions(+), 437 deletions(-)

diff --git a/fe/src/compat-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java b/fe/src/compat-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java
index 8ce149942..578dd6d0f 100644
--- a/fe/src/compat-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java
+++ b/fe/src/compat-hive-3/java/org/apache/impala/compat/HiveEsriGeospatialBuiltins.java
@@ -64,7 +64,6 @@ public class HiveEsriGeospatialBuiltins {
     addLegacyUDFs(db);
     addGenericUDFs(db);
     addVarargsUDFs(db);
-    addWorkaroundForStSetSrid(db);
   }
 
   private static void addLegacyUDFs(Db db) {
@@ -84,7 +83,7 @@ public class HiveEsriGeospatialBuiltins {
         new ST_NumInteriorRing(), new ST_NumPoints(), new ST_Point(),
         new ST_PointFromWKB(), new ST_PointN(), new ST_PointZ(), new ST_PolyFromWKB(),
         new ST_Relate(), new ST_SRID(), new ST_StartPoint(), new ST_SymmetricDiff(),
-        new ST_X(), new ST_Y(), new ST_Z());
+        new ST_X(), new ST_Y(), new ST_Z(), new ST_SetSRID());
 
     for (UDF udf : legacyUDFs) {
       for (Function fn : extractFromLegacyHiveBuiltin(udf, db.getName())) {
@@ -207,38 +206,4 @@ public class HiveEsriGeospatialBuiltins {
         })
         .collect(Collectors.toList());
   }
-
-  /*
-    TODO: IMPALA-11854: A workaround must be applied for ST_SetSRID UDF because the
-    GeometryUtils.setWKID method assumes that the incoming geomref's buffer can
-    be modified through the array returned by ImpalaBytesWritable.getBytes.
-   */
-  private static void addWorkaroundForStSetSrid(Db db) {
-    db.addBuiltin(
-        createScalarFunction(ST_SetSRID_Wrapper.class, ST_SetSRID.class.getSimpleName(),
-            Type.BINARY, new Type[] {Type.BINARY, Type.INT}));
-  }
-
-  public static class ST_SetSRID_Wrapper extends ST_SetSRID {
-    private static final Logger LOG = LoggerFactory.getLogger(ST_SetSRID_Wrapper.class);
-
-    @Override
-    public BytesWritable evaluate(BytesWritable geomref, IntWritable wkwrap) {
-      if (geomref != null && geomref.getLength() != 0) {
-        if (wkwrap != null) {
-          int wkid = wkwrap.get();
-          if (GeometryUtils.getWKID(geomref) != wkid) {
-            ByteBuffer bb = ByteBuffer.allocate(geomref.getLength());
-            bb.putInt(wkid);
-            bb.put(Arrays.copyOfRange(geomref.getBytes(), 4, geomref.getLength()));
-            return new BytesWritable(bb.array());
-          }
-        }
-        return geomref;
-      } else {
-        LogUtils.Log_ArgumentsNull(LOG);
-        return null;
-      }
-    }
-  }
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutor.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutor.java
index 31721f666..ee54066e9 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutor.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutor.java
@@ -115,7 +115,7 @@ public abstract class HiveUdfExecutor {
   }
 
   /**
-   * Evalutes the UDF with 'args' as the input to the UDF. This is exposed
+   * Evaluates the UDF with 'args' as the input to the UDF. This is exposed
    * for testing and not the version of evaluate() the backend uses.
    */
   public long evaluateForTesting(Object... args) throws ImpalaRuntimeException {
@@ -234,8 +234,7 @@ public abstract class HiveUdfExecutor {
     }
     UnsafeUtil.Copy(outBufferStringPtr_, bytes, 0, bytes.length);
     UnsafeUtil.UNSAFE.putInt(
-        outputBufferPtr_ + ImpalaStringWritable.STRING_VALUE_LEN_OFFSET,
-        bytes.length);
+        outputBufferPtr_ + JavaUdfDataType.STRING_VALUE_LEN_OFFSET, bytes.length);
   }
 
   // Preallocate the input objects that will be passed to the underlying UDF.
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutorGeneric.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutorGeneric.java
index 5d292e17e..9024ab536 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutorGeneric.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutorGeneric.java
@@ -17,46 +17,18 @@
 
 package org.apache.impala.hive.executor;
 
-import sun.misc.Unsafe;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.List;
 
-import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.impala.catalog.Type;
-import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.ImpalaRuntimeException;
-import org.apache.impala.common.JniUtil;
 import org.apache.impala.thrift.THiveUdfExecutorCtorParams;
-import org.apache.impala.thrift.TPrimitiveType;
 import org.apache.impala.util.UnsafeUtil;
 import org.apache.log4j.Logger;
-import org.apache.thrift.protocol.TBinaryProtocol;
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 
 /**
  * Wrapper object to run hive GenericUDFs. This class works with UdfCallExpr in the
@@ -98,17 +70,20 @@ public class HiveUdfExecutorGeneric extends HiveUdfExecutor {
   }
 
   /**
-   * Evalutes the UDF with 'args' as the input to the UDF.
+   * Evaluates the UDF with 'args' as the input to the UDF.
    */
   @Override
   protected Object evaluateDerived(JavaUdfDataType[] argTypes,
       long inputNullsPtr, Object[] inputObjectArgs) throws ImpalaRuntimeException {
     try {
       for (int i = 0; i < runtimeDeferredParameters_.length; ++i) {
-        runtimeDeferredParameters_[i] =
-            (UnsafeUtil.UNSAFE.getByte(inputNullsPtr + i) == 0)
-                ? deferredParameters_[i]
-                : deferredNullParameter_;
+        if (UnsafeUtil.UNSAFE.getByte(inputNullsPtr + i) == 0) {
+          runtimeDeferredParameters_[i] = deferredParameters_[i];
+          // argument 'i' is unused in DeferredJavaObject and in DeferredWritable as well
+          runtimeDeferredParameters_[i].prepare(0);
+        } else {
+          runtimeDeferredParameters_[i] = deferredNullParameter_;
+        }
       }
       return genericUDF_.evaluate(runtimeDeferredParameters_);
     } catch (HiveException e) {
@@ -140,8 +115,29 @@ public class HiveUdfExecutorGeneric extends HiveUdfExecutor {
   private DeferredObject[] createDeferredObjects() {
     DeferredObject[] deferredObjects = new DeferredObject[getNumParams()];
     for (int i = 0; i < deferredObjects.length; ++i) {
-      deferredObjects[i] = new DeferredJavaObject(getInputObject(i));
+      Object inputObject = getInputObject(i);
+      if (inputObject instanceof Reloadable) {
+        deferredObjects[i] = new DeferredWritable<>((Reloadable) inputObject);
+      } else {
+        deferredObjects[i] = new DeferredJavaObject(inputObject);
+      }
     }
     return deferredObjects;
   }
+
+  private static class DeferredWritable<T extends Reloadable> implements DeferredObject {
+    private final T writable;
+
+    public DeferredWritable(T writable) { this.writable = writable; }
+
+    @Override
+    public void prepare(int ignored) throws HiveException {
+      writable.reload();
+    }
+
+    @Override
+    public Object get() throws HiveException {
+      return writable;
+    }
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutorLegacy.java b/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutorLegacy.java
index 0f9c59328..c665997d9 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutorLegacy.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/HiveUdfExecutorLegacy.java
@@ -82,10 +82,14 @@ public class HiveUdfExecutorLegacy extends HiveUdfExecutor {
             case INT_WRITABLE:
             case LONG_WRITABLE:
             case FLOAT_WRITABLE:
-            case DOUBLE_WRITABLE:
+            case DOUBLE_WRITABLE: inputArgs_[i] = inputObjects[i]; break;
             case BYTE_ARRAY:
             case BYTES_WRITABLE:
+              ((ImpalaBytesWritable) inputObjects[i]).reload();
+              inputArgs_[i] = inputObjects[i];
+              break;
             case TEXT:
+              ((ImpalaTextWritable) inputObjects[i]).reload();
               inputArgs_[i] = inputObjects[i];
               break;
             case BOOLEAN:
@@ -111,8 +115,9 @@ public class HiveUdfExecutorLegacy extends HiveUdfExecutor {
               break;
             case STRING:
               Preconditions.checkState(inputObjects[i] instanceof ImpalaBytesWritable);
-              inputArgs_[i] =
-                  new String(((ImpalaBytesWritable)inputObjects[i]).getBytes());
+              ImpalaBytesWritable inputObject = (ImpalaBytesWritable) inputObjects[i];
+              inputObject.reload();
+              inputArgs_[i] = new String(inputObject.getBytes());
               break;
           }
         } else {
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaBytesWritable.java b/fe/src/main/java/org/apache/impala/hive/executor/ImpalaBytesWritable.java
index 94824e02c..f223f1194 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaBytesWritable.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/ImpalaBytesWritable.java
@@ -21,36 +21,18 @@ import org.apache.hadoop.io.BytesWritable;
 
 /**
  * Impala writable type that implements the BytesWritable interface. The data
- * marshalling is handled by the underlying {@link ImpalaStringWritable} object.
+ * marshalling is handled by {@link JavaUdfDataType#loadStringValueFromNativeHeap(long)}.
  */
-public class ImpalaBytesWritable extends BytesWritable {
-  private final ImpalaStringWritable string_;
+public class ImpalaBytesWritable extends BytesWritable implements Reloadable {
+  private final long ptr_;
 
-  public ImpalaBytesWritable(long ptr) {
-    string_ = new ImpalaStringWritable(ptr);
-  }
+  public ImpalaBytesWritable(long ptr) { this.ptr_ = ptr; }
 
   @Override
-  public byte[] copyBytes() {
-    byte[] src = getBytes();
-    return src.clone();
+  public void reload() {
+    byte[] bytes = JavaUdfDataType.loadStringValueFromNativeHeap(ptr_);
+    super.setCapacity(bytes.length);
+    super.set(bytes, 0, bytes.length);
   }
 
-  @Override
-  public byte[] get() { return getBytes(); }
-  @Override
-  public byte[] getBytes() { return string_.getBytes(); }
-  @Override
-  public int getCapacity() { return string_.getCapacity(); }
-  @Override
-  public int getLength() { return string_.getLength(); }
-
-  public ImpalaStringWritable getStringWritable() { return string_; }
-
-  @Override
-  public void set(byte[] v, int offset, int len) { string_.set(v, offset, len); }
-  @Override
-  public void setCapacity(int newCap) { string_.setCapacity(newCap); }
-  @Override
-  public void setSize(int size) { string_.setSize(size); }
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaStringWritable.java b/fe/src/main/java/org/apache/impala/hive/executor/ImpalaStringWritable.java
deleted file mode 100644
index 2ef6adf47..000000000
--- a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaStringWritable.java
+++ /dev/null
@@ -1,117 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-package org.apache.impala.hive.executor;
-
-import java.nio.ByteBuffer;
-
-import org.apache.impala.util.UnsafeUtil;
-
-@SuppressWarnings("restriction")
-/**
- * Underlying class for Text and Bytes writable. This class understands marshalling
- * values that map to StringValue in the BE.
- * StringValue is replicated here:
- * struct StringValue {
- *   char* ptr;
- *   int len;
- * };
- */
-public class ImpalaStringWritable {
-  // The length is 8 bytes into the struct.
-  static public final int STRING_VALUE_LEN_OFFSET = 8;
-
-  // Ptr (to native heap) where the value should be read from and written to.
-  // This needs to be ABI compatible with the BE StringValue class
-  private final long stringValPtr_;
-
-  // Array object to convert between native and java heap (i.e. byte[]).
-  private ByteBuffer array_;
-
-  // Set if this object had to allocate from the native heap on the java side. If this
-  // is set, it will always be stringValPtr_->ptr
-  // We only need to allocate from the java side if we are trying to set the
-  // StringValue to a bigger size than what the native side allocated.
-  // If this object is used as a read-only input argument, this value will stay
-  // 0.
-  private long bufferPtr_;
-
-  // Allocation size of stringValPtr_'s ptr.
-  private int bufferCapacity_;
-
-  // Creates a string writable backed by a StringValue object. Ptr must be a valid
-  // StringValue (in the native heap).
-  public ImpalaStringWritable(long ptr) {
-    stringValPtr_ = ptr;
-    bufferPtr_= 0;
-    bufferCapacity_ = getLength();
-    array_ = ByteBuffer.allocate(0);
-  }
-
-  /*
-   * Implement finalize() to clean up any allocations from the native heap.
-   */
-  @Override
-  protected void finalize() throws Throwable {
-    UnsafeUtil.UNSAFE.freeMemory(bufferPtr_);
-    super.finalize();
-  }
-
-  // Returns the underlying bytes as a byte[]
-  public byte[] getBytes() {
-    int len = getLength();
-    // TODO: reuse this array.
-    array_ = ByteBuffer.allocate(len);
-    byte[] buffer = array_.array();
-
-    long srcPtr = UnsafeUtil.UNSAFE.getLong(stringValPtr_);
-    UnsafeUtil.Copy(buffer, 0, srcPtr, len);
-    return buffer;
-  }
-
-  // Returns the capacity of the underlying array
-  public int getCapacity() {
-    return bufferCapacity_;
-  }
-
-  // Updates the new capacity. No-op if the new capacity is smaller.
-  public void setCapacity(int newCap) {
-    if (newCap <= bufferCapacity_) return;
-    bufferPtr_ = UnsafeUtil.UNSAFE.reallocateMemory(bufferPtr_, newCap);
-    UnsafeUtil.UNSAFE.putLong(stringValPtr_, bufferPtr_);
-    bufferCapacity_ = newCap;
-  }
-
-  // Returns the length of the string
-  public int getLength() {
-    return UnsafeUtil.UNSAFE.getInt(stringValPtr_ + STRING_VALUE_LEN_OFFSET);
-  }
-
-  // Updates the length of the string. If the new length is bigger,
-  // the additional bytes are undefined.
-  public void setSize(int s) {
-    setCapacity(s);
-    UnsafeUtil.UNSAFE.putInt(stringValPtr_ + 8, s);
-  }
-
-  // Sets (v[offset], len) to the underlying buffer, growing it as necessary.
-  public void set(byte[] v, int offset, int len) {
-    setSize(len);
-    long strPtr = UnsafeUtil.UNSAFE.getLong(stringValPtr_);
-    UnsafeUtil.Copy(strPtr, v, offset, len);
-  }
-}
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaTextWritable.java b/fe/src/main/java/org/apache/impala/hive/executor/ImpalaTextWritable.java
index 2a8745418..860014b8d 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaTextWritable.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/ImpalaTextWritable.java
@@ -21,25 +21,17 @@ import org.apache.hadoop.io.Text;
 
 /**
  * Impala writable type that implements the Text interface. The data marshalling is
- * handled by the underlying {@link ImpalaStringWritable} object.
+ * handled by {@link JavaUdfDataType#loadStringValueFromNativeHeap(long)}.
  */
-public class ImpalaTextWritable extends Text {
-  private final ImpalaStringWritable string_;
+public class ImpalaTextWritable extends Text implements Reloadable {
+  private final long ptr_;
 
-  public ImpalaTextWritable(long ptr) {
-    string_ = new ImpalaStringWritable(ptr);
-  }
-
-  @Override
-  public String toString() { return new String(getBytes()); }
-  @Override
-  public byte[] getBytes() { return string_.getBytes(); }
-  @Override
-  public int getLength() { return string_.getLength(); }
+  public ImpalaTextWritable(long ptr) { this.ptr_ = ptr; }
 
   @Override
-  public void set(byte[] v, int offset, int len) {
-    string_.set(v, offset, len);
+  public void reload() {
+    byte[] bytes = JavaUdfDataType.loadStringValueFromNativeHeap(ptr_);
+    super.set(bytes);
   }
 
 }
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/JavaUdfDataType.java b/fe/src/main/java/org/apache/impala/hive/executor/JavaUdfDataType.java
index d0824cfbe..981c1d37c 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/JavaUdfDataType.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/JavaUdfDataType.java
@@ -23,193 +23,192 @@ import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveWritableObjectInspector;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.thrift.TPrimitiveType;
 
 import com.google.common.base.Preconditions;
+import org.apache.impala.util.UnsafeUtil;
+
+// Data types that are supported as return or argument types in Java UDFs.
+public enum JavaUdfDataType {
+  INVALID_TYPE("INVALID_TYPE", TPrimitiveType.INVALID_TYPE),
+  BOOLEAN("BOOLEAN", TPrimitiveType.BOOLEAN),
+  BOOLEAN_WRITABLE("BOOLEAN_WRITABLE", TPrimitiveType.BOOLEAN),
+  TINYINT("TINYINT", TPrimitiveType.TINYINT),
+  BYTE_WRITABLE("BYTE_WRITABLE", TPrimitiveType.TINYINT),
+  SMALLINT("SMALLINT", TPrimitiveType.SMALLINT),
+  SHORT_WRITABLE("SHORT_WRITABLE", TPrimitiveType.SMALLINT),
+  INT("INT", TPrimitiveType.INT),
+  INT_WRITABLE("INT_WRITABLE", TPrimitiveType.INT),
+  BIGINT("BIGINT", TPrimitiveType.BIGINT),
+  LONG_WRITABLE("LONG_WRITABLE", TPrimitiveType.BIGINT),
+  FLOAT("FLOAT", TPrimitiveType.FLOAT),
+  FLOAT_WRITABLE("FLOAT_WRITABLE", TPrimitiveType.FLOAT),
+  DOUBLE("DOUBLE", TPrimitiveType.DOUBLE),
+  DOUBLE_WRITABLE("DOUBLE", TPrimitiveType.DOUBLE),
+  STRING("STRING", TPrimitiveType.STRING),
+  TEXT("TEXT", TPrimitiveType.STRING),
+  BYTES_WRITABLE("BYTES_WRITABLE", TPrimitiveType.STRING),
+  BYTE_ARRAY("BYTE_ARRAY", TPrimitiveType.STRING);
+
+  public static final int STRING_VALUE_LEN_OFFSET = 8;
+
+  private final String description_;
+  private final TPrimitiveType thriftType_;
+
+  private JavaUdfDataType(String description, TPrimitiveType thriftType) {
+    description_ = description;
+    thriftType_ = thriftType;
+  }
 
-  // Data types that are supported as return or argument types in Java UDFs.
-  public enum JavaUdfDataType {
-    INVALID_TYPE("INVALID_TYPE", TPrimitiveType.INVALID_TYPE),
-    BOOLEAN("BOOLEAN", TPrimitiveType.BOOLEAN),
-    BOOLEAN_WRITABLE("BOOLEAN_WRITABLE", TPrimitiveType.BOOLEAN),
-    TINYINT("TINYINT", TPrimitiveType.TINYINT),
-    BYTE_WRITABLE("BYTE_WRITABLE", TPrimitiveType.TINYINT),
-    SMALLINT("SMALLINT", TPrimitiveType.SMALLINT),
-    SHORT_WRITABLE("SHORT_WRITABLE", TPrimitiveType.SMALLINT),
-    INT("INT", TPrimitiveType.INT),
-    INT_WRITABLE("INT_WRITABLE", TPrimitiveType.INT),
-    BIGINT("BIGINT", TPrimitiveType.BIGINT),
-    LONG_WRITABLE("LONG_WRITABLE", TPrimitiveType.BIGINT),
-    FLOAT("FLOAT", TPrimitiveType.FLOAT),
-    FLOAT_WRITABLE("FLOAT_WRITABLE", TPrimitiveType.FLOAT),
-    DOUBLE("DOUBLE", TPrimitiveType.DOUBLE),
-    DOUBLE_WRITABLE("DOUBLE", TPrimitiveType.DOUBLE),
-    STRING("STRING", TPrimitiveType.STRING),
-    TEXT("TEXT", TPrimitiveType.STRING),
-    BYTES_WRITABLE("BYTES_WRITABLE", TPrimitiveType.STRING),
-    BYTE_ARRAY("BYTE_ARRAY", TPrimitiveType.STRING);
-
-    private final String description_;
-    private final TPrimitiveType thriftType_;
-
-    private JavaUdfDataType(String description, TPrimitiveType thriftType) {
-      description_ = description;
-      thriftType_ = thriftType;
-    }
-
-    @Override
-    public String toString() { return description_; }
+  @Override
+  public String toString() { return description_; }
 
-    public String getDescription() { return description_; }
+  public String getDescription() { return description_; }
 
-    public TPrimitiveType getPrimitiveType() { return thriftType_; }
+  public TPrimitiveType getPrimitiveType() { return thriftType_; }
 
-    public static JavaUdfDataType[] getTypes(Type typeArray[]) {
-      JavaUdfDataType[] types = new JavaUdfDataType[typeArray.length];
-      for (int i = 0; i < typeArray.length; ++i) {
-        types[i] = getType(typeArray[i]);
-      }
-      return types;
+  public static JavaUdfDataType[] getTypes(Type[] typeArray) {
+    JavaUdfDataType[] types = new JavaUdfDataType[typeArray.length];
+    for (int i = 0; i < typeArray.length; ++i) {
+      types[i] = getType(typeArray[i]);
     }
+    return types;
+  }
 
-    public static JavaUdfDataType[] getTypes(Class<?>[] typeArray) {
-      JavaUdfDataType[] types = new JavaUdfDataType[typeArray.length];
-      for (int i = 0; i < typeArray.length; ++i) {
-        types[i] = getType(typeArray[i]);
-      }
-      return types;
+  public static JavaUdfDataType[] getTypes(Class<?>[] typeArray) {
+    JavaUdfDataType[] types = new JavaUdfDataType[typeArray.length];
+    for (int i = 0; i < typeArray.length; ++i) {
+      types[i] = getType(typeArray[i]);
     }
+    return types;
+  }
 
-    public static JavaUdfDataType getType(Type t) {
-      switch (t.getPrimitiveType().toThrift()) {
-        case BOOLEAN:
-          return JavaUdfDataType.BOOLEAN_WRITABLE;
-        case TINYINT:
-          return JavaUdfDataType.BYTE_WRITABLE;
-        case SMALLINT:
-          return JavaUdfDataType.SHORT_WRITABLE;
-        case INT:
-          return JavaUdfDataType.INT_WRITABLE;
-        case BIGINT:
-          return JavaUdfDataType.LONG_WRITABLE;
-        case FLOAT:
-          return JavaUdfDataType.FLOAT_WRITABLE;
-        case DOUBLE:
-          return JavaUdfDataType.DOUBLE_WRITABLE;
-        case STRING:
-          return JavaUdfDataType.TEXT;
-        case BINARY:
-          return JavaUdfDataType.BYTES_WRITABLE;
-        default:
-          return null;
-      }
+  public static JavaUdfDataType getType(Type t) {
+    switch (t.getPrimitiveType().toThrift()) {
+      case BOOLEAN: return JavaUdfDataType.BOOLEAN_WRITABLE;
+      case TINYINT: return JavaUdfDataType.BYTE_WRITABLE;
+      case SMALLINT: return JavaUdfDataType.SHORT_WRITABLE;
+      case INT: return JavaUdfDataType.INT_WRITABLE;
+      case BIGINT: return JavaUdfDataType.LONG_WRITABLE;
+      case FLOAT: return JavaUdfDataType.FLOAT_WRITABLE;
+      case DOUBLE: return JavaUdfDataType.DOUBLE_WRITABLE;
+      case STRING: return JavaUdfDataType.TEXT;
+      case BINARY: return JavaUdfDataType.BYTES_WRITABLE;
+      default: return null;
     }
+  }
 
-    public static JavaUdfDataType getType(ObjectInspector oi) {
-      // Only primitive objects are supported currently.
-      Preconditions.checkState(oi instanceof PrimitiveObjectInspector);
-      PrimitiveObjectInspector primOi = (PrimitiveObjectInspector) oi;
-      PrimitiveCategory cat = primOi.getPrimitiveCategory();
-      boolean writable = primOi.preferWritable();
-      switch (cat) {
-        case BOOLEAN:
-          return writable ? JavaUdfDataType.BOOLEAN_WRITABLE : JavaUdfDataType.BOOLEAN;
-        case BYTE:
-          return writable ? JavaUdfDataType.BYTE_WRITABLE : JavaUdfDataType.TINYINT;
-        case SHORT:
-          return writable ? JavaUdfDataType.SHORT_WRITABLE : JavaUdfDataType.SMALLINT;
-        case INT:
-          return writable ? JavaUdfDataType.INT_WRITABLE : JavaUdfDataType.INT;
-        case LONG:
-          return writable ? JavaUdfDataType.LONG_WRITABLE : JavaUdfDataType.BIGINT;
-        case FLOAT:
-          return writable ? JavaUdfDataType.FLOAT_WRITABLE : JavaUdfDataType.FLOAT;
-        case DOUBLE:
-          return writable ? JavaUdfDataType.DOUBLE_WRITABLE : JavaUdfDataType.DOUBLE;
-        case STRING:
-          return writable ? JavaUdfDataType.TEXT : JavaUdfDataType.STRING;
-        case BINARY:
-          return writable ? JavaUdfDataType.BYTES_WRITABLE : JavaUdfDataType.BYTE_ARRAY;
-        default:
-          return null;
-      }
+  public static JavaUdfDataType getType(ObjectInspector oi) {
+    // Only primitive objects are supported currently.
+    Preconditions.checkState(oi instanceof PrimitiveObjectInspector);
+    PrimitiveObjectInspector primOi = (PrimitiveObjectInspector) oi;
+    PrimitiveCategory cat = primOi.getPrimitiveCategory();
+    boolean writable = primOi.preferWritable();
+    switch (cat) {
+      case BOOLEAN:
+        return writable ? JavaUdfDataType.BOOLEAN_WRITABLE : JavaUdfDataType.BOOLEAN;
+      case BYTE:
+        return writable ? JavaUdfDataType.BYTE_WRITABLE : JavaUdfDataType.TINYINT;
+      case SHORT:
+        return writable ? JavaUdfDataType.SHORT_WRITABLE : JavaUdfDataType.SMALLINT;
+      case INT:
+        return writable ? JavaUdfDataType.INT_WRITABLE : JavaUdfDataType.INT;
+      case LONG:
+        return writable ? JavaUdfDataType.LONG_WRITABLE : JavaUdfDataType.BIGINT;
+      case FLOAT:
+        return writable ? JavaUdfDataType.FLOAT_WRITABLE : JavaUdfDataType.FLOAT;
+      case DOUBLE:
+        return writable ? JavaUdfDataType.DOUBLE_WRITABLE : JavaUdfDataType.DOUBLE;
+      case STRING:
+        return writable ? JavaUdfDataType.TEXT : JavaUdfDataType.STRING;
+      case BINARY:
+        return writable ? JavaUdfDataType.BYTES_WRITABLE : JavaUdfDataType.BYTE_ARRAY;
+      default:
+        return null;
     }
+  }
 
-    public static JavaUdfDataType getType(Class<?> c) {
-      if (c == BooleanWritable.class) {
-        return JavaUdfDataType.BOOLEAN_WRITABLE;
-      } else if (c == boolean.class || c == Boolean.class) {
-        return JavaUdfDataType.BOOLEAN;
-      } else if (c == ByteWritable.class) {
-        return JavaUdfDataType.BYTE_WRITABLE;
-      } else if (c == byte.class || c == Byte.class) {
-        return JavaUdfDataType.TINYINT;
-      } else if (c == ShortWritable.class) {
-        return JavaUdfDataType.SHORT_WRITABLE;
-      } else if (c == short.class || c == Short.class) {
-        return JavaUdfDataType.SMALLINT;
-      } else if (c == IntWritable.class) {
-        return JavaUdfDataType.INT_WRITABLE;
-      } else if (c == int.class || c == Integer.class) {
-        return JavaUdfDataType.INT;
-      } else if (c == LongWritable.class) {
-        return JavaUdfDataType.LONG_WRITABLE;
-      } else if (c == long.class || c == Long.class) {
-        return JavaUdfDataType.BIGINT;
-      } else if (c == FloatWritable.class) {
-        return JavaUdfDataType.FLOAT_WRITABLE;
-      } else if (c == float.class || c == Float.class) {
-        return JavaUdfDataType.FLOAT;
-      } else if (c == DoubleWritable.class) {
-        return JavaUdfDataType.DOUBLE_WRITABLE;
-      } else if (c == double.class || c == Double.class) {
-        return JavaUdfDataType.DOUBLE;
-      } else if (c == byte[].class) {
-        return JavaUdfDataType.BYTE_ARRAY;
-      } else if (c == BytesWritable.class) {
-        return JavaUdfDataType.BYTES_WRITABLE;
-      } else if (c == Text.class) {
-        return JavaUdfDataType.TEXT;
-      } else if (c == String.class) {
-        return JavaUdfDataType.STRING;
-      }
-      return JavaUdfDataType.INVALID_TYPE;
+  public static JavaUdfDataType getType(Class<?> c) {
+    if (c == BooleanWritable.class) {
+      return JavaUdfDataType.BOOLEAN_WRITABLE;
+    } else if (c == boolean.class || c == Boolean.class) {
+      return JavaUdfDataType.BOOLEAN;
+    } else if (c == ByteWritable.class) {
+      return JavaUdfDataType.BYTE_WRITABLE;
+    } else if (c == byte.class || c == Byte.class) {
+      return JavaUdfDataType.TINYINT;
+    } else if (c == ShortWritable.class) {
+      return JavaUdfDataType.SHORT_WRITABLE;
+    } else if (c == short.class || c == Short.class) {
+      return JavaUdfDataType.SMALLINT;
+    } else if (c == IntWritable.class) {
+      return JavaUdfDataType.INT_WRITABLE;
+    } else if (c == int.class || c == Integer.class) {
+      return JavaUdfDataType.INT;
+    } else if (c == LongWritable.class) {
+      return JavaUdfDataType.LONG_WRITABLE;
+    } else if (c == long.class || c == Long.class) {
+      return JavaUdfDataType.BIGINT;
+    } else if (c == FloatWritable.class) {
+      return JavaUdfDataType.FLOAT_WRITABLE;
+    } else if (c == float.class || c == Float.class) {
+      return JavaUdfDataType.FLOAT;
+    } else if (c == DoubleWritable.class) {
+      return JavaUdfDataType.DOUBLE_WRITABLE;
+    } else if (c == double.class || c == Double.class) {
+      return JavaUdfDataType.DOUBLE;
+    } else if (c == byte[].class) {
+      return JavaUdfDataType.BYTE_ARRAY;
+    } else if (c == BytesWritable.class) {
+      return JavaUdfDataType.BYTES_WRITABLE;
+    } else if (c == Text.class) {
+      return JavaUdfDataType.TEXT;
+    } else if (c == String.class) {
+      return JavaUdfDataType.STRING;
     }
+    return JavaUdfDataType.INVALID_TYPE;
+  }
 
-    public static boolean isSupported(Type t) {
-      if (TPrimitiveType.INVALID_TYPE == t.getPrimitiveType().toThrift()) {
-        return false;
-      }
+  public static boolean isSupported(Type t) {
+    if (TPrimitiveType.INVALID_TYPE == t.getPrimitiveType().toThrift()) {
+      return false;
+    }
 
-      // While BYTES_WRITABLE and BYTE_ARRAY maps to STRING to keep compatibility,
-      // BINARY is also accepted (IMPALA-11340).
-      if (t.isBinary()) return true;
+    // While BYTES_WRITABLE and BYTE_ARRAY maps to STRING to keep compatibility,
+    // BINARY is also accepted (IMPALA-11340).
+    if (t.isBinary()) return true;
 
-      for(JavaUdfDataType javaType: JavaUdfDataType.values()) {
-        if (javaType.getPrimitiveType() == t.getPrimitiveType().toThrift()) {
-          return true;
-        }
+    for (JavaUdfDataType javaType : JavaUdfDataType.values()) {
+      if (javaType.getPrimitiveType() == t.getPrimitiveType().toThrift()) {
+        return true;
       }
-      return false;
     }
+    return false;
+  }
 
-    public boolean isCompatibleWith(TPrimitiveType t) {
-      if (t == getPrimitiveType()) return true;
-      if (t == TPrimitiveType.BINARY) {
-        // While BYTES_WRITABLE and BYTE_ARRAY maps to STRING to keep compatibility,
-        // BINARY is also accepted (IMPALA-11340).
-        if (this == BYTE_ARRAY || this == BYTES_WRITABLE) return true;
-      }
-      return false;
+  public boolean isCompatibleWith(TPrimitiveType t) {
+    if (t == getPrimitiveType()) return true;
+    if (t == TPrimitiveType.BINARY) {
+      // While BYTES_WRITABLE and BYTE_ARRAY maps to STRING to keep compatibility,
+      // BINARY is also accepted (IMPALA-11340).
+      if (this == BYTE_ARRAY || this == BYTES_WRITABLE) return true;
     }
+    return false;
   }
 
+  // Returns a backend-allocated string as a Java byte array
+  public static byte[] loadStringValueFromNativeHeap(long ptr) {
+    int length = UnsafeUtil.UNSAFE.getInt(ptr + STRING_VALUE_LEN_OFFSET);
+    byte[] buffer = new byte[length];
+    long srcPtr = UnsafeUtil.UNSAFE.getLong(ptr);
+    UnsafeUtil.Copy(buffer, 0, srcPtr, length);
+    return buffer;
+  }
+}
diff --git a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaTextWritable.java b/fe/src/main/java/org/apache/impala/hive/executor/Reloadable.java
similarity index 54%
copy from fe/src/main/java/org/apache/impala/hive/executor/ImpalaTextWritable.java
copy to fe/src/main/java/org/apache/impala/hive/executor/Reloadable.java
index 2a8745418..cb0b9c430 100644
--- a/fe/src/main/java/org/apache/impala/hive/executor/ImpalaTextWritable.java
+++ b/fe/src/main/java/org/apache/impala/hive/executor/Reloadable.java
@@ -17,29 +17,4 @@
 
 package org.apache.impala.hive.executor;
 
-import org.apache.hadoop.io.Text;
-
-/**
- * Impala writable type that implements the Text interface. The data marshalling is
- * handled by the underlying {@link ImpalaStringWritable} object.
- */
-public class ImpalaTextWritable extends Text {
-  private final ImpalaStringWritable string_;
-
-  public ImpalaTextWritable(long ptr) {
-    string_ = new ImpalaStringWritable(ptr);
-  }
-
-  @Override
-  public String toString() { return new String(getBytes()); }
-  @Override
-  public byte[] getBytes() { return string_.getBytes(); }
-  @Override
-  public int getLength() { return string_.getLength(); }
-
-  @Override
-  public void set(byte[] v, int offset, int len) {
-    string_.set(v, offset, len);
-  }
-
-}
+public interface Reloadable { void reload(); }
diff --git a/fe/src/test/java/org/apache/impala/hive/executor/UdfExecutorTest.java b/fe/src/test/java/org/apache/impala/hive/executor/UdfExecutorTest.java
index 0ea62bbe2..16f4debda 100644
--- a/fe/src/test/java/org/apache/impala/hive/executor/UdfExecutorTest.java
+++ b/fe/src/test/java/org/apache/impala/hive/executor/UdfExecutorTest.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.hive.ql.udf.UDFUnbase64;
 import org.apache.hadoop.hive.ql.udf.UDFUnhex;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBRound;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper;
-import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.impala.catalog.PrimitiveType;
@@ -158,21 +157,30 @@ public class UdfExecutorTest {
   Writable createDouble(double v) { return createObject(PrimitiveType.DOUBLE, v); }
 
   Writable createBytes(String v) {
-    long ptr = allocate(16);
-    UnsafeUtil.UNSAFE.putInt(ptr + 8, 0);
-    ImpalaBytesWritable tw = new ImpalaBytesWritable(ptr);
-    byte[] array = v.getBytes();
-    tw.set(array, 0, array.length);
-    return tw;
+    long ptr = allocateStringValue(v);
+    ImpalaBytesWritable bytesWritable = new ImpalaBytesWritable(ptr);
+    bytesWritable.reload();
+    return bytesWritable;
   }
 
   Writable createText(String v) {
-    long ptr = allocate(16);
-    UnsafeUtil.UNSAFE.putInt(ptr + 8, 0);
-    ImpalaTextWritable tw = new ImpalaTextWritable(ptr);
-    byte[] array = v.getBytes();
-    tw.set(array, 0, array.length);
-    return tw;
+    long ptr = allocateStringValue(v);
+    ImpalaTextWritable textWritable = new ImpalaTextWritable(ptr);
+    textWritable.reload();
+    return textWritable;
+  }
+
+  private long allocateStringValue(String v) {
+    // Allocate StringValue: sizeof(StringValue) = 8 (pointer) + 4 (length)
+    long ptr = allocate(12);
+    // Setting length
+    UnsafeUtil.UNSAFE.putInt(ptr + 8, v.length());
+    // Allocate buffer for v
+    long stringPtr = allocate(v.length());
+    // Setting string pointer
+    UnsafeUtil.UNSAFE.putLong(ptr, stringPtr);
+    UnsafeUtil.Copy(stringPtr, v.getBytes(), 0, v.length());
+    return ptr;
   }
 
   // Returns the primitive type for w
@@ -402,17 +410,16 @@ public class UdfExecutorTest {
         } else {
           Preconditions.checkState(false);
         }
-        ImpalaStringWritable sw = new ImpalaStringWritable(r);
-        if (Arrays.equals(expectedBytes, sw.getBytes())) break;
+        byte[] bytes = JavaUdfDataType.loadStringValueFromNativeHeap(r);
+        if (Arrays.equals(expectedBytes, bytes)) break;
 
         errMsgs.add("Expected string: " + Bytes.toString(expectedBytes));
-        errMsgs.add("Actual string:   " + Bytes.toString(sw.getBytes()));
+        errMsgs.add("Actual string:   " + Bytes.toString(bytes));
         errMsgs.add("Expected bytes:  " + Arrays.toString(expectedBytes));
-        errMsgs.add("Actual bytes:    " + Arrays.toString(sw.getBytes()));
+        errMsgs.add("Actual bytes:    " + Arrays.toString(bytes));
         break;
       }
-      default:
-        Preconditions.checkArgument(false);
+      default: Preconditions.checkArgument(false);
     }
   }
 
diff --git a/java/test-hive-udfs/src/main/java/org/apache/impala/BufferAlteringUdf.java b/java/test-hive-udfs/src/main/java/org/apache/impala/BufferAlteringUdf.java
new file mode 100644
index 000000000..9fbe07a04
--- /dev/null
+++ b/java/test-hive-udfs/src/main/java/org/apache/impala/BufferAlteringUdf.java
@@ -0,0 +1,85 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala;
+
+import java.text.ParseException;
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class intends to test a UDF that manipulates array-backed writables and checks
+ * their data retention behavior. Before IMPALA-11854, every getBytes() method call
+ * directly read the native heap and there was no option to store intermediate results in
+ * the writable buffers. IMPALA-11854 changed this behavior, now every data manipulation
+ * operation on array-backed writables loads the native heap data before the 'evaluate'
+ * phase and stores it in the writable, any subsequent manipulations use the writable's
+ * interface.
+ */
+public class BufferAlteringUdf extends UDF {
+  /**
+   * Increments the first byte by one in a Text and returns the result as a Text
+   */
+  public Text evaluate(Text text) throws ParseException {
+    if ((null == text) || ("".equals(text.toString()))) {
+      return null;
+    }
+    byte[] bytes = text.getBytes();
+
+    incrementByteArray(bytes);
+    return text;
+  }
+
+  /**
+   * Increments the first byte by one in a BytesWritable and returns the result as a
+   * BytesWritable
+   */
+  public BytesWritable evaluate(BytesWritable bytesWritable) throws ParseException {
+    if (null == bytesWritable) {
+      return null;
+    }
+    byte[] bytes = bytesWritable.getBytes();
+
+    incrementByteArray(bytes);
+    return bytesWritable;
+  }
+
+  /**
+   * Copies the source BytesWritable to the target BytesWritable, implicitly resizing it.
+   * After the copy, the first byte of the target BytesWritable is incremented by one.
+   */
+  public BytesWritable evaluate(BytesWritable target, BytesWritable source)
+      throws ParseException {
+    if (null == source || null == target) {
+      return null;
+    }
+    byte[] sourceArray = source.getBytes();
+    target.set(sourceArray, 0, source.getLength());
+
+    byte[] targetArray = target.getBytes();
+    incrementByteArray(targetArray);
+
+    return target;
+  }
+
+  private void incrementByteArray(byte[] array) {
+    if (array.length > 0) {
+      array[0] += 1;
+    }
+  }
+}
diff --git a/java/test-hive-udfs/src/main/java/org/apache/impala/GenericBufferAlteringUdf.java b/java/test-hive-udfs/src/main/java/org/apache/impala/GenericBufferAlteringUdf.java
new file mode 100644
index 000000000..3a6844324
--- /dev/null
+++ b/java/test-hive-udfs/src/main/java/org/apache/impala/GenericBufferAlteringUdf.java
@@ -0,0 +1,105 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala;
+
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is the generic version of BufferAlteringUdf. The main purpose is to check
+ * data retention capability after IMPALA-11854.
+ */
+public class GenericBufferAlteringUdf extends GenericUDF {
+  public static final String ARGUMENT_LIST_LENGTH_FORMAT =
+      "This function takes 1 argument, %d argument(s) provided";
+  private PrimitiveCategory argAndRetType_;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] objectInspectors)
+      throws UDFArgumentException {
+    if (objectInspectors.length != 1) {
+      throw new UDFArgumentException(
+          String.format(ARGUMENT_LIST_LENGTH_FORMAT, objectInspectors.length));
+    }
+    if (!(objectInspectors[0] instanceof PrimitiveObjectInspector)) {
+      throw new UDFArgumentException("Found an input that is not a primitive.");
+    }
+    PrimitiveObjectInspector objectInspector =
+        (PrimitiveObjectInspector) objectInspectors[0];
+    argAndRetType_ = objectInspector.getPrimitiveCategory();
+
+    // Return type is same as the input parameter
+    return objectInspector;
+  }
+
+  /**
+   * This function expects a Text or a BytesWritable and increments their underlying byte
+   * array's first element by one.
+   */
+  @Override
+  public Object evaluate(DeferredObject[] deferredObjects) throws HiveException {
+    if (deferredObjects.length != 1) {
+      throw new UDFArgumentException(
+          String.format(ARGUMENT_LIST_LENGTH_FORMAT, deferredObjects.length));
+    }
+    DeferredObject argument = deferredObjects[0];
+    if (argument.get() == null) {
+      return null;
+    }
+    Object object = argument.get();
+
+    switch (argAndRetType_) {
+      case STRING: {
+        if (!(object instanceof Text)) {
+          throw new HiveException("Expected Text but got " + object.getClass());
+        }
+        Text text = (Text) object;
+        byte[] bytes = text.getBytes();
+        incrementByteArray(bytes);
+        return text;
+      }
+      case BINARY: {
+        if (!(object instanceof BytesWritable)) {
+          throw new HiveException("Expected BytesWritable but got " + object.getClass());
+        }
+        BytesWritable bytesWritable = (BytesWritable) object;
+        byte[] bytes = bytesWritable.getBytes();
+        incrementByteArray(bytes);
+        return bytesWritable;
+      }
+      default: throw new IllegalStateException("Unexpected type: " + argAndRetType_);
+    }
+  }
+
+  @Override
+  public String getDisplayString(String[] strings) {
+    return "GenericBufferAltering";
+  }
+
+  private void incrementByteArray(byte[] array) {
+    if (array.length > 0) {
+      array[0] += 1;
+    }
+  }
+}
diff --git a/testdata/workloads/functional-query/queries/QueryTest/generic-java-udf.test b/testdata/workloads/functional-query/queries/QueryTest/generic-java-udf.test
index 3c9fca948..3bd35e8ed 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/generic-java-udf.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/generic-java-udf.test
@@ -335,3 +335,31 @@ symbol='org.apache.impala.TestUdf';
 ---- CATCH
 CatalogException: Variable arguments not supported in Hive UDFs.
 ====
+---- QUERY
+select increment("a");
+---- TYPES
+STRING
+---- RESULTS
+'b'
+====
+---- QUERY
+select increment(NULL);
+---- TYPES
+STRING
+---- RESULTS
+'NULL'
+====
+---- QUERY
+select increment("");
+---- TYPES
+STRING
+---- RESULTS
+''
+====
+---- QUERY
+select increment(cast("a" as binary));
+---- TYPES
+BINARY
+---- RESULTS
+'b'
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/java-udf.test b/testdata/workloads/functional-query/queries/QueryTest/java-udf.test
index 96769dc77..fb2bc4f2a 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/java-udf.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/java-udf.test
@@ -349,3 +349,38 @@ symbol='org.apache.impala.TestUdf';
 ---- CATCH
 CatalogException: Variable arguments not supported in Hive UDFs.
 ====
+---- QUERY
+select increment("a");
+---- TYPES
+STRING
+---- RESULTS
+'b'
+====
+---- QUERY
+select increment(NULL);
+---- TYPES
+STRING
+---- RESULTS
+'NULL'
+====
+---- QUERY
+select increment("");
+---- TYPES
+STRING
+---- RESULTS
+''
+====
+---- QUERY
+select increment(cast("a" as binary));
+---- TYPES
+BINARY
+---- RESULTS
+'b'
+====
+---- QUERY
+select copy_and_increment(cast("bbb" as binary), cast("aaaa" as binary));
+---- TYPES
+BINARY
+---- RESULTS
+'baaa'
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/load-generic-java-udfs.test b/testdata/workloads/functional-query/queries/QueryTest/load-generic-java-udfs.test
index ca3415c9c..e029a1a59 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/load-generic-java-udfs.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/load-generic-java-udfs.test
@@ -139,4 +139,12 @@ symbol='org.apache.impala.TestGenericUdfWithJavaReturnTypes';
 create function generic_add_java_ret_type(string, string, string) returns string
 location '$FILESYSTEM_PREFIX/test-warehouse/impala-hive-udfs.jar'
 symbol='org.apache.impala.TestGenericUdfWithJavaReturnTypes';
+
+create function increment(binary) returns binary
+location '$FILESYSTEM_PREFIX/test-warehouse/impala-hive-udfs.jar'
+symbol='org.apache.impala.GenericBufferAlteringUdf';
+
+create function increment(string) returns string
+location '$FILESYSTEM_PREFIX/test-warehouse/impala-hive-udfs.jar'
+symbol='org.apache.impala.GenericBufferAlteringUdf';
 ====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/load-java-udfs.test b/testdata/workloads/functional-query/queries/QueryTest/load-java-udfs.test
index 45827011b..81c8d579a 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/load-java-udfs.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/load-java-udfs.test
@@ -88,4 +88,16 @@ symbol='org.apache.impala.ReplaceStringUdf';
 create function import_nearby_classes(string) returns string
 location '$FILESYSTEM_PREFIX/test-warehouse/impala-hive-udfs.jar'
 symbol='org.apache.impala.ImportsNearbyClassesUdf';
+
+create function increment(binary) returns binary
+location '$FILESYSTEM_PREFIX/test-warehouse/impala-hive-udfs.jar'
+symbol='org.apache.impala.BufferAlteringUdf';
+
+create function increment(string) returns string
+location '$FILESYSTEM_PREFIX/test-warehouse/impala-hive-udfs.jar'
+symbol='org.apache.impala.BufferAlteringUdf';
+
+create function copy_and_increment(binary, binary) returns binary
+location '$FILESYSTEM_PREFIX/test-warehouse/impala-hive-udfs.jar'
+symbol='org.apache.impala.BufferAlteringUdf';
 ====