You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2019/09/25 22:30:22 UTC

[impala] branch master updated (750f659 -> ab975c9)

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

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


    from 750f659  IMPALA-8926, IMPALA-8957: Fix result spooling flaky tests
     new 803323b  IMPALA-7637: Add more hash table stats to profile
     new 3c193c3  IMPALA-2138: part 2: clean up result expr handling
     new ab975c9  IMPALA-8969: Grouping aggregator can cause segmentation fault when doing multiple aggregations

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


Summary of changes:
 be/src/benchmarks/expr-benchmark.cc                |   2 +-
 be/src/exec/data-sink.cc                           |   2 +-
 be/src/exec/grouping-aggregator-ir.cc              |   2 +-
 be/src/exec/grouping-aggregator-partition.cc       |   5 +-
 be/src/exec/grouping-aggregator.cc                 |   5 +-
 be/src/exec/grouping-aggregator.h                  |   4 +-
 be/src/exec/hash-table.cc                          |  25 +
 be/src/exec/hash-table.h                           |  40 +-
 be/src/exec/partitioned-hash-join-builder.cc       |  10 +-
 be/src/exec/partitioned-hash-join-builder.h        |  11 +-
 common/thrift/DataSinks.thrift                     |   5 +
 common/thrift/Planner.thrift                       |   4 -
 .../org/apache/impala/analysis/DeleteStmt.java     |   4 +-
 .../main/java/org/apache/impala/analysis/Expr.java |  16 +
 .../org/apache/impala/analysis/InsertStmt.java     |   4 +-
 .../org/apache/impala/analysis/ModifyStmt.java     |   2 +-
 .../java/org/apache/impala/analysis/QueryStmt.java |   4 +-
 .../org/apache/impala/analysis/UpdateStmt.java     |   4 +-
 .../org/apache/impala/planner/AggregationNode.java |   6 +-
 .../apache/impala/planner/AnalyticEvalNode.java    |   2 +-
 .../java/org/apache/impala/planner/DataSink.java   |   8 +
 .../apache/impala/planner/DataSourceScanNode.java  |   6 +-
 .../org/apache/impala/planner/DataStreamSink.java  |   8 +
 .../org/apache/impala/planner/HBaseScanNode.java   |   7 +-
 .../org/apache/impala/planner/HBaseTableSink.java  |  16 +-
 .../org/apache/impala/planner/HashJoinNode.java    |   4 +-
 .../org/apache/impala/planner/HdfsScanNode.java    |  13 +-
 .../org/apache/impala/planner/HdfsTableSink.java   |  16 +-
 .../org/apache/impala/planner/JoinBuildSink.java   |   5 +
 .../org/apache/impala/planner/KuduScanNode.java    |   4 +-
 .../org/apache/impala/planner/KuduTableSink.java   |  15 +-
 .../apache/impala/planner/NestedLoopJoinNode.java  |   4 +-
 .../org/apache/impala/planner/PlanFragment.java    |  11 -
 .../java/org/apache/impala/planner/PlanNode.java   |  16 -
 .../org/apache/impala/planner/PlanRootSink.java    |  23 +
 .../java/org/apache/impala/planner/Planner.java    |  32 +-
 .../java/org/apache/impala/planner/SelectNode.java |   2 +-
 .../org/apache/impala/planner/SubplanNode.java     |   3 +-
 .../java/org/apache/impala/planner/TableSink.java  |  24 +-
 .../java/org/apache/impala/planner/UnionNode.java  |   2 +-
 .../java/org/apache/impala/planner/UnnestNode.java |   2 +-
 .../queries/PlannerTest/constant-folding.test      |  40 +-
 ...k-join-detection-hdfs-num-rows-est-enabled.test |  10 +-
 .../queries/PlannerTest/fk-pk-join-detection.test  |  61 +-
 .../queries/PlannerTest/kudu-selectivity.test      |  13 +
 .../queries/PlannerTest/max-row-size.test          |  49 +-
 ...-runtime-filters-hdfs-num-rows-est-enabled.test |   9 +-
 .../PlannerTest/min-max-runtime-filters.test       |   6 +-
 ...t-dop-validation-hdfs-num-rows-est-enabled.test |  20 +-
 .../queries/PlannerTest/mt-dop-validation.test     |  50 +-
 .../PlannerTest/parquet-filtering-disabled.test    |  41 +-
 .../queries/PlannerTest/parquet-filtering.test     |  71 ++-
 .../queries/PlannerTest/partition-pruning.test     |   3 +-
 .../queries/PlannerTest/resource-requirements.test | 133 +++++
 .../queries/PlannerTest/result-spooling.test       |   5 +
 .../PlannerTest/sort-expr-materialization.test     |  27 +-
 .../PlannerTest/spillable-buffer-sizing.test       | 126 +++--
 .../queries/PlannerTest/tablesample.test           |  35 +-
 .../queries/PlannerTest/union.test                 | 624 +++++++++++----------
 .../queries/QueryTest/explain-level2.test          |   1 +
 .../queries/QueryTest/explain-level3.test          |   1 +
 .../queries/QueryTest/multiple-distinct-aggs.test  |  18 +-
 .../queries/QueryTest/stats-extrapolation.test     |   7 +-
 tests/query_test/test_observability.py             |  28 +
 64 files changed, 1102 insertions(+), 654 deletions(-)


[impala] 02/03: IMPALA-2138: part 2: clean up result expr handling

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

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

commit 3c193c33b2f3004588fa435d7d545fb8fd4db552
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Thu Sep 19 17:03:07 2019 -0700

    IMPALA-2138: part 2: clean up result expr handling
    
    The main refactoring is to move result expressions into the
    DataSink implementations, which is where they are used
    in the backend. This will make it easier to explicitly collect
    all the expressions in the plan tree for the purposes of
    projection. Previously the expressions were owned by
    the PlanFragment and passed into the DataSink.
    
    Show result exprs in explain plan of the table sinks
    at higher verbosity.
    
    Change-Id: I163a393b5ce6b8a926b3fee9b4b920e31d6846b2
    Reviewed-on: http://gerrit.cloudera.org:8080/14270
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/benchmarks/expr-benchmark.cc                |   2 +-
 be/src/exec/data-sink.cc                           |   2 +-
 common/thrift/DataSinks.thrift                     |   5 +
 common/thrift/Planner.thrift                       |   4 -
 .../org/apache/impala/analysis/DeleteStmt.java     |   4 +-
 .../main/java/org/apache/impala/analysis/Expr.java |  16 +
 .../org/apache/impala/analysis/InsertStmt.java     |   4 +-
 .../org/apache/impala/analysis/ModifyStmt.java     |   2 +-
 .../java/org/apache/impala/analysis/QueryStmt.java |   4 +-
 .../org/apache/impala/analysis/UpdateStmt.java     |   4 +-
 .../org/apache/impala/planner/AggregationNode.java |   6 +-
 .../apache/impala/planner/AnalyticEvalNode.java    |   2 +-
 .../java/org/apache/impala/planner/DataSink.java   |   8 +
 .../apache/impala/planner/DataSourceScanNode.java  |   6 +-
 .../org/apache/impala/planner/DataStreamSink.java  |   8 +
 .../org/apache/impala/planner/HBaseScanNode.java   |   7 +-
 .../org/apache/impala/planner/HBaseTableSink.java  |  16 +-
 .../org/apache/impala/planner/HashJoinNode.java    |   4 +-
 .../org/apache/impala/planner/HdfsScanNode.java    |  13 +-
 .../org/apache/impala/planner/HdfsTableSink.java   |  16 +-
 .../org/apache/impala/planner/JoinBuildSink.java   |   5 +
 .../org/apache/impala/planner/KuduScanNode.java    |   4 +-
 .../org/apache/impala/planner/KuduTableSink.java   |  15 +-
 .../apache/impala/planner/NestedLoopJoinNode.java  |   4 +-
 .../org/apache/impala/planner/PlanFragment.java    |  11 -
 .../java/org/apache/impala/planner/PlanNode.java   |  16 -
 .../org/apache/impala/planner/PlanRootSink.java    |  23 +
 .../java/org/apache/impala/planner/Planner.java    |  32 +-
 .../java/org/apache/impala/planner/SelectNode.java |   2 +-
 .../org/apache/impala/planner/SubplanNode.java     |   3 +-
 .../java/org/apache/impala/planner/TableSink.java  |  24 +-
 .../java/org/apache/impala/planner/UnionNode.java  |   2 +-
 .../java/org/apache/impala/planner/UnnestNode.java |   2 +-
 .../queries/PlannerTest/constant-folding.test      |  40 +-
 ...k-join-detection-hdfs-num-rows-est-enabled.test |  10 +-
 .../queries/PlannerTest/fk-pk-join-detection.test  |  61 +-
 .../queries/PlannerTest/kudu-selectivity.test      |  13 +
 .../queries/PlannerTest/max-row-size.test          |  49 +-
 ...-runtime-filters-hdfs-num-rows-est-enabled.test |   9 +-
 .../PlannerTest/min-max-runtime-filters.test       |   6 +-
 ...t-dop-validation-hdfs-num-rows-est-enabled.test |  20 +-
 .../queries/PlannerTest/mt-dop-validation.test     |  50 +-
 .../PlannerTest/parquet-filtering-disabled.test    |  41 +-
 .../queries/PlannerTest/parquet-filtering.test     |  71 ++-
 .../queries/PlannerTest/partition-pruning.test     |   3 +-
 .../queries/PlannerTest/resource-requirements.test | 133 +++++
 .../queries/PlannerTest/result-spooling.test       |   5 +
 .../PlannerTest/sort-expr-materialization.test     |  27 +-
 .../PlannerTest/spillable-buffer-sizing.test       | 126 +++--
 .../queries/PlannerTest/tablesample.test           |  35 +-
 .../queries/PlannerTest/union.test                 | 624 +++++++++++----------
 .../queries/QueryTest/explain-level2.test          |   1 +
 .../queries/QueryTest/explain-level3.test          |   1 +
 .../queries/QueryTest/stats-extrapolation.test     |   7 +-
 54 files changed, 976 insertions(+), 632 deletions(-)

diff --git a/be/src/benchmarks/expr-benchmark.cc b/be/src/benchmarks/expr-benchmark.cc
index 5715cc1..79ad8ae 100644
--- a/be/src/benchmarks/expr-benchmark.cc
+++ b/be/src/benchmarks/expr-benchmark.cc
@@ -110,7 +110,7 @@ MemPool mem_pool(&tracker);
 static Status PrepareSelectList(
     const TExecRequest& request, ScalarExprEvaluator** eval) {
   const TQueryExecRequest& query_request = request.query_exec_request;
-  vector<TExpr> texprs = query_request.plan_exec_info[0].fragments[0].output_exprs;
+  vector<TExpr> texprs = query_request.plan_exec_info[0].fragments[0].output_sink.output_exprs;
   DCHECK_EQ(texprs.size(), 1);
   RuntimeState* state = planner->GetRuntimeState();
   ScalarExpr* expr;
diff --git a/be/src/exec/data-sink.cc b/be/src/exec/data-sink.cc
index 68780e6..ed3820f 100644
--- a/be/src/exec/data-sink.cc
+++ b/be/src/exec/data-sink.cc
@@ -69,7 +69,7 @@ Status DataSink::Create(const TPlanFragmentCtx& fragment_ctx,
     const TPlanFragmentInstanceCtx& fragment_instance_ctx, const RowDescriptor* row_desc,
     RuntimeState* state, DataSink** sink) {
   const TDataSink& thrift_sink = fragment_ctx.fragment.output_sink;
-  const vector<TExpr>& thrift_output_exprs = fragment_ctx.fragment.output_exprs;
+  const vector<TExpr>& thrift_output_exprs = thrift_sink.output_exprs;
   ObjectPool* pool = state->obj_pool();
   // We have one fragment per sink, so we can use the fragment index as the sink ID.
   TDataSinkId sink_id = fragment_ctx.fragment.idx;
diff --git a/common/thrift/DataSinks.thrift b/common/thrift/DataSinks.thrift
index 3c26c20..ddb15d5 100644
--- a/common/thrift/DataSinks.thrift
+++ b/common/thrift/DataSinks.thrift
@@ -62,6 +62,7 @@ struct TDataStreamSink {
 // and materializes all its input RowBatches as a Hdfs file.
 struct THdfsTableSink {
   1: required list<Exprs.TExpr> partition_key_exprs
+
   2: required bool overwrite
 
   // The 'skip.header.line.count' property of the target Hdfs table. We will insert this
@@ -127,4 +128,8 @@ struct TDataSink {
 
   // Estimated execution stats generated by the planner.
   7: optional ExecStats.TExecStats estimated_stats
+
+  // Exprs that produce values for slots of output tuple (one expr per slot).
+  // Only set by the DataSink implementations that require it.
+  8: optional list<Exprs.TExpr> output_exprs
 }
diff --git a/common/thrift/Planner.thrift b/common/thrift/Planner.thrift
index 2619b44..e0c8911 100644
--- a/common/thrift/Planner.thrift
+++ b/common/thrift/Planner.thrift
@@ -40,10 +40,6 @@ struct TPlanFragment {
   // no plan or descriptor table: query without From clause
   3: optional PlanNodes.TPlan plan
 
-  // exprs that produce values for slots of output tuple (one expr per slot);
-  // if not set, plan fragment materializes full rows of plan_tree
-  4: optional list<Exprs.TExpr> output_exprs
-
   // Specifies the destination of this plan fragment's output rows.
   // For example, the destination could be a stream sink which forwards
   // the data to a remote plan fragment, or a sink which writes to a table (for
diff --git a/fe/src/main/java/org/apache/impala/analysis/DeleteStmt.java b/fe/src/main/java/org/apache/impala/analysis/DeleteStmt.java
index 6caa95c..796486b 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DeleteStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DeleteStmt.java
@@ -51,11 +51,11 @@ public class DeleteStmt extends ModifyStmt {
   }
 
   @Override
-  public DataSink createDataSink() {
+  public DataSink createDataSink(List<Expr> resultExprs) {
     // analyze() must have been called before.
     Preconditions.checkState(table_ != null);
     TableSink tableSink = TableSink.create(table_, TableSink.Op.DELETE,
-        ImmutableList.<Expr>of(), referencedColumns_, false, false,
+        ImmutableList.<Expr>of(), resultExprs, referencedColumns_, false, false,
         ImmutableList.<Integer>of());
     Preconditions.checkState(!referencedColumns_.isEmpty());
     return tableSink;
diff --git a/fe/src/main/java/org/apache/impala/analysis/Expr.java b/fe/src/main/java/org/apache/impala/analysis/Expr.java
index 76e62d6..e4dfeaa 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Expr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Expr.java
@@ -41,6 +41,7 @@ import org.apache.impala.common.TreeNode;
 import org.apache.impala.rewrite.ExprRewriter;
 import org.apache.impala.service.FeSupport;
 import org.apache.impala.thrift.TColumnValue;
+import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TExpr;
 import org.apache.impala.thrift.TExprNode;
 import org.apache.impala.thrift.TFunction;
@@ -1592,6 +1593,21 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
     return Joiner.on(",").join(Iterables.transform(exprs, toSql));
   }
 
+  public static String getExplainString(
+      List<? extends Expr> exprs, TExplainLevel detailLevel) {
+    if (exprs == null) return "";
+    ToSqlOptions toSqlOptions =
+        detailLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal() ?
+        ToSqlOptions.SHOW_IMPLICIT_CASTS :
+        ToSqlOptions.DEFAULT;
+    StringBuilder output = new StringBuilder();
+    for (int i = 0; i < exprs.size(); ++i) {
+      if (i > 0) output.append(", ");
+      output.append(exprs.get(i).toSql(toSqlOptions));
+    }
+    return output.toString();
+  }
+
   /**
    * Analyzes and evaluates expression to an integral value, returned as a long.
    * Throws if the expression cannot be evaluated or if the value evaluates to null.
diff --git a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
index 13d83c2..a5371ec 100644
--- a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
@@ -940,8 +940,8 @@ public class InsertStmt extends StatementBase {
     // analyze() must have been called before.
     Preconditions.checkState(table_ != null);
     return TableSink.create(table_, isUpsert_ ? TableSink.Op.UPSERT : TableSink.Op.INSERT,
-        partitionKeyExprs_, mentionedColumns_, overwrite_, requiresClustering(),
-        sortColumns_, writeId_);
+        partitionKeyExprs_, resultExprs_, mentionedColumns_, overwrite_,
+        requiresClustering(), sortColumns_, writeId_);
   }
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java b/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
index bdacc88..06ec415 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ModifyStmt.java
@@ -313,7 +313,7 @@ public abstract class ModifyStmt extends StatementBase {
   }
 
   public QueryStmt getQueryStmt() { return sourceStmt_; }
-  public abstract DataSink createDataSink();
+  public abstract DataSink createDataSink(List<Expr> resultExprs);
   @Override
   public abstract String toSql(ToSqlOptions options);
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java b/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
index f792e40..7dacfcf 100644
--- a/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
@@ -477,8 +477,8 @@ public abstract class QueryStmt extends StatementBase {
     resultExprs_ = Expr.substituteList(resultExprs_, smap, analyzer, true);
   }
 
-  public DataSink createDataSink() {
-    return new PlanRootSink();
+  public DataSink createDataSink(List<Expr> resultExprs) {
+    return new PlanRootSink(resultExprs);
   }
 
   public List<OrderByElement> cloneOrderByElements() {
diff --git a/fe/src/main/java/org/apache/impala/analysis/UpdateStmt.java b/fe/src/main/java/org/apache/impala/analysis/UpdateStmt.java
index 28af740..f38c253 100644
--- a/fe/src/main/java/org/apache/impala/analysis/UpdateStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/UpdateStmt.java
@@ -61,11 +61,11 @@ public class UpdateStmt extends ModifyStmt {
    * Return an instance of a KuduTableSink specialized as an Update operation.
    */
   @Override
-  public DataSink createDataSink() {
+  public DataSink createDataSink(List<Expr> resultExprs) {
     // analyze() must have been called before.
     Preconditions.checkState(table_ != null);
     DataSink dataSink = TableSink.create(table_, TableSink.Op.UPDATE,
-        ImmutableList.<Expr>of(), referencedColumns_, false, false,
+        ImmutableList.<Expr>of(), resultExprs, referencedColumns_, false, false,
         ImmutableList.<Integer>of());
     Preconditions.checkState(!referencedColumns_.isEmpty());
     return dataSink;
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 f48b2de..5a03452 100644
--- a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
@@ -458,7 +458,7 @@ public class AggregationNode extends PlanNode {
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix)
             .append("having: ")
-            .append(getExplainString(conjuncts_, detailLevel))
+            .append(Expr.getExplainString(conjuncts_, detailLevel))
             .append("\n");
       }
     }
@@ -473,13 +473,13 @@ public class AggregationNode extends PlanNode {
     if (!aggExprs.isEmpty()) {
       output.append(prefix)
           .append("output: ")
-          .append(getExplainString(aggExprs, detailLevel))
+          .append(Expr.getExplainString(aggExprs, detailLevel))
           .append("\n");
     }
     if (!groupingExprs.isEmpty()) {
       output.append(prefix)
           .append("group by: ")
-          .append(getExplainString(groupingExprs, detailLevel))
+          .append(Expr.getExplainString(groupingExprs, detailLevel))
           .append("\n");
     }
     return output;
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 12f7fcf..d019619 100644
--- a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
@@ -235,7 +235,7 @@ public class AnalyticEvalNode extends PlanNode {
 
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix
-            + "predicates: " + getExplainString(conjuncts_, detailLevel) + "\n");
+            + "predicates: " + Expr.getExplainString(conjuncts_, detailLevel) + "\n");
       }
     }
     return output.toString();
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 cb4426c..b78e67c 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSink.java
@@ -17,6 +17,9 @@
 
 package org.apache.impala.planner;
 
+import java.util.List;
+
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExecStats;
@@ -97,4 +100,9 @@ public abstract class DataSink {
    */
   public abstract void computeResourceProfile(TQueryOptions queryOptions);
 
+  /**
+   * 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 d048723..c65677f 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
@@ -358,11 +358,11 @@ public class DataSourceScanNode extends ScanNode {
 
     if (!acceptedConjuncts_.isEmpty()) {
       output.append(prefix + "data source predicates: "
-          + getExplainString(acceptedConjuncts_, detailLevel) + "\n");
+          + Expr.getExplainString(acceptedConjuncts_, detailLevel) + "\n");
     }
     if (!conjuncts_.isEmpty()) {
-      output.append(
-          prefix + "predicates: " + getExplainString(conjuncts_, detailLevel) + "\n");
+      output.append(prefix + "predicates: " +
+            Expr.getExplainString(conjuncts_, detailLevel) + "\n");
     }
 
     // Add table and column stats in verbose mode.
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 5a2a8b4..3cffe68 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
@@ -17,6 +17,9 @@
 
 package org.apache.impala.planner;
 
+import java.util.List;
+
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TDataStreamSink;
@@ -72,4 +75,9 @@ public class DataStreamSink extends DataSink {
   }
 
   public DataPartition getOutputPartition() { return outputPartition_; }
+
+  @Override
+  public void collectExprs(List<Expr> exprs) {
+    exprs.addAll(outputPartition_.getPartitionExprs());
+  }
 }
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 d0b5b1d..13960fa 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
@@ -45,7 +45,6 @@ import org.apache.impala.catalog.PrimitiveType;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.Pair;
-import org.apache.impala.service.FeSupport;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.THBaseFilter;
 import org.apache.impala.thrift.THBaseKeyRange;
@@ -562,8 +561,8 @@ public class HBaseScanNode extends ScanNode {
         displayName_, table.getFullName(), aliasStr));
     if (detailLevel.ordinal() >= TExplainLevel.STANDARD.ordinal()) {
       if (!keyConjuncts_.isEmpty()) {
-        output.append(detailPrefix
-            + "key predicates: " + getExplainString(keyConjuncts_, detailLevel) + "\n");
+        output.append(detailPrefix + "key predicates: " +
+            Expr.getExplainString(keyConjuncts_, detailLevel) + "\n");
       }
       if (!Bytes.equals(startKey_, HConstants.EMPTY_START_ROW)) {
         output.append(detailPrefix + "start key: " + printKey(startKey_) + "\n");
@@ -590,7 +589,7 @@ public class HBaseScanNode extends ScanNode {
       }
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix
-            + "predicates: " + getExplainString(conjuncts_, detailLevel) + "\n");
+            + "predicates: " + Expr.getExplainString(conjuncts_, detailLevel) + "\n");
       }
     }
     if (detailLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
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 abdf2db..cfbb335 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java
@@ -18,7 +18,10 @@
 
 package org.apache.impala.planner;
 
+import java.util.List;
+
 import org.apache.impala.analysis.DescriptorTable;
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.catalog.FeTable;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
@@ -32,14 +35,18 @@ import org.apache.impala.thrift.TTableSinkType;
  * data from a plan fragment into an HBase table using HTable.
  */
 public class HBaseTableSink extends TableSink {
-  public HBaseTableSink(FeTable targetTable) {
-    super(targetTable, Op.INSERT);
+  public HBaseTableSink(FeTable targetTable, List<Expr> outputExprs) {
+    super(targetTable, Op.INSERT, outputExprs);
   }
 
   @Override
   public void appendSinkExplainString(String prefix, String detailPrefix,
       TQueryOptions queryOptions, TExplainLevel explainLevel, StringBuilder output) {
     output.append(prefix + "WRITE TO HBASE table=" + targetTable_.getFullName() + "\n");
+    if (explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+      output.append(detailPrefix + "output exprs: ")
+          .append(Expr.getExplainString(outputExprs_, explainLevel) + "\n");
+    }
   }
 
   @Override
@@ -57,6 +64,7 @@ public class HBaseTableSink extends TableSink {
     TTableSink tTableSink = new TTableSink(DescriptorTable.TABLE_SINK_ID,
         TTableSinkType.HBASE, sinkOp_.toThrift());
     tsink.table_sink = tTableSink;
+    tsink.output_exprs = Expr.treesToThrift(outputExprs_);
   }
 
   @Override
@@ -64,4 +72,8 @@ public class HBaseTableSink extends TableSink {
     return TDataSinkType.TABLE_SINK;
   }
 
+  @Override
+  public void collectExprs(List<Expr> exprs) {
+    exprs.addAll(outputExprs_);
+  }
 }
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 5360fe2..053fffb 100644
--- a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
@@ -188,11 +188,11 @@ public class HashJoinNode extends JoinNode {
 
       if (!otherJoinConjuncts_.isEmpty()) {
         output.append(detailPrefix + "other join predicates: ")
-            .append(getExplainString(otherJoinConjuncts_, detailLevel) + "\n");
+            .append(Expr.getExplainString(otherJoinConjuncts_, detailLevel) + "\n");
       }
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix + "other predicates: ")
-            .append(getExplainString(conjuncts_, detailLevel) + "\n");
+            .append(Expr.getExplainString(conjuncts_, detailLevel) + "\n");
       }
       if (!runtimeFilters_.isEmpty()) {
         output.append(detailPrefix + "runtime filters: ");
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 c4885b6..716b611 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -94,7 +94,6 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Objects.ToStringHelper;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
@@ -1412,7 +1411,7 @@ public class HdfsScanNode extends ScanNode {
       if (partitionConjuncts_ != null && !partitionConjuncts_.isEmpty()) {
         output.append(detailPrefix)
           .append(String.format("partition predicates: %s\n",
-              getExplainString(partitionConjuncts_, detailLevel)));
+              Expr.getExplainString(partitionConjuncts_, detailLevel)));
       }
       String partMetaTemplate = "partitions=%d/%d files=%d size=%s\n";
       if (!numPartitionsPerFs_.isEmpty()) {
@@ -1444,7 +1443,7 @@ public class HdfsScanNode extends ScanNode {
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix)
           .append(String.format("predicates: %s\n",
-            getExplainString(conjuncts_, detailLevel)));
+              Expr.getExplainString(conjuncts_, detailLevel)));
       }
       if (!collectionConjuncts_.isEmpty()) {
         for (Map.Entry<TupleDescriptor, List<Expr>> entry:
@@ -1452,7 +1451,7 @@ public class HdfsScanNode extends ScanNode {
           String alias = entry.getKey().getAlias();
           output.append(detailPrefix)
             .append(String.format("predicates on %s: %s\n", alias,
-              getExplainString(entry.getValue(), detailLevel)));
+                Expr.getExplainString(entry.getValue(), detailLevel)));
         }
       }
       if (!runtimeFilters_.isEmpty()) {
@@ -1501,11 +1500,11 @@ public class HdfsScanNode extends ScanNode {
       if (tupleDesc == getTupleDesc()) {
         output.append(prefix)
         .append(String.format("parquet statistics predicates: %s\n",
-            getExplainString(exprs, detailLevel)));
+            Expr.getExplainString(exprs, detailLevel)));
       } else {
         output.append(prefix)
         .append(String.format("parquet statistics predicates on %s: %s\n",
-            tupleDesc.getAlias(), getExplainString(exprs, detailLevel)));
+            tupleDesc.getAlias(), Expr.getExplainString(exprs, detailLevel)));
       }
     }
     return output.toString();
@@ -1550,7 +1549,7 @@ public class HdfsScanNode extends ScanNode {
         exprList.add(conjuncts.get(idx));
       }
       output.append(String.format("%sparquet dictionary predicates%s: %s\n", prefix,
-          tupleName, getExplainString(exprList, detailLevel)));
+          tupleName, Expr.getExplainString(exprList, detailLevel)));
     }
     return output.toString();
   }
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 ea7fc3d..3eaf356 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
@@ -51,6 +51,7 @@ public class HdfsTableSink extends TableSink {
 
   // Exprs for computing the output partition(s).
   protected final List<Expr> partitionKeyExprs_;
+
   // Whether to overwrite the existing partition(s).
   protected final boolean overwrite_;
 
@@ -71,9 +72,10 @@ public class HdfsTableSink extends TableSink {
   private long writeId_;
 
   public HdfsTableSink(FeTable targetTable, List<Expr> partitionKeyExprs,
+      List<Expr> outputExprs,
       boolean overwrite, boolean inputIsClustered, List<Integer> sortColumns,
       long writeId) {
-    super(targetTable, Op.INSERT);
+    super(targetTable, Op.INSERT, outputExprs);
     Preconditions.checkState(targetTable instanceof FeFsTable);
     partitionKeyExprs_ = partitionKeyExprs;
     overwrite_ = overwrite;
@@ -176,6 +178,10 @@ public class HdfsTableSink extends TableSink {
       }
       output.append("\n");
     }
+    if (explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+      output.append(detailPrefix + "output exprs: ")
+          .append(Expr.getExplainString(outputExprs_, explainLevel) + "\n");
+    }
   }
 
   @Override
@@ -202,10 +208,18 @@ public class HdfsTableSink extends TableSink {
         TTableSinkType.HDFS, sinkOp_.toThrift());
     tTableSink.hdfs_table_sink = hdfsTableSink;
     tsink.table_sink = tTableSink;
+    tsink.output_exprs = Expr.treesToThrift(outputExprs_);
   }
 
   @Override
   protected TDataSinkType getSinkType() {
     return TDataSinkType.TABLE_SINK;
   }
+
+  @Override
+  public void collectExprs(List<Expr> exprs) {
+    exprs.addAll(partitionKeyExprs_);
+    // Avoid adding any partition exprs redundantly.
+    exprs.addAll(outputExprs_.subList(0, targetTable_.getNonClusteringColumns().size()));
+  }
 }
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 25ea7da..5610646 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
@@ -99,4 +99,9 @@ public class JoinBuildSink extends DataSink {
     // The memory consumption is counted against the join PlanNode.
     resourceProfile_ = ResourceProfile.noReservation(0);
   }
+
+  @Override
+  public void collectExprs(List<Expr> exprs) {
+    exprs.addAll(buildExprs_);
+  }
 }
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 62e6419..90e1b0f 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -313,11 +313,11 @@ public class KuduScanNode extends ScanNode {
       case VERBOSE: {
         if (!conjuncts_.isEmpty()) {
           result.append(detailPrefix
-              + "predicates: " + getExplainString(conjuncts_, detailLevel) + "\n");
+              + "predicates: " + Expr.getExplainString(conjuncts_, detailLevel) + "\n");
         }
         if (!kuduConjuncts_.isEmpty()) {
           result.append(detailPrefix + "kudu predicates: "
-              + getExplainString(kuduConjuncts_, detailLevel) + "\n");
+              + Expr.getExplainString(kuduConjuncts_, detailLevel) + "\n");
         }
         if (!runtimeFilters_.isEmpty()) {
           result.append(detailPrefix + "runtime filters: ");
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 a395279..4721228 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java
@@ -21,6 +21,7 @@ package org.apache.impala.planner;
 import java.util.List;
 
 import org.apache.impala.analysis.DescriptorTable;
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.catalog.FeTable;
 import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TDataSink;
@@ -44,8 +45,8 @@ public class KuduTableSink extends TableSink {
   private final List<Integer> targetColIdxs_;
 
   public KuduTableSink(FeTable targetTable, Op sinkOp,
-      List<Integer> referencedColumns) {
-    super(targetTable, sinkOp);
+      List<Integer> referencedColumns, List<Expr> outputExprs) {
+    super(targetTable, sinkOp, outputExprs);
     targetColIdxs_ = referencedColumns != null
         ? Lists.newArrayList(referencedColumns) : null;
   }
@@ -55,6 +56,10 @@ public class KuduTableSink extends TableSink {
       TQueryOptions queryOptions, TExplainLevel explainLevel, StringBuilder output) {
     output.append(prefix + sinkOp_.toExplainString());
     output.append(" KUDU [" + targetTable_.getFullName() + "]\n");
+    if (explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+      output.append(detailPrefix + "output exprs: ")
+          .append(Expr.getExplainString(outputExprs_, explainLevel) + "\n");
+    }
   }
 
   @Override
@@ -84,10 +89,16 @@ public class KuduTableSink extends TableSink {
     tKuduSink.setReferenced_columns(targetColIdxs_);
     tTableSink.setKudu_table_sink(tKuduSink);
     tsink.table_sink = tTableSink;
+    tsink.output_exprs = Expr.treesToThrift(outputExprs_);
   }
 
   @Override
   protected TDataSinkType getSinkType() {
     return TDataSinkType.TABLE_SINK;
   }
+
+  @Override
+  public void collectExprs(List<Expr> exprs) {
+    exprs.addAll(outputExprs_);
+  }
 }
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 afa75e7..6f2f4f9 100644
--- a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
@@ -105,11 +105,11 @@ public class NestedLoopJoinNode extends JoinNode {
       }
       if (!otherJoinConjuncts_.isEmpty()) {
         output.append(detailPrefix + "join predicates: ")
-            .append(getExplainString(otherJoinConjuncts_, detailLevel) + "\n");
+            .append(Expr.getExplainString(otherJoinConjuncts_, detailLevel) + "\n");
       }
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix + "predicates: ")
-            .append(getExplainString(conjuncts_, detailLevel) + "\n");
+            .append(Expr.getExplainString(conjuncts_, detailLevel) + "\n");
       }
     }
     return output.toString();
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 b55a546..5c039c4 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -84,9 +84,6 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   // exchange node to which this fragment sends its output
   private ExchangeNode destNode_;
 
-  // if null, outputs the entire row produced by planRoot_
-  private List<Expr> outputExprs_;
-
   // created in finalize() or set in setSink()
   private DataSink sink_;
 
@@ -156,11 +153,6 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     for (PlanNode child: root.getChildren()) collectPlanNodesHelper(child, nodes);
   }
 
-  public void setOutputExprs(List<Expr> outputExprs) {
-    outputExprs_ = Expr.cloneList(outputExprs);
-  }
-  public List<Expr> getOutputExprs() { return outputExprs_; }
-
   /**
    * 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
@@ -335,9 +327,6 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     TPlanFragment result = new TPlanFragment();
     result.setDisplay_name(fragmentId_.toString());
     if (planRoot_ != null) result.setPlan(planRoot_.treeToThrift());
-    if (outputExprs_ != null) {
-      result.setOutput_exprs(Expr.treesToThrift(outputExprs_));
-    }
     if (sink_ != null) result.setOutput_sink(sink_.toThrift());
     result.setPartition(dataPartition_.toThrift());
     if (resourceProfile_.isValid()) {
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 43642ab..0ab96e1 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -29,7 +29,6 @@ import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.ExprId;
 import org.apache.impala.analysis.ExprSubstitutionMap;
-import org.apache.impala.analysis.ToSqlOptions;
 import org.apache.impala.analysis.TupleDescriptor;
 import org.apache.impala.analysis.TupleId;
 import org.apache.impala.common.ImpalaException;
@@ -622,21 +621,6 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     return output.toString();
   }
 
-  protected String getExplainString(
-      List<? extends Expr> exprs, TExplainLevel detailLevel) {
-    if (exprs == null) return "";
-    ToSqlOptions toSqlOptions =
-        detailLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal() ?
-        ToSqlOptions.SHOW_IMPLICIT_CASTS :
-        ToSqlOptions.DEFAULT;
-    StringBuilder output = new StringBuilder();
-    for (int i = 0; i < exprs.size(); ++i) {
-      if (i > 0) output.append(", ");
-      output.append(exprs.get(i).toSql(toSqlOptions));
-    }
-    return output.toString();
-  }
-
   /**
    * Returns true if stats-related variables are valid.
    */
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 f3f44b4..6d68bff 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
@@ -17,12 +17,17 @@
 
 package org.apache.impala.planner;
 
+import java.util.List;
+
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.thrift.TDataSink;
 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 com.google.common.base.Preconditions;
+
 
 /**
  * Sink for the root of a query plan that produces result rows. Allows coordination
@@ -36,10 +41,22 @@ public class PlanRootSink extends DataSink {
   // IMPALA-4268 for details on how this value was chosen.
   private static final long DEFAULT_RESULT_SPOOLING_ESTIMATED_MEMORY = 10 * 1024 * 1024;
 
+  // One expression per result column for the query.
+  private final List<Expr> outputExprs_;
+
+  public PlanRootSink(List<Expr> outputExprs) {
+    Preconditions.checkState(outputExprs != null);
+    outputExprs_ = outputExprs;
+  }
+
   @Override
   public void appendSinkExplainString(String prefix, String detailPrefix,
       TQueryOptions queryOptions, TExplainLevel explainLevel, StringBuilder output) {
     output.append(String.format("%sPLAN-ROOT SINK\n", prefix));
+    if (explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+      output.append(detailPrefix + "output exprs: ")
+          .append(Expr.getExplainString(outputExprs_, explainLevel) + "\n");
+    }
   }
 
   @Override
@@ -101,10 +118,16 @@ public class PlanRootSink extends DataSink {
   protected void toThriftImpl(TDataSink tsink) {
     TPlanRootSink tPlanRootSink = new TPlanRootSink(resourceProfile_.toThrift());
     tsink.setPlan_root_sink(tPlanRootSink);
+    tsink.output_exprs = Expr.treesToThrift(outputExprs_);
   }
 
   @Override
   protected TDataSinkType getSinkType() {
     return TDataSinkType.PLAN_ROOT_SINK;
   }
+
+  @Override
+  public void collectExprs(List<Expr> exprs) {
+    exprs.addAll(outputExprs_);
+  }
 }
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 d6989cd..f265ae9 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -144,7 +144,6 @@ public class Planner {
 
     rootFragment.verifyTree();
     ExprSubstitutionMap rootNodeSmap = rootFragment.getPlanRoot().getOutputSmap();
-    List<Expr> resultExprs = null;
     if (ctx_.isInsertOrCtas()) {
       InsertStmt insertStmt = ctx_.getAnalysisResult().getInsertStmt();
       insertStmt.substituteResultExprs(rootNodeSmap, ctx_.getRootAnalyzer());
@@ -157,22 +156,23 @@ public class Planner {
       createPreInsertSort(insertStmt, rootFragment, ctx_.getRootAnalyzer());
       // set up table sink for root fragment
       rootFragment.setSink(insertStmt.createDataSink());
-      resultExprs = insertStmt.getResultExprs();
     } else {
+      QueryStmt queryStmt = ctx_.getQueryStmt();
+      queryStmt.substituteResultExprs(rootNodeSmap, ctx_.getRootAnalyzer());
+      List<Expr> resultExprs = queryStmt.getResultExprs();
       if (ctx_.isUpdate()) {
         // Set up update sink for root fragment
-        rootFragment.setSink(ctx_.getAnalysisResult().getUpdateStmt().createDataSink());
+        rootFragment.setSink(
+            ctx_.getAnalysisResult().getUpdateStmt().createDataSink(resultExprs));
       } else if (ctx_.isDelete()) {
         // Set up delete sink for root fragment
-        rootFragment.setSink(ctx_.getAnalysisResult().getDeleteStmt().createDataSink());
+        rootFragment.setSink(
+            ctx_.getAnalysisResult().getDeleteStmt().createDataSink(resultExprs));
       } else if (ctx_.isQuery()) {
-        rootFragment.setSink(ctx_.getAnalysisResult().getQueryStmt().createDataSink());
+        rootFragment.setSink(
+            ctx_.getAnalysisResult().getQueryStmt().createDataSink(resultExprs));
       }
-      QueryStmt queryStmt = ctx_.getQueryStmt();
-      queryStmt.substituteResultExprs(rootNodeSmap, ctx_.getRootAnalyzer());
-      resultExprs = queryStmt.getResultExprs();
     }
-    rootFragment.setOutputExprs(resultExprs);
 
     // The check for disabling codegen uses estimates of rows per node so must be done
     // on the distributed plan.
@@ -180,7 +180,8 @@ public class Planner {
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("desctbl: " + ctx_.getRootAnalyzer().getDescTbl().debugString());
-      LOG.trace("resultexprs: " + Expr.debugString(rootFragment.getOutputExprs()));
+      LOG.trace("root sink: " + rootFragment.getSink().getExplainString(
+            "", "", ctx_.getQueryOptions(), TExplainLevel.VERBOSE));
       LOG.trace("finalize plan fragments");
     }
     for (PlanFragment fragment: fragments) {
@@ -197,7 +198,6 @@ public class Planner {
       // Compute the column lineage graph
       if (ctx_.isInsertOrCtas()) {
         InsertStmt insertStmt = ctx_.getAnalysisResult().getInsertStmt();
-        List<Expr> exprs = new ArrayList<>();
         FeTable targetTable = insertStmt.getTargetTable();
         Preconditions.checkNotNull(targetTable);
         if (targetTable instanceof FeKuduTable) {
@@ -215,23 +215,19 @@ public class Planner {
           } else {
             graph.addTargetColumnLabels(targetTable);
           }
-          exprs.addAll(resultExprs);
         } else if (targetTable instanceof FeHBaseTable) {
           graph.addTargetColumnLabels(targetTable);
-          exprs.addAll(resultExprs);
         } else {
           graph.addTargetColumnLabels(targetTable);
-          exprs.addAll(ctx_.getAnalysisResult().getInsertStmt().getPartitionKeyExprs());
-          exprs.addAll(resultExprs.subList(0,
-              targetTable.getNonClusteringColumns().size()));
         }
-        graph.computeLineageGraph(exprs, ctx_.getRootAnalyzer());
       } else {
         graph.addTargetColumnLabels(ctx_.getQueryStmt().getColLabels().stream()
             .map(col -> new ColumnLabel(col))
             .collect(Collectors.toList()));
-        graph.computeLineageGraph(resultExprs, ctx_.getRootAnalyzer());
       }
+      List<Expr> outputExprs = new ArrayList<>();
+      rootFragment.getSink().collectExprs(outputExprs);
+      graph.computeLineageGraph(outputExprs, ctx_.getRootAnalyzer());
       if (LOG.isTraceEnabled()) LOG.trace("lineage: " + graph.debugString());
       ctx_.getTimeline().markEvent("Lineage info computed");
     }
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 ee17695..c657d28 100644
--- a/fe/src/main/java/org/apache/impala/planner/SelectNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SelectNode.java
@@ -101,7 +101,7 @@ public class SelectNode extends PlanNode {
     if (detailLevel.ordinal() >= TExplainLevel.STANDARD.ordinal()) {
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix
-            + "predicates: " + getExplainString(conjuncts_, detailLevel) + "\n");
+            + "predicates: " + Expr.getExplainString(conjuncts_, detailLevel) + "\n");
       }
     }
     return output.toString();
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 cbd8ca3..f964c6d 100644
--- a/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
@@ -20,6 +20,7 @@ package org.apache.impala.planner;
 import java.util.ArrayList;
 
 import org.apache.impala.analysis.Analyzer;
+import org.apache.impala.analysis.Expr;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.thrift.TExecNodePhase;
 import org.apache.impala.thrift.TExplainLevel;
@@ -139,7 +140,7 @@ public class SubplanNode extends PlanNode {
     if (detailLevel.ordinal() >= TExplainLevel.STANDARD.ordinal()) {
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix
-            + "predicates: " + getExplainString(conjuncts_, detailLevel) + "\n");
+            + "predicates: " + Expr.getExplainString(conjuncts_, detailLevel) + "\n");
       }
     }
     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 c85cd74..f37a233 100644
--- a/fe/src/main/java/org/apache/impala/planner/TableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/TableSink.java
@@ -76,10 +76,14 @@ public abstract class TableSink extends DataSink {
   protected final FeTable targetTable_;
   // The type of operation to be performed by this sink.
   protected final Op sinkOp_;
+  // One expression per result column for the query. Always non-null.
+  protected final List<Expr> outputExprs_;
 
-  public TableSink(FeTable targetTable, Op sinkAction) {
+  public TableSink(FeTable targetTable, Op sinkAction, List<Expr> outputExprs) {
+    Preconditions.checkState(outputExprs != null);
     targetTable_ = targetTable;
     sinkOp_ = sinkAction;
+    outputExprs_ = outputExprs;
   }
 
   /**
@@ -91,17 +95,19 @@ public abstract class TableSink extends DataSink {
    * columns of the target table that are stored in the 'sort.columns' table property.
    */
   public static TableSink create(FeTable table, Op sinkAction,
-      List<Expr> partitionKeyExprs,  List<Integer> referencedColumns,
+      List<Expr> partitionKeyExprs, List<Expr> outputExprs,
+      List<Integer> referencedColumns,
       boolean overwrite, boolean inputIsClustered, List<Integer> sortColumns) {
-    return create(table, sinkAction, partitionKeyExprs, referencedColumns, overwrite,
-        inputIsClustered, sortColumns, -1);
+    return create(table, sinkAction, partitionKeyExprs, outputExprs, referencedColumns,
+        overwrite, inputIsClustered, sortColumns, -1);
   }
 
   /**
    * Same as above, plus it takes an ACID write id in parameter 'writeId'.
    */
   public static TableSink create(FeTable table, Op sinkAction,
-      List<Expr> partitionKeyExprs,  List<Integer> referencedColumns,
+      List<Expr> partitionKeyExprs, List<Expr> outputExprs,
+      List<Integer> referencedColumns,
       boolean overwrite, boolean inputIsClustered, List<Integer> sortColumns,
       long writeId) {
     Preconditions.checkNotNull(partitionKeyExprs);
@@ -112,8 +118,8 @@ public abstract class TableSink extends DataSink {
       Preconditions.checkState(sinkAction == Op.INSERT);
       // Referenced columns don't make sense for an Hdfs table.
       Preconditions.checkState(referencedColumns.isEmpty());
-      return new HdfsTableSink(table, partitionKeyExprs, overwrite, inputIsClustered,
-          sortColumns, writeId);
+      return new HdfsTableSink(table, partitionKeyExprs,outputExprs, overwrite,
+          inputIsClustered, sortColumns, writeId);
     } else if (table instanceof FeHBaseTable) {
       // HBase only supports inserts.
       Preconditions.checkState(sinkAction == Op.INSERT);
@@ -126,13 +132,13 @@ public abstract class TableSink extends DataSink {
       // Sort columns are not supported for HBase tables.
       Preconditions.checkState(sortColumns.isEmpty());
       // Create the HBaseTableSink and return it.
-      return new HBaseTableSink(table);
+      return new HBaseTableSink(table, outputExprs);
     } else if (table instanceof FeKuduTable) {
       // Kudu doesn't have a way to perform INSERT OVERWRITE.
       Preconditions.checkState(overwrite == false);
       // Sort columns are not supported for Kudu tables.
       Preconditions.checkState(sortColumns.isEmpty());
-      return new KuduTableSink(table, sinkAction, referencedColumns);
+      return new KuduTableSink(table, sinkAction, referencedColumns, outputExprs);
     } else {
       throw new UnsupportedOperationException(
           "Cannot create data sink into table of type: " + table.getClass().getName());
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 cfdd076..d9087ee 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnionNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
@@ -319,7 +319,7 @@ public class UnionNode extends PlanNode {
     // and the enclosing select stmt has predicates referring to the inline view.
     if (!conjuncts_.isEmpty()) {
       output.append(detailPrefix
-          + "predicates: " + getExplainString(conjuncts_, detailLevel) + "\n");
+          + "predicates: " + Expr.getExplainString(conjuncts_, detailLevel) + "\n");
     }
     if (!constExprLists_.isEmpty()) {
       output.append(detailPrefix + "constant-operands=" + constExprLists_.size() + "\n");
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 13acc51..4866961 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
@@ -92,7 +92,7 @@ public class UnnestNode extends PlanNode {
     if (detailLevel.ordinal() >= TExplainLevel.STANDARD.ordinal()) {
       if (!conjuncts_.isEmpty()) {
         output.append(detailPrefix
-            + "predicates: " + getExplainString(conjuncts_, detailLevel) + "\n");
+            + "predicates: " + Expr.getExplainString(conjuncts_, detailLevel) + "\n");
       }
     }
     return output.toString();
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
index 20ca2f7..99990c9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
@@ -11,6 +11,7 @@ CAST(0 AS BIGINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=264.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -58,14 +59,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: c_custkey > CAST(10 AS BIGINT), !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems), o_orderkey = CAST(4 AS BIGINT)
    predicates on o_lineitems: CAST(20 AS BIGINT) + CAST(l_linenumber AS BIGINT) < CAST(0 AS BIGINT)
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=44.23K
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    parquet statistics predicates: c_custkey > CAST(10 AS BIGINT)
    parquet statistics predicates on o: o_orderkey = CAST(4 AS BIGINT)
    parquet dictionary predicates: c_custkey > CAST(10 AS BIGINT)
@@ -85,6 +86,7 @@ AND tinyint_col = CAST(5 AS TINYINT) AND id >= '10' AND id <= '20'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_hbase.stringids.id, functional_hbase.stringids.bigint_col, functional_hbase.stringids.bool_col, functional_hbase.stringids.date_string_col, functional_hbase.stringids.day, functional_hbase.stringids.double_col, functional_hbase.stringids.float_col, functional_hbase.stringids.int_col, functional_hbase.stringids.month, functional_hbase.stringids.smallint_col, functional_hbase.stringids.string_col, functional_hbase.stringids.timestamp_col, functional_hbase.string [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -110,6 +112,7 @@ FLOAT) AND int_col < CAST(2 AS INT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes_datasource.id, functional.alltypes_datasource.bool_col, functional.alltypes_datasource.tinyint_col, functional.alltypes_datasource.smallint_col, functional.alltypes_datasource.int_col, functional.alltypes_datasource.bigint_col, functional.alltypes_datasource.float_col, functional.alltypes_datasource.double_col, functional.alltypes_datasource.timestamp_col, functional.alltypes_datasource.string_col, functional.alltypes_datasource.dec_col1, functional.a [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
@@ -136,6 +139,7 @@ AS BIGINT) + CAST(id AS BIGINT)) <= CAST(10 AS BIGINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=1.97MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: sum(2 + id)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -147,7 +151,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -173,6 +177,7 @@ DECIMAL(6,1))
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=257.94MB mem-reservation=2.00MB thread-reservation=3
 PLAN-ROOT SINK
+|  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [LEFT OUTER JOIN]
@@ -185,7 +190,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional.alltypes b]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     predicates: CAST(b.double_col AS DECIMAL(3,2)) > CAST(11.1 AS DECIMAL(6,1))
 |     stored statistics:
 |       table: rows=7.30K size=478.45KB
@@ -197,7 +202,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional.alltypes a]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -224,6 +229,7 @@ DECIMAL(6,1))
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=256.01MB mem-reservation=64.00KB thread-reservation=3
 PLAN-ROOT SINK
+|  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:NESTED LOOP JOIN [LEFT OUTER JOIN]
@@ -234,7 +240,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional.alltypes b]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     predicates: CAST(b.double_col AS DECIMAL(3,2)) > CAST(11.1 AS DECIMAL(6,1))
 |     stored statistics:
 |       table: rows=7.30K size=478.45KB
@@ -246,7 +252,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional.alltypes a]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -269,6 +275,7 @@ BIGINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=3.88MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: sum(2 + id)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:AGGREGATE [FINALIZE]
@@ -287,7 +294,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -308,6 +315,7 @@ BIGINT) = CAST(0 AS BIGINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=1.97MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: sum(2 + id)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:AGGREGATE [FINALIZE]
@@ -325,7 +333,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -347,6 +355,7 @@ bigint_col) ASC) FROM functional.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=144.00MB mem-reservation=16.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: first_value(2 + int_col - 0)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:ANALYTIC
@@ -366,7 +375,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -385,6 +394,7 @@ AS DOUBLE) ASC
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=134.00MB mem-reservation=6.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: int_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -394,7 +404,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -416,6 +426,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=38.00MB mem-reservation=6.01MB thread-reservation=2
 WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(CAST(3 + year AS INT),CAST(month - -1 AS INT))]
 |  partitions=4
+|  output exprs: id, NULL, NULL, NULL, int_col, NULL, NULL, NULL, NULL, NULL, NULL, CAST(CAST(3 AS BIGINT) + CAST(year AS BIGINT) AS INT), CAST(CAST(month AS BIGINT) - CAST(-1 AS BIGINT) AS INT)
 |  mem-estimate=1.56KB mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -425,7 +436,7 @@ WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(CAST(3 + ye
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypessmall]
-   partitions=4/4 files=4 size=6.32KB
+   HDFS partitions=4/4 files=4 size=6.32KB
    stored statistics:
      table: rows=100 size=6.32KB
      partitions: 4/4 rows=100
@@ -450,6 +461,7 @@ functional.alltypes LIMIT CAST(2 AS TINYINT)) v1) v2) v3
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: sum(id + c3)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -459,7 +471,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
index b8263e1..e144a2c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection-hdfs-num-rows-est-enabled.test
@@ -12,6 +12,7 @@ on ss_customer_sk = c_customer_sk
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=178.94MB mem-reservation=18.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -23,7 +24,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds_seq_snap.customer]
-|     partitions=1/1 files=1 size=8.58MB
+|     HDFS partitions=1/1 files=1 size=8.58MB
 |     stored statistics:
 |       table: rows=unavailable size=8.58MB
 |       columns: unavailable
@@ -33,7 +34,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000[bloom] -> ss_customer_sk
    stored statistics:
      table: rows=2.88M size=346.60MB
@@ -52,6 +53,7 @@ on ss_customer_sk = c_customer_sk
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=178.94MB mem-reservation=18.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -63,7 +65,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.customer]
-|     partitions=1/1 files=1 size=12.60MB
+|     HDFS partitions=1/1 files=1 size=12.60MB
 |     stored statistics:
 |       table: rows=100.00K size=12.60MB
 |       columns: all
@@ -73,7 +75,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds_seq_snap.store_sales]
-   partitions=1824/1824 files=1824 size=212.80MB
+   HDFS partitions=1824/1824 files=1824 size=212.80MB
    runtime filters: RF000[bloom] -> ss_customer_sk
    stored statistics:
      table: rows=unavailable size=unavailable
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
index bfe5e2e..86c0d32 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
@@ -7,6 +7,7 @@ where c_salutation = 'Mrs.'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=181.75MB mem-reservation=21.75MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -18,7 +19,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.customer]
-|     partitions=1/1 files=1 size=12.60MB
+|     HDFS partitions=1/1 files=1 size=12.60MB
 |     predicates: c_salutation = 'Mrs.'
 |     stored statistics:
 |       table: rows=100.00K size=12.60MB
@@ -29,7 +30,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000[bloom] -> ss_customer_sk
    stored statistics:
      table: rows=2.88M size=346.60MB
@@ -50,6 +51,7 @@ where c_salutation = 'Mrs.'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=180.75MB mem-reservation=20.75MB thread-reservation=3
 PLAN-ROOT SINK
+|  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [LEFT OUTER JOIN]
@@ -61,7 +63,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.customer]
-|     partitions=1/1 files=1 size=12.60MB
+|     HDFS partitions=1/1 files=1 size=12.60MB
 |     predicates: c_salutation = 'Mrs.'
 |     stored statistics:
 |       table: rows=100.00K size=12.60MB
@@ -72,7 +74,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    stored statistics:
      table: rows=2.88M size=346.60MB
      partitions: 1824/1824 rows=2.88M
@@ -92,6 +94,7 @@ where c_salutation = 'Mrs.'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=181.75MB mem-reservation=21.75MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [RIGHT OUTER JOIN]
@@ -103,7 +106,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.customer]
-|     partitions=1/1 files=1 size=12.60MB
+|     HDFS partitions=1/1 files=1 size=12.60MB
 |     predicates: c_salutation = 'Mrs.'
 |     stored statistics:
 |       table: rows=100.00K size=12.60MB
@@ -114,7 +117,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000[bloom] -> ss_customer_sk
    stored statistics:
      table: rows=2.88M size=346.60MB
@@ -134,6 +137,7 @@ where sr_return_quantity < 10
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=214.75MB mem-reservation=22.75MB thread-reservation=3 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
+|  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -145,7 +149,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.store_returns]
-|     partitions=1/1 files=1 size=31.19MB
+|     HDFS partitions=1/1 files=1 size=31.19MB
 |     predicates: sr_return_quantity < CAST(10 AS INT)
 |     stored statistics:
 |       table: rows=287.51K size=31.19MB
@@ -156,7 +160,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000[bloom] -> ss_item_sk, RF001[bloom] -> ss_ticket_number
    stored statistics:
      table: rows=2.88M size=346.60MB
@@ -175,6 +179,7 @@ on ss_sold_time_sk = ws_sold_time_sk
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=397.67MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -186,7 +191,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.web_sales]
-|     partitions=1/1 files=1 size=140.07MB
+|     HDFS partitions=1/1 files=1 size=140.07MB
 |     stored statistics:
 |       table: rows=719.38K size=140.07MB
 |       columns: all
@@ -196,7 +201,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000[bloom] -> ss_sold_time_sk
    stored statistics:
      table: rows=2.88M size=346.60MB
@@ -216,6 +221,7 @@ where a.d_holiday = "Y"
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=114.00MB mem-reservation=34.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: a.d_date_sk, a.d_date_id, a.d_date, a.d_month_seq, a.d_week_seq, a.d_quarter_seq, a.d_year, a.d_dow, a.d_moy, a.d_dom, a.d_qoy, a.d_fy_year, a.d_fy_quarter_seq, a.d_fy_week_seq, a.d_day_name, a.d_quarter_name, a.d_holiday, a.d_weekend, a.d_following_holiday, a.d_first_dom, a.d_last_dom, a.d_same_day_ly, a.d_same_day_lq, a.d_current_day, a.d_current_week, a.d_current_month, a.d_current_quarter, a.d_current_year, b.d_date_sk, b.d_date_id, b.d_date, b.d_month_seq, b.d_week_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -227,7 +233,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--00:SCAN HDFS [tpcds.date_dim a]
-|     partitions=1/1 files=1 size=9.84MB
+|     HDFS partitions=1/1 files=1 size=9.84MB
 |     predicates: a.d_holiday = 'Y'
 |     stored statistics:
 |       table: rows=73.05K size=9.84MB
@@ -238,7 +244,7 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 01:SCAN HDFS [tpcds.date_dim b]
-   partitions=1/1 files=1 size=9.84MB
+   HDFS partitions=1/1 files=1 size=9.84MB
    runtime filters: RF000[bloom] -> b.d_date_sk
    stored statistics:
      table: rows=73.05K size=9.84MB
@@ -261,6 +267,7 @@ where ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=362.81MB mem-reservation=50.81MB thread-reservation=6 runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
+|  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 08:HASH JOIN [INNER JOIN]
@@ -272,7 +279,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN HDFS [tpcds.customer]
-|     partitions=1/1 files=1 size=12.60MB
+|     HDFS partitions=1/1 files=1 size=12.60MB
 |     stored statistics:
 |       table: rows=100.00K size=12.60MB
 |       columns: all
@@ -290,7 +297,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 04(OPEN)
 |
 |--04:SCAN HDFS [tpcds.date_dim d2]
-|     partitions=1/1 files=1 size=9.84MB
+|     HDFS partitions=1/1 files=1 size=9.84MB
 |     stored statistics:
 |       table: rows=73.05K size=9.84MB
 |       columns: all
@@ -316,7 +323,7 @@ PLAN-ROOT SINK
 |  |  in pipelines: 00(GETNEXT), 03(OPEN)
 |  |
 |  |--03:SCAN HDFS [tpcds.date_dim d1]
-|  |     partitions=1/1 files=1 size=9.84MB
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
 |  |     predicates: d1.d_fy_week_seq = CAST(1000 AS INT)
 |  |     stored statistics:
 |  |       table: rows=73.05K size=9.84MB
@@ -327,7 +334,7 @@ PLAN-ROOT SINK
 |  |     in pipelines: 03(GETNEXT)
 |  |
 |  00:SCAN HDFS [tpcds.store_sales]
-|     partitions=1824/1824 files=1824 size=346.60MB
+|     HDFS partitions=1824/1824 files=1824 size=346.60MB
 |     runtime filters: RF000[bloom] -> ss_addr_sk, RF008[bloom] -> ss_sold_date_sk
 |     stored statistics:
 |       table: rows=2.88M size=346.60MB
@@ -339,7 +346,7 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 01:SCAN HDFS [tpcds.store_returns]
-   partitions=1/1 files=1 size=31.19MB
+   HDFS partitions=1/1 files=1 size=31.19MB
    runtime filters: RF002[bloom] -> sr_returned_date_sk, RF004[bloom] -> sr_item_sk, RF005[bloom] -> sr_ticket_number
    stored statistics:
      table: rows=287.51K size=31.19MB
@@ -357,6 +364,7 @@ on ss_customer_sk % 10 = c_customer_sk / 100
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=211.00MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -368,7 +376,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.customer]
-|     partitions=1/1 files=1 size=12.60MB
+|     HDFS partitions=1/1 files=1 size=12.60MB
 |     stored statistics:
 |       table: rows=100.00K size=12.60MB
 |       columns: all
@@ -378,7 +386,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000[bloom] -> ss_customer_sk % 10
    stored statistics:
      table: rows=2.88M size=346.60MB
@@ -398,6 +406,7 @@ on ss_customer_sk = c_customer_sk
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=2.17GB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -409,7 +418,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds_seq_snap.customer]
-|     partitions=1/1 files=1 size=8.58MB
+|     HDFS partitions=1/1 files=1 size=8.58MB
 |     stored statistics:
 |       table: rows=unavailable size=8.58MB
 |       columns: unavailable
@@ -419,7 +428,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000[bloom] -> ss_customer_sk
    stored statistics:
      table: rows=2.88M size=346.60MB
@@ -438,6 +447,7 @@ on ss_customer_sk = c_customer_sk
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=178.94MB mem-reservation=18.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: CAST(1 AS TINYINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -449,7 +459,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [tpcds.customer]
-|     partitions=1/1 files=1 size=12.60MB
+|     HDFS partitions=1/1 files=1 size=12.60MB
 |     stored statistics:
 |       table: rows=100.00K size=12.60MB
 |       columns: all
@@ -459,7 +469,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds_seq_snap.store_sales]
-   partitions=1824/1824 files=1824 size=212.80MB
+   HDFS partitions=1824/1824 files=1824 size=212.80MB
    runtime filters: RF000[bloom] -> ss_customer_sk
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -480,6 +490,7 @@ on ss_sold_time_sk = ws_sold_time_sk
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=299.00MB mem-reservation=18.94MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: tpcds.store_sales.ss_sold_time_sk, tpcds.store_sales.ss_item_sk, tpcds.store_sales.ss_customer_sk, tpcds.store_sales.ss_cdemo_sk, tpcds.store_sales.ss_hdemo_sk, tpcds.store_sales.ss_addr_sk, tpcds.store_sales.ss_store_sk, tpcds.store_sales.ss_promo_sk, tpcds.store_sales.ss_ticket_number, tpcds.store_sales.ss_quantity, tpcds.store_sales.ss_wholesale_cost, tpcds.store_sales.ss_list_price, tpcds.store_sales.ss_sales_price, tpcds.store_sales.ss_ext_discount_amt, tpcds.store_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:HASH JOIN [INNER JOIN]
@@ -497,7 +508,7 @@ PLAN-ROOT SINK
 |  |  in pipelines: 02(GETNEXT), 01(OPEN)
 |  |
 |  01:SCAN HDFS [tpcds.web_sales]
-|     partitions=1/1 files=1 size=140.07MB
+|     HDFS partitions=1/1 files=1 size=140.07MB
 |     stored statistics:
 |       table: rows=719.38K size=140.07MB
 |       columns: all
@@ -507,7 +518,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds.store_sales]
-   partitions=1824/1824 files=1824 size=346.60MB
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000[bloom] -> ss_sold_time_sk
    stored statistics:
      table: rows=2.88M size=346.60MB
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
index d39ba76..cd08942 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
@@ -3,6 +3,7 @@ select * from functional_kudu.zipcode_incomes where id = '8600000US00601'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=1.88MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
@@ -14,6 +15,7 @@ Per-Host Resources: mem-estimate=1.88MB mem-reservation=0B thread-reservation=2
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   01:EXCHANGE [UNPARTITIONED]
@@ -37,6 +39,7 @@ select * from functional_kudu.zipcode_incomes where id != '1' and zip = '2'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
@@ -49,6 +52,7 @@ Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   01:EXCHANGE [UNPARTITIONED]
@@ -72,6 +76,7 @@ select * from functional_kudu.zipcode_incomes where id > '1' and zip > '2'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
@@ -83,6 +88,7 @@ Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=517.89KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   01:EXCHANGE [UNPARTITIONED]
@@ -105,6 +111,7 @@ select * from functional_kudu.zipcode_incomes where id = '1' or id = '2' or zip
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
@@ -116,6 +123,7 @@ Per-Host Resources: mem-estimate=3.75MB mem-reservation=0B thread-reservation=2
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.zipcode_incomes.id, functional_kudu.zipcode_incomes.zip, functional_kudu.zipcode_incomes.description1, functional_kudu.zipcode_incomes.description2, functional_kudu.zipcode_incomes.income
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   01:EXCHANGE [UNPARTITIONED]
@@ -161,6 +169,7 @@ id in (int_col)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.alltypes]
@@ -176,6 +185,7 @@ select * from functional_kudu.decimal_tbl where d1 in (1234, 12345);
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=4.50MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.decimal_tbl.d1, functional_kudu.decimal_tbl.d2, functional_kudu.decimal_tbl.d3, functional_kudu.decimal_tbl.d4, functional_kudu.decimal_tbl.d5, functional_kudu.decimal_tbl.d6
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.decimal_tbl]
@@ -194,6 +204,7 @@ timestamp_col < (seconds_add(cast('9999-12-31 24:59:59' as timestamp), 10))
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.alltypes]
@@ -210,6 +221,7 @@ timestamp_col in (cast('2010-03-01 00:00:00' as timestamp),
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.alltypes]
@@ -226,6 +238,7 @@ timestamp_col in (cast('2010-03-01 00:00:00' as timestamp),
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 Per-Host Resources: mem-estimate=9.75MB mem-reservation=0B thread-reservation=2
   PLAN-ROOT SINK
+  |  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
   |  mem-estimate=0B mem-reservation=0B thread-reservation=0
   |
   00:SCAN KUDU [functional_kudu.alltypes]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
index 867268b..f36b5e2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
@@ -12,6 +12,7 @@ JOIN tpch_parquet.nation ON c_nationkey = n_nationkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.33MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.customer.c_acctbal, tpch_parquet.customer.c_mktsegment, tpch_parquet.customer.c_comment, tpch_parquet.nation.n_nationkey, tpch_parquet.nation.n_name, tpch_parquet.nation.n_regionkey, tpch_parquet.nation.n_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -37,9 +38,9 @@ Per-Host Resources: mem-estimate=41.95MB mem-reservation=33.94MB thread-reservat
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.nation, RANDOM]
-|     partitions=1/1 files=1 size=2.75KB
+|     HDFS partitions=1/1 files=1 size=3.04KB
 |     stored statistics:
-|       table: rows=25 size=2.75KB
+|       table: rows=25 size=3.04KB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=25
 |     mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1
@@ -47,10 +48,10 @@ Per-Host Resources: mem-estimate=41.95MB mem-reservation=33.94MB thread-reservat
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.customer, RANDOM]
-   partitions=1/1 files=1 size=12.31MB
+   HDFS partitions=1/1 files=1 size=12.34MB
    runtime filters: RF000[bloom] -> c_nationkey
    stored statistics:
-     table: rows=150.00K size=12.31MB
+     table: rows=150.00K size=12.34MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=150.00K
    mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1
@@ -71,6 +72,7 @@ OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -95,9 +97,9 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=86.00MB thread-reserva
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
-|     partitions=1/1 files=2 size=54.07MB
+|     HDFS partitions=1/1 files=2 size=54.21MB
 |     stored statistics:
-|       table: rows=1.50M size=54.07MB
+|       table: rows=1.50M size=54.21MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.18M
 |     mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
@@ -105,9 +107,9 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=86.00MB thread-reserva
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -128,6 +130,7 @@ Analyzed query: SELECT * FROM tpch_parquet.lineitem NULL AWARE LEFT ANTI JOIN
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -151,9 +154,9 @@ Per-Host Resources: mem-estimate=124.02MB mem-reservation=74.00MB thread-reserva
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
-|     partitions=1/1 files=2 size=54.07MB
+|     HDFS partitions=1/1 files=2 size=54.21MB
 |     stored statistics:
-|       table: rows=1.50M size=54.07MB
+|       table: rows=1.50M size=54.21MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.18M
 |     mem-estimate=40.00MB mem-reservation=4.00MB thread-reservation=1
@@ -161,9 +164,9 @@ Per-Host Resources: mem-estimate=124.02MB mem-reservation=74.00MB thread-reserva
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -188,6 +191,7 @@ BIGINT)
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.10MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, o_orderstatus, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 08:EXCHANGE [UNPARTITIONED]
@@ -235,9 +239,9 @@ Per-Host Resources: mem-estimate=99.65MB mem-reservation=66.00MB thread-reservat
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
-|     partitions=1/1 files=2 size=54.07MB
+|     HDFS partitions=1/1 files=2 size=54.21MB
 |     stored statistics:
-|       table: rows=1.50M size=54.07MB
+|       table: rows=1.50M size=54.21MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.18M
 |     mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=1
@@ -252,10 +256,10 @@ Per-Host Resources: mem-estimate=99.65MB mem-reservation=66.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
@@ -274,6 +278,7 @@ Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -303,9 +308,9 @@ Per-Host Resources: mem-estimate=806.43MB mem-reservation=74.00MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -326,6 +331,7 @@ FROM tpch_parquet.lineitem GROUP BY l_orderkey, l_partkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.09MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, group_concat(l_linestatus, ',')
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -357,9 +363,9 @@ Per-Host Resources: mem-estimate=168.14MB mem-reservation=50.00MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=16.00MB thread-reservation=1
@@ -380,6 +386,7 @@ functional.alltypes
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=56.26KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: max(tinyint_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -410,7 +417,7 @@ Per-Host Resources: mem-estimate=40.04MB mem-reservation=40.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2
 00:SCAN HDFS [functional.alltypes, RANDOM]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
index 6a1d531..9210208 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters-hdfs-num-rows-est-enabled.test
@@ -10,6 +10,7 @@ where a.int_col = b.int_col and a.int_col = c.int_col
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=31.88MB mem-reservation=5.89MB thread-reservation=4 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
@@ -23,7 +24,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: none
 |  runtime filters: RF000[bloom] <- c.int_col, RF001[min_max] <- c.int_col
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,0,2 row-size=12B cardinality=9.31M
+|  tuple-ids=1,0,2 row-size=12B cardinality=9.30M
 |  in pipelines: 01(GETNEXT), 02(OPEN)
 |
 |--02:SCAN KUDU [functional_kudu.alltypes c]
@@ -36,7 +37,7 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: assumed fk/pk
 |  runtime filters: RF002[bloom] <- a.int_col
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=1,0 row-size=8B cardinality=12.75K
+|  tuple-ids=1,0 row-size=8B cardinality=12.74K
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 |--00:SCAN KUDU [functional_kudu.alltypes a]
@@ -46,7 +47,7 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 01:SCAN HDFS [functional_parquet.alltypes b]
-   HDFS partitions=24/24 files=24 size=200.43KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    runtime filters: RF000[bloom] -> b.int_col, RF002[bloom] -> b.int_col
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -54,6 +55,6 @@ PLAN-ROOT SINK
      columns: unavailable
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1
-   tuple-ids=1 row-size=4B cardinality=12.75K
+   tuple-ids=1 row-size=4B cardinality=12.74K
    in pipelines: 01(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
index 0a63ef6..12f4b97 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
@@ -5,6 +5,7 @@ where a.int_col = b.tinyint_col + 1 and a.string_col = b.string_col
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=13.44MB mem-reservation=1.94MB thread-reservation=3
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -42,6 +43,7 @@ where a.int_col + 1 = b.int_col
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=14.19MB mem-reservation=1.94MB thread-reservation=3
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -78,6 +80,7 @@ where a.tinyint_col = b.bigint_col
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=14.94MB mem-reservation=1.94MB thread-reservation=3
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -113,6 +116,7 @@ where a.int_col = b.int_col and a.int_col = c.int_col
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=2.02GB mem-reservation=36.95MB thread-reservation=4 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
@@ -143,7 +147,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional_parquet.alltypes b]
-|     partitions=24/24 files=24 size=189.28KB
+|     HDFS partitions=24/24 files=24 size=200.33KB
 |     runtime filters: RF000[bloom] -> b.int_col
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation-hdfs-num-rows-est-enabled.test b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation-hdfs-num-rows-est-enabled.test
index cc56083..4a3361e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation-hdfs-num-rows-est-enabled.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation-hdfs-num-rows-est-enabled.test
@@ -11,6 +11,7 @@ functional_parquet.alltypestiny b
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=42.00MB mem-reservation=16.00KB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -57,6 +58,7 @@ where a.id = b.id
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=28.94MB mem-reservation=2.95MB thread-reservation=1 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -108,6 +110,7 @@ limit 10
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.02MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(int_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:TOP-N [LIMIT=10]
@@ -124,7 +127,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   HDFS partitions=24/24 files=24 size=200.43KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -140,6 +143,7 @@ PLAN-ROOT SINK
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(int_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:MERGING-EXCHANGE [UNPARTITIONED]
@@ -165,7 +169,7 @@ Per-Host Resources: mem-estimate=384.55MB mem-reservation=102.00MB thread-reserv
 |  in pipelines: 04(GETNEXT), 00(OPEN)
 |
 03:EXCHANGE [HASH(bigint_col)]
-|  mem-estimate=186.64KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=186.63KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=1 row-size=16B cardinality=1.27K
 |  in pipelines: 00(GETNEXT)
 |
@@ -179,7 +183,7 @@ Per-Host Resources: mem-estimate=432.00MB mem-reservation=102.07MB thread-reserv
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   HDFS partitions=24/24 files=24 size=200.43KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -200,6 +204,7 @@ where id < 10
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=26.00MB mem-reservation=10.02MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:ANALYTIC
@@ -218,7 +223,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   HDFS partitions=24/24 files=24 size=200.43KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -232,12 +237,13 @@ PLAN-ROOT SINK
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=222.64KB mem-reservation=0B thread-reservation=1
+|  Per-Host Resources: mem-estimate=222.63KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
-|  mem-estimate=222.64KB mem-reservation=0B thread-reservation=0
+|  mem-estimate=222.63KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=4,3 row-size=16B cardinality=1.27K
 |  in pipelines: 01(GETNEXT)
 |
@@ -266,7 +272,7 @@ Per-Host Resources: mem-estimate=30.33MB mem-reservation=30.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
 Per-Host Resources: mem-estimate=48.00MB mem-reservation=48.00KB thread-reservation=3
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   HDFS partitions=24/24 files=24 size=200.43KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
index 9b7098d..a8abffe 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
@@ -6,6 +6,7 @@ functional_parquet.alltypestiny b
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=2.04GB mem-reservation=16.00KB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -20,7 +21,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional_parquet.alltypestiny b]
-|     partitions=4/4 files=4 size=9.92KB
+|     HDFS partitions=4/4 files=4 size=11.67KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
@@ -31,7 +32,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny a]
-   partitions=4/4 files=4 size=9.92KB
+   HDFS partitions=4/4 files=4 size=11.67KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/4 rows=unavailable
@@ -52,6 +53,7 @@ where a.id = b.id
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=2.03GB mem-reservation=35.01MB thread-reservation=1 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -69,7 +71,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional_parquet.alltypestiny b]
-|     partitions=4/4 files=4 size=9.92KB
+|     HDFS partitions=4/4 files=4 size=11.67KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
@@ -80,7 +82,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny a]
-   partitions=4/4 files=4 size=9.92KB
+   HDFS partitions=4/4 files=4 size=11.67KB
    runtime filters: RF000[bloom] -> a.id
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -118,6 +120,7 @@ limit 10
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.02MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(int_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:TOP-N [LIMIT=10]
@@ -134,7 +137,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=190.12KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -150,6 +153,7 @@ PLAN-ROOT SINK
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(int_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:MERGING-EXCHANGE [UNPARTITIONED]
@@ -189,7 +193,7 @@ Per-Host Resources: mem-estimate=432.00MB mem-reservation=102.07MB thread-reserv
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   partitions=24/24 files=24 size=190.12KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -210,6 +214,7 @@ where id < 10
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=26.00MB mem-reservation=10.02MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:ANALYTIC
@@ -228,7 +233,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=190.12KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -244,6 +249,7 @@ PLAN-ROOT SINK
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=216.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -276,7 +282,7 @@ Per-Host Resources: mem-estimate=30.32MB mem-reservation=30.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
 Per-Host Resources: mem-estimate=48.00MB mem-reservation=48.00KB thread-reservation=3
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   partitions=24/24 files=24 size=190.12KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -297,6 +303,7 @@ where c_custkey < 10 and o_orderkey < 5 and l_linenumber < 3
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=104.00MB mem-reservation=104.00MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, o.item.o_orderkey, o.item.o_orderstatus, o.item.o_totalprice, o.item.o_orderdate, o.item.o_orderpriority, o.item.o_clerk, o.item.o_shippriority, o.item.o_comment, o_lineitems.item.l_partkey, o_lineitems.item.l_suppkey, o_lineitems.item.l_linenumber, o_lineitems.item.l_quantity, o_lineitems.item.l_extendedprice, o_lineitems.item.l_discount, o_lineitems.item.l_tax, o_lin [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -344,14 +351,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: c_custkey < CAST(10 AS BIGINT), !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems), o_orderkey < CAST(5 AS BIGINT)
    predicates on o_lineitems: l_linenumber < CAST(3 AS INT)
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=50.11K
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    parquet statistics predicates: c_custkey < CAST(10 AS BIGINT)
    parquet statistics predicates on o: o_orderkey < CAST(5 AS BIGINT)
    parquet statistics predicates on o_lineitems: l_linenumber < CAST(3 AS INT)
@@ -365,6 +372,7 @@ PLAN-ROOT SINK
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=14.34MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, o.item.o_orderkey, o.item.o_orderstatus, o.item.o_totalprice, o.item.o_orderdate, o.item.o_orderpriority, o.item.o_clerk, o.item.o_shippriority, o.item.o_comment, o_lineitems.item.l_partkey, o_lineitems.item.l_suppkey, o_lineitems.item.l_linenumber, o_lineitems.item.l_quantity, o_lineitems.item.l_extendedprice, o_lineitems.item.l_discount, o_lineitems.item.l_tax, o_lin [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 09:EXCHANGE [UNPARTITIONED]
@@ -419,14 +427,14 @@ Per-Host Resources: mem-estimate=312.00MB mem-reservation=312.00MB thread-reserv
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: c_custkey < CAST(10 AS BIGINT), !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems), o_orderkey < CAST(5 AS BIGINT)
    predicates on o_lineitems: l_linenumber < CAST(3 AS INT)
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=50.11K
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    parquet statistics predicates: c_custkey < CAST(10 AS BIGINT)
    parquet statistics predicates on o: o_orderkey < CAST(5 AS BIGINT)
    parquet statistics predicates on o_lineitems: l_linenumber < CAST(3 AS INT)
@@ -445,6 +453,7 @@ where o1.o_orderkey = o2.o_orderkey + 2 and o1.o_orderkey < 5
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=89.94MB mem-reservation=17.94MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -483,13 +492,13 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: !empty(c.c_orders), !empty(c.c_orders)
    predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders, c_orders
-   extrapolated-rows=disabled max-scan-range-rows=50.11K
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    parquet statistics predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    parquet dictionary predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    mem-estimate=88.00MB mem-reservation=16.00MB thread-reservation=0
@@ -499,6 +508,7 @@ PLAN-ROOT SINK
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=12.37MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 07:EXCHANGE [UNPARTITIONED]
@@ -544,13 +554,13 @@ Per-Host Resources: mem-estimate=269.81MB mem-reservation=53.81MB thread-reserva
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: !empty(c.c_orders), !empty(c.c_orders)
    predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders, c_orders
-   extrapolated-rows=disabled max-scan-range-rows=50.11K
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    parquet statistics predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    parquet dictionary predicates on o1: o1.o_orderkey < CAST(5 AS BIGINT)
    mem-estimate=88.00MB mem-reservation=16.00MB thread-reservation=0
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
index 3908f90..d54b4ec 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering-disabled.test
@@ -11,6 +11,7 @@ and int_col > tinyint_col;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=42.00MB mem-reservation=16.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -20,7 +21,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=189.28KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: int_col IS NULL, int_col > CAST(1 AS INT), int_col > CAST(tinyint_col AS INT), CAST(int_col AS DOUBLE) * rand() > CAST(50 AS DOUBLE)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -28,7 +29,7 @@ PLAN-ROOT SINK
      columns: unavailable
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
-   tuple-ids=0 row-size=5B cardinality=unavailable
+   tuple-ids=0 row-size=5B cardinality=1.27K
    in pipelines: 00(GETNEXT)
 ====
 # Parquet predicates to be skipped:
@@ -48,6 +49,7 @@ and year > 2000 and month < 12;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -58,7 +60,7 @@ PLAN-ROOT SINK
 |
 00:SCAN HDFS [functional_parquet.alltypes]
    partition predicates: `year` > CAST(2000 AS INT), `month` < CAST(12 AS INT)
-   partitions=22/24 files=22 size=173.16KB
+   HDFS partitions=22/24 files=22 size=183.48KB
    predicates: bool_col, bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), id = CAST(1 AS INT), tinyint_col < CAST(50 AS TINYINT), int_col % CAST(2 AS INT) = CAST(1 AS INT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = CAST(1 AS INT), date_string_col > ' [...]
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -66,7 +68,7 @@ PLAN-ROOT SINK
      columns missing stats: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=128.00MB mem-reservation=88.00KB thread-reservation=1
-   tuple-ids=0 row-size=72B cardinality=unavailable
+   tuple-ids=0 row-size=72B cardinality=1.17K
    in pipelines: 00(GETNEXT)
 ====
 # Parquet predicates to be skipped:
@@ -79,6 +81,7 @@ and id IN (int_col);
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=58.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -88,7 +91,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=189.28KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id IN (int_col), id NOT IN (CAST(0 AS INT), CAST(1 AS INT), CAST(2 AS INT)), int_col % CAST(50 AS INT) IN (CAST(0 AS INT), CAST(1 AS INT)), string_col IN ('aaaa', 'bbbb', 'cccc', NULL)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -96,7 +99,7 @@ PLAN-ROOT SINK
      columns: unavailable
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=unavailable
+   tuple-ids=0 row-size=20B cardinality=1.27K
    in pipelines: 00(GETNEXT)
 ====
 # Nested parquet predicates to be skipped:
@@ -108,11 +111,12 @@ where a.item.e < -10;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=2,1,0 row-size=36B cardinality=unavailable
+|  tuple-ids=2,1,0 row-size=36B cardinality=440.00K
 |  in pipelines: 00(GETNEXT)
 |
 |--08:NESTED LOOP JOIN [CROSS JOIN]
@@ -155,7 +159,7 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.complextypestbl c]
-   partitions=1/1 files=2 size=6.92KB
+   HDFS partitions=1/1 files=2 size=6.92KB
    predicates: !empty(c.nested_struct.c.d)
    predicates on cn: !empty(cn.item)
    predicates on a: a.item.e < CAST(-10 AS INT)
@@ -164,7 +168,7 @@ PLAN-ROOT SINK
      columns missing stats: id
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=unavailable
+   tuple-ids=0 row-size=20B cardinality=4.40K
    in pipelines: 00(GETNEXT)
 ====
 # Parquet predicates to be skipped at each level:
@@ -176,6 +180,7 @@ o.o_lineitems l where c_custkey > 0 and o.o_orderkey > 0 and l.l_partkey > 0;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=264.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_custkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -223,14 +228,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: c_custkey > CAST(0 AS BIGINT), !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems), o.o_orderkey > CAST(0 AS BIGINT)
    predicates on l: l.l_partkey > CAST(0 AS BIGINT)
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=44.22K
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    mem-estimate=264.00MB mem-reservation=16.00MB thread-reservation=1
    tuple-ids=0 row-size=20B cardinality=15.00K
    in pipelines: 00(GETNEXT)
@@ -249,6 +254,7 @@ l.l_comment is null;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=616.00MB mem-reservation=32.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_name, o.o_clerk
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -296,14 +302,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems)
    predicates on l: l.l_shipdate = '1994-08-19', l.l_receiptdate = '1994-08-24', l.l_shipmode = 'RAIL', l.l_returnflag = 'R', l.l_comment IS NULL
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=44225
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    mem-estimate=616.00MB mem-reservation=32.00MB thread-reservation=1
    tuple-ids=0 row-size=42B cardinality=150.00K
    in pipelines: 00(GETNEXT)
@@ -325,6 +331,7 @@ and year > 2000 and month < 12;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -335,7 +342,7 @@ PLAN-ROOT SINK
 |
 00:SCAN HDFS [functional.alltypesmixedformat]
    partition predicates: `year` > CAST(2000 AS INT), `month` < CAST(12 AS INT)
-   partitions=4/4 files=4 size=66.12KB
+   HDFS partitions=4/4 files=4 size=66.33KB
    predicates: bool_col, bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), id = CAST(1 AS INT), tinyint_col < CAST(50 AS TINYINT), int_col % CAST(2 AS INT) = CAST(1 AS INT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = CAST(1 AS INT), date_string_col > ' [...]
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -343,6 +350,6 @@ PLAN-ROOT SINK
      columns missing stats: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=128.00MB mem-reservation=88.00KB thread-reservation=1
-   tuple-ids=0 row-size=72B cardinality=unavailable
+   tuple-ids=0 row-size=72B cardinality=254
    in pipelines: 00(GETNEXT)
 ====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
index 33edda9..7b94c17 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
@@ -10,6 +10,7 @@ and int_col > tinyint_col;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=42.00MB mem-reservation=16.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -19,7 +20,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=189.28KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: int_col IS NULL, int_col > CAST(1 AS INT), int_col > CAST(tinyint_col AS INT), CAST(int_col AS DOUBLE) * rand() > CAST(50 AS DOUBLE)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -29,7 +30,7 @@ PLAN-ROOT SINK
    parquet statistics predicates: int_col > CAST(1 AS INT)
    parquet dictionary predicates: int_col > CAST(1 AS INT)
    mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
-   tuple-ids=0 row-size=5B cardinality=unavailable
+   tuple-ids=0 row-size=5B cardinality=1.27K
    in pipelines: 00(GETNEXT)
 ====
 # Test non-parquet types to ensure that parquet predicates are skipped
@@ -40,6 +41,7 @@ and int_col > tinyint_col;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -49,7 +51,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    predicates: int_col IS NULL, int_col > CAST(1 AS INT), int_col > CAST(tinyint_col AS INT), CAST(int_col AS DOUBLE) * rand() > CAST(50 AS DOUBLE)
    stored statistics:
      table: rows=7.30K size=478.45KB
@@ -72,6 +74,7 @@ and year > 2000 and month < 12;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -82,7 +85,7 @@ PLAN-ROOT SINK
 |
 00:SCAN HDFS [functional_parquet.alltypes]
    partition predicates: `year` > CAST(2000 AS INT), `month` < CAST(12 AS INT)
-   partitions=22/24 files=22 size=173.16KB
+   HDFS partitions=22/24 files=22 size=183.48KB
    predicates: bool_col, bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), id = CAST(1 AS INT), tinyint_col < CAST(50 AS TINYINT), int_col % CAST(2 AS INT) = CAST(1 AS INT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = CAST(1 AS INT), date_string_col > ' [...]
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -92,7 +95,7 @@ PLAN-ROOT SINK
    parquet statistics predicates: bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), id = CAST(1 AS INT), tinyint_col < CAST(50 AS TINYINT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), date_string_col > '1993-10-01'
    parquet dictionary predicates: bool_col, bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), id = CAST(1 AS INT), tinyint_col < CAST(50 AS TINYINT), int_col % CAST(2 AS INT) = CAST(1 AS INT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = CAST(1 AS INT),  [...]
    mem-estimate=128.00MB mem-reservation=88.00KB thread-reservation=1
-   tuple-ids=0 row-size=72B cardinality=unavailable
+   tuple-ids=0 row-size=72B cardinality=1.17K
    in pipelines: 00(GETNEXT)
 ====
 # Test non-parquet files for a variety of predicates
@@ -107,6 +110,7 @@ and year > 2000 and month < 12;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -117,7 +121,7 @@ PLAN-ROOT SINK
 |
 00:SCAN HDFS [functional.alltypes]
    partition predicates: `year` > CAST(2000 AS INT), `month` < CAST(12 AS INT)
-   partitions=22/24 files=22 size=437.72KB
+   HDFS partitions=22/24 files=22 size=437.72KB
    predicates: id = CAST(1 AS INT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), bool_col, bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), tinyint_col < CAST(50 AS TINYINT), int_col % CAST(2 AS INT) = CAST(1 AS INT), timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = CAST(1 AS INT), date_string_col > ' [...]
    stored statistics:
      table: rows=7.30K size=478.45KB
@@ -141,6 +145,7 @@ and id IN (int_col);
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=58.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -150,7 +155,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=189.28KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    predicates: id IN (int_col), id NOT IN (CAST(0 AS INT), CAST(1 AS INT), CAST(2 AS INT)), int_col % CAST(50 AS INT) IN (CAST(0 AS INT), CAST(1 AS INT)), string_col IN ('aaaa', 'bbbb', 'cccc', NULL)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -159,7 +164,7 @@ PLAN-ROOT SINK
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    parquet dictionary predicates: id NOT IN (CAST(0 AS INT), CAST(1 AS INT), CAST(2 AS INT)), int_col % CAST(50 AS INT) IN (CAST(0 AS INT), CAST(1 AS INT)), string_col IN ('aaaa', 'bbbb', 'cccc', NULL)
    mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=unavailable
+   tuple-ids=0 row-size=20B cardinality=1.27K
    in pipelines: 00(GETNEXT)
 ====
 # Test collection types where all collections on the path are required (inner
@@ -171,11 +176,12 @@ where a.item.e < -10;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=2,1,0 row-size=36B cardinality=unavailable
+|  tuple-ids=2,1,0 row-size=36B cardinality=440.00K
 |  in pipelines: 00(GETNEXT)
 |
 |--08:NESTED LOOP JOIN [CROSS JOIN]
@@ -218,7 +224,7 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.complextypestbl c]
-   partitions=1/1 files=2 size=6.92KB
+   HDFS partitions=1/1 files=2 size=6.92KB
    predicates: !empty(c.nested_struct.c.d)
    predicates on cn: !empty(cn.item)
    predicates on a: a.item.e < CAST(-10 AS INT)
@@ -229,7 +235,7 @@ PLAN-ROOT SINK
    parquet statistics predicates on a: a.item.e < CAST(-10 AS INT)
    parquet dictionary predicates on a: a.item.e < CAST(-10 AS INT)
    mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=unavailable
+   tuple-ids=0 row-size=20B cardinality=4.40K
    in pipelines: 00(GETNEXT)
 ====
 # Test collection types where the lower collection in the path is optional
@@ -243,11 +249,12 @@ where a.item.e < -10;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=2N,1,0 row-size=36B cardinality=unavailable
+|  tuple-ids=2N,1,0 row-size=36B cardinality=44.00K
 |  in pipelines: 00(GETNEXT)
 |
 |--08:SUBPLAN
@@ -291,7 +298,7 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.complextypestbl c]
-   partitions=1/1 files=2 size=6.92KB
+   HDFS partitions=1/1 files=2 size=6.92KB
    predicates: !empty(c.nested_struct.c.d)
    predicates on a: a.item.e < CAST(-10 AS INT)
    stored statistics:
@@ -299,7 +306,7 @@ PLAN-ROOT SINK
      columns missing stats: id
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=unavailable
+   tuple-ids=0 row-size=20B cardinality=4.40K
    in pipelines: 00(GETNEXT)
 ====
 # Tests collection types where the outer is optional (outer join descent)
@@ -311,11 +318,12 @@ left outer join c.nested_struct.c.d cn, cn.item a where a.item.e < -10;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=2,1N,0 row-size=36B cardinality=unavailable
+|  tuple-ids=2,1N,0 row-size=36B cardinality=44.00K
 |  in pipelines: 00(GETNEXT)
 |
 |--08:SUBPLAN
@@ -358,14 +366,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.complextypestbl c]
-   partitions=1/1 files=2 size=6.92KB
+   HDFS partitions=1/1 files=2 size=6.92KB
    predicates on a: a.item.e < CAST(-10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
      columns missing stats: id
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=48.00MB mem-reservation=24.00KB thread-reservation=1
-   tuple-ids=0 row-size=20B cardinality=unavailable
+   tuple-ids=0 row-size=20B cardinality=4.40K
    in pipelines: 00(GETNEXT)
 ====
 # Test collections so that each level has a filter applied.
@@ -375,6 +383,7 @@ o.o_lineitems l where c_custkey > 0 and o.o_orderkey > 0 and l.l_partkey > 0;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=264.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_custkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -422,14 +431,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: c_custkey > CAST(0 AS BIGINT), !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems), o.o_orderkey > CAST(0 AS BIGINT)
    predicates on l: l.l_partkey > CAST(0 AS BIGINT)
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=44.23K
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    parquet statistics predicates: c_custkey > CAST(0 AS BIGINT)
    parquet statistics predicates on o: o.o_orderkey > CAST(0 AS BIGINT)
    parquet statistics predicates on l: l.l_partkey > CAST(0 AS BIGINT)
@@ -448,6 +457,7 @@ select count(*) from functional_parquet.complextypestbl c left outer join
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=26.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:AGGREGATE [FINALIZE]
@@ -458,7 +468,7 @@ PLAN-ROOT SINK
 |
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=1N,0 row-size=16B cardinality=unavailable
+|  tuple-ids=1N,0 row-size=16B cardinality=4.40K
 |  in pipelines: 00(GETNEXT)
 |
 |--04:NESTED LOOP JOIN [RIGHT OUTER JOIN]
@@ -479,14 +489,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.complextypestbl c]
-   partitions=1/1 files=2 size=6.92KB
+   HDFS partitions=1/1 files=2 size=6.92KB
    predicates on int_array: item > CAST(10 AS INT)
    stored statistics:
      table: rows=unavailable size=unavailable
      columns: unavailable
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1
-   tuple-ids=0 row-size=12B cardinality=unavailable
+   tuple-ids=0 row-size=12B cardinality=4.40K
    in pipelines: 00(GETNEXT)
 ====
 # Multiple nested collection values (at the same nesting level) where dictionary
@@ -500,6 +510,7 @@ l.l_comment is null;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=616.00MB mem-reservation=32.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_name, o.o_clerk
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -547,14 +558,14 @@ PLAN-ROOT SINK
 |     in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    predicates: !empty(c.c_orders)
    predicates on o: !empty(o.o_lineitems)
    predicates on l: l.l_shipdate = '1994-08-19', l.l_receiptdate = '1994-08-24', l.l_shipmode = 'RAIL', l.l_returnflag = 'R', l.l_comment IS NULL
    stored statistics:
-     table: rows=150.00K size=288.98MB
+     table: rows=150.00K size=288.99MB
      columns missing stats: c_orders
-   extrapolated-rows=disabled max-scan-range-rows=44.23K
+   extrapolated-rows=disabled max-scan-range-rows=50.12K
    parquet statistics predicates on l: l.l_shipdate = '1994-08-19', l.l_receiptdate = '1994-08-24', l.l_shipmode = 'RAIL', l.l_returnflag = 'R'
    parquet dictionary predicates on l: l.l_shipdate = '1994-08-19', l.l_receiptdate = '1994-08-24', l.l_shipmode = 'RAIL', l.l_returnflag = 'R'
    mem-estimate=616.00MB mem-reservation=32.00MB thread-reservation=1
@@ -574,6 +585,7 @@ and year > 2000 and month < 12;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -584,7 +596,7 @@ PLAN-ROOT SINK
 |
 00:SCAN HDFS [functional.alltypesmixedformat]
    partition predicates: `year` > CAST(2000 AS INT), `month` < CAST(12 AS INT)
-   partitions=4/4 files=4 size=66.12KB
+   HDFS partitions=4/4 files=4 size=66.33KB
    predicates: bool_col, bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), id = CAST(1 AS INT), tinyint_col < CAST(50 AS TINYINT), int_col % CAST(2 AS INT) = CAST(1 AS INT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = CAST(1 AS INT), date_string_col > ' [...]
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -594,7 +606,7 @@ PLAN-ROOT SINK
    parquet statistics predicates: bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), id = CAST(1 AS INT), tinyint_col < CAST(50 AS TINYINT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), date_string_col > '1993-10-01'
    parquet dictionary predicates: bool_col, bigint_col < CAST(5000 AS BIGINT), double_col > CAST(100.00 AS DOUBLE), float_col > CAST(50.00 AS FLOAT), id = CAST(1 AS INT), tinyint_col < CAST(50 AS TINYINT), int_col % CAST(2 AS INT) = CAST(1 AS INT), string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (CAST(1 AS SMALLINT), CAST(2 AS SMALLINT), CAST(3 AS SMALLINT), CAST(4 AS SMALLINT), CAST(5 AS SMALLINT)), timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = CAST(1 AS INT),  [...]
    mem-estimate=128.00MB mem-reservation=88.00KB thread-reservation=1
-   tuple-ids=0 row-size=72B cardinality=unavailable
+   tuple-ids=0 row-size=72B cardinality=254
    in pipelines: 00(GETNEXT)
 ====
 # Test a variety of predicates on a mixed format table.
@@ -610,12 +622,13 @@ and year != 2009 and month != 4;
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.00MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  mem-estimate=10.00MB mem-reservation=0B spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=1 row-size=8B cardinality=0
+|  tuple-ids=1 row-size=8B cardinality=1
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypesmixedformat]
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test b/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
index ef7d8c4..4d4403e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
@@ -6,11 +6,12 @@ where string_col=cast("2009-01-01 00:00:00" as timestamp);
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.stringpartitionkey.id, functional.stringpartitionkey.string_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.stringpartitionkey]
    partition predicates: CAST(string_col AS TIMESTAMP) = TIMESTAMP '2009-01-01 00:00:00'
-   partitions=1/2 files=1 size=2B
+   HDFS partitions=1/2 files=1 size=2B
    stored statistics:
      table: rows=1 size=2B
      partitions: 1/1 rows=1
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
index cd0c49a..80df92d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -8,6 +8,7 @@ Analyzed query: SELECT * FROM tpch_parquet.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
@@ -27,6 +28,7 @@ Analyzed query: SELECT * FROM tpch_parquet.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -53,6 +55,7 @@ Analyzed query: SELECT * FROM tpch_parquet.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -83,6 +86,7 @@ Analyzed query: SELECT l_comment FROM tpch_parquet.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
@@ -102,6 +106,7 @@ Analyzed query: SELECT l_comment FROM tpch_parquet.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.12MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -128,6 +133,7 @@ Analyzed query: SELECT l_comment FROM tpch_parquet.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.25MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -159,6 +165,7 @@ Analyzed query: SELECT string_col FROM functional_parquet.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: string_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
@@ -181,6 +188,7 @@ Analyzed query: SELECT string_col FROM functional_parquet.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: string_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -210,6 +218,7 @@ Analyzed query: SELECT string_col FROM functional_parquet.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=96.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: string_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -244,6 +253,7 @@ functional_parquet.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: int_col, float_col, string_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
@@ -267,6 +277,7 @@ functional_parquet.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=71.99KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: int_col, float_col, string_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -297,6 +308,7 @@ functional_parquet.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=143.99KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: int_col, float_col, string_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -330,6 +342,7 @@ Analyzed query: SELECT 'foo' FROM functional_parquet.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.00MB mem-reservation=16.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: 'foo'
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
@@ -352,6 +365,7 @@ Analyzed query: SELECT 'foo' FROM functional_parquet.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: 'foo'
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -381,6 +395,7 @@ Analyzed query: SELECT 'foo' FROM functional_parquet.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=24.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: 'foo'
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -412,6 +427,7 @@ Analyzed query: SELECT c_nationkey FROM tpch_parquet.customer
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=24.00MB mem-reservation=128.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_nationkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_parquet.customer]
@@ -435,6 +451,7 @@ Analyzed query: SELECT c_custkey FROM tpch_parquet.customer
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_custkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_parquet.customer]
@@ -461,6 +478,7 @@ Analyzed query: SELECT c_mktsegment FROM functional_parquet.customer_multiblock
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=512.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_mktsegment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.customer_multiblock]
@@ -485,6 +503,7 @@ o_orderpriority, o_clerk FROM tpch_nested_parquet.customer.c_orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=24.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: o_orderkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
@@ -510,6 +529,7 @@ tpch_nested_parquet.customer.c_orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: o_orderkey, pos
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
@@ -534,6 +554,7 @@ Analyzed query: SELECT pos FROM tpch_nested_parquet.customer.c_orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: pos
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
@@ -561,6 +582,7 @@ c.c_orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=32.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_custkey, o_orderkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -609,6 +631,7 @@ tpch_nested_parquet.customer c, c.c_orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_custkey, o_orderkey, pos
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -658,6 +681,7 @@ c.c_orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_custkey, pos
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -706,6 +730,7 @@ Analyzed query: SELECT c_custkey FROM tpch_nested_parquet.customer c, c.c_orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_custkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -754,6 +779,7 @@ c.c_orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=4.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: o_orderkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -802,6 +828,7 @@ tpch_nested_parquet.customer c, c.c_orders o, o.o_lineitems
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_custkey, o_orderkey, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -870,6 +897,7 @@ Analyzed query: SELECT * FROM tpch.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch.lineitem]
@@ -889,6 +917,7 @@ Analyzed query: SELECT * FROM tpch.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -915,6 +944,7 @@ Analyzed query: SELECT * FROM tpch.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -944,6 +974,7 @@ Analyzed query: SELECT l_comment FROM tpch.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch.lineitem]
@@ -963,6 +994,7 @@ Analyzed query: SELECT l_comment FROM tpch.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.12MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -989,6 +1021,7 @@ Analyzed query: SELECT l_comment FROM tpch.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.25MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1019,6 +1052,7 @@ Analyzed query: SELECT * FROM functional.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
@@ -1040,6 +1074,7 @@ Analyzed query: SELECT * FROM functional.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=490.49KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1068,6 +1103,7 @@ Analyzed query: SELECT * FROM functional.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=769.49KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1103,6 +1139,7 @@ Analyzed query: SELECT * FROM tpch_avro.orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_avro.orders]
@@ -1124,6 +1161,7 @@ Analyzed query: SELECT * FROM tpch_avro.orders
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=183.96KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1152,6 +1190,7 @@ Analyzed query: SELECT * FROM tpch_avro.orders
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=367.96KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1185,6 +1224,7 @@ Analyzed query: SELECT * FROM tpch_avro.orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=64.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_avro.orders]
@@ -1206,6 +1246,7 @@ Analyzed query: SELECT * FROM tpch_avro.orders
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=275.97KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_avro.orders.o_orderkey, tpch_avro.orders.o_custkey, tpch_avro.orders.o_orderstatus, tpch_avro.orders.o_totalprice, tpch_avro.orders.o_orderdate, tpch_avro.orders.o_orderpriority, tpch_avro.orders.o_clerk, tpch_avro.orders.o_shippriority, tpch_avro.orders.o_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1237,6 +1278,7 @@ Analyzed query: SELECT * FROM tpch_rc.customer
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch_rc.customer.c_custkey, tpch_rc.customer.c_name, tpch_rc.customer.c_address, tpch_rc.customer.c_nationkey, tpch_rc.customer.c_phone, tpch_rc.customer.c_acctbal, tpch_rc.customer.c_mktsegment, tpch_rc.customer.c_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_rc.customer]
@@ -1258,6 +1300,7 @@ Analyzed query: SELECT * FROM tpch_rc.customer
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=81.92KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_rc.customer.c_custkey, tpch_rc.customer.c_name, tpch_rc.customer.c_address, tpch_rc.customer.c_nationkey, tpch_rc.customer.c_phone, tpch_rc.customer.c_acctbal, tpch_rc.customer.c_mktsegment, tpch_rc.customer.c_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1286,6 +1329,7 @@ Analyzed query: SELECT * FROM tpch_rc.customer
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=163.92KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_rc.customer.c_custkey, tpch_rc.customer.c_name, tpch_rc.customer.c_address, tpch_rc.customer.c_nationkey, tpch_rc.customer.c_phone, tpch_rc.customer.c_acctbal, tpch_rc.customer.c_mktsegment, tpch_rc.customer.c_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1317,6 +1361,7 @@ Analyzed query: SELECT * FROM tpcds_seq_snap.web_returns
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpcds_seq_snap.web_returns.wr_returned_date_sk, tpcds_seq_snap.web_returns.wr_returned_time_sk, tpcds_seq_snap.web_returns.wr_item_sk, tpcds_seq_snap.web_returns.wr_refunded_customer_sk, tpcds_seq_snap.web_returns.wr_refunded_cdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_hdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_addr_sk, tpcds_seq_snap.web_returns.wr_returning_customer_sk, tpcds_seq_snap.web_returns.wr_returning_cdemo_sk, tpcds_seq_snap.web_returns.wr_returning [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpcds_seq_snap.web_returns]
@@ -1338,6 +1383,7 @@ Analyzed query: SELECT * FROM tpcds_seq_snap.web_returns
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=107.90KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpcds_seq_snap.web_returns.wr_returned_date_sk, tpcds_seq_snap.web_returns.wr_returned_time_sk, tpcds_seq_snap.web_returns.wr_item_sk, tpcds_seq_snap.web_returns.wr_refunded_customer_sk, tpcds_seq_snap.web_returns.wr_refunded_cdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_hdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_addr_sk, tpcds_seq_snap.web_returns.wr_returning_customer_sk, tpcds_seq_snap.web_returns.wr_returning_cdemo_sk, tpcds_seq_snap.web_returns.wr_returning [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1366,6 +1412,7 @@ Analyzed query: SELECT * FROM tpcds_seq_snap.web_returns
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=215.90KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpcds_seq_snap.web_returns.wr_returned_date_sk, tpcds_seq_snap.web_returns.wr_returned_time_sk, tpcds_seq_snap.web_returns.wr_item_sk, tpcds_seq_snap.web_returns.wr_refunded_customer_sk, tpcds_seq_snap.web_returns.wr_refunded_cdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_hdemo_sk, tpcds_seq_snap.web_returns.wr_refunded_addr_sk, tpcds_seq_snap.web_returns.wr_returning_customer_sk, tpcds_seq_snap.web_returns.wr_returning_cdemo_sk, tpcds_seq_snap.web_returns.wr_returning [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1398,6 +1445,7 @@ Analyzed query: SELECT * FROM tpch_orc_def.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch_orc_def.lineitem.l_orderkey, tpch_orc_def.lineitem.l_partkey, tpch_orc_def.lineitem.l_suppkey, tpch_orc_def.lineitem.l_linenumber, tpch_orc_def.lineitem.l_quantity, tpch_orc_def.lineitem.l_extendedprice, tpch_orc_def.lineitem.l_discount, tpch_orc_def.lineitem.l_tax, tpch_orc_def.lineitem.l_returnflag, tpch_orc_def.lineitem.l_linestatus, tpch_orc_def.lineitem.l_shipdate, tpch_orc_def.lineitem.l_commitdate, tpch_orc_def.lineitem.l_receiptdate, tpch_orc_def.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_orc_def.lineitem]
@@ -1423,6 +1471,7 @@ Analyzed query: SELECT * FROM tpch_orc_def.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch_orc_def.lineitem.l_orderkey, tpch_orc_def.lineitem.l_partkey, tpch_orc_def.lineitem.l_suppkey, tpch_orc_def.lineitem.l_linenumber, tpch_orc_def.lineitem.l_quantity, tpch_orc_def.lineitem.l_extendedprice, tpch_orc_def.lineitem.l_discount, tpch_orc_def.lineitem.l_tax, tpch_orc_def.lineitem.l_returnflag, tpch_orc_def.lineitem.l_linestatus, tpch_orc_def.lineitem.l_shipdate, tpch_orc_def.lineitem.l_commitdate, tpch_orc_def.lineitem.l_receiptdate, tpch_orc_def.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_orc_def.lineitem]
@@ -1448,6 +1497,7 @@ Analyzed query: SELECT l_comment FROM tpch_orc_def.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_orc_def.lineitem]
@@ -1472,6 +1522,7 @@ Analyzed query: SELECT l_comment FROM tpch_orc_def.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch_orc_def.lineitem]
@@ -1496,6 +1547,7 @@ Analyzed query: SELECT * FROM functional_orc_def.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional_orc_def.alltypes.id, functional_orc_def.alltypes.bool_col, functional_orc_def.alltypes.tinyint_col, functional_orc_def.alltypes.smallint_col, functional_orc_def.alltypes.int_col, functional_orc_def.alltypes.bigint_col, functional_orc_def.alltypes.float_col, functional_orc_def.alltypes.double_col, functional_orc_def.alltypes.date_string_col, functional_orc_def.alltypes.string_col, functional_orc_def.alltypes.timestamp_col, functional_orc_def.alltypes.year, func [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional_orc_def.alltypes]
@@ -1521,6 +1573,7 @@ Analyzed query: SELECT * FROM functional.alltypesmixedformat
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypesmixedformat.id, functional.alltypesmixedformat.bool_col, functional.alltypesmixedformat.tinyint_col, functional.alltypesmixedformat.smallint_col, functional.alltypesmixedformat.int_col, functional.alltypesmixedformat.bigint_col, functional.alltypesmixedformat.float_col, functional.alltypesmixedformat.double_col, functional.alltypesmixedformat.date_string_col, functional.alltypesmixedformat.string_col, functional.alltypesmixedformat.timestamp_col, funct [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypesmixedformat]
@@ -1543,6 +1596,7 @@ Analyzed query: SELECT * FROM functional.alltypesmixedformat
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=251.97KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypesmixedformat.id, functional.alltypesmixedformat.bool_col, functional.alltypesmixedformat.tinyint_col, functional.alltypesmixedformat.smallint_col, functional.alltypesmixedformat.int_col, functional.alltypesmixedformat.bigint_col, functional.alltypesmixedformat.float_col, functional.alltypesmixedformat.double_col, functional.alltypesmixedformat.date_string_col, functional.alltypesmixedformat.string_col, functional.alltypesmixedformat.timestamp_col, funct [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1572,6 +1626,7 @@ Analyzed query: SELECT * FROM functional.alltypesmixedformat
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=503.97KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypesmixedformat.id, functional.alltypesmixedformat.bool_col, functional.alltypesmixedformat.tinyint_col, functional.alltypesmixedformat.smallint_col, functional.alltypesmixedformat.int_col, functional.alltypesmixedformat.bigint_col, functional.alltypesmixedformat.float_col, functional.alltypesmixedformat.double_col, functional.alltypesmixedformat.date_string_col, functional.alltypesmixedformat.string_col, functional.alltypesmixedformat.timestamp_col, funct [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1604,6 +1659,7 @@ Analyzed query: SELECT * FROM functional_hbase.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=256.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_hbase.alltypes.id, functional_hbase.alltypes.bigint_col, functional_hbase.alltypes.bool_col, functional_hbase.alltypes.date_string_col, functional_hbase.alltypes.double_col, functional_hbase.alltypes.float_col, functional_hbase.alltypes.int_col, functional_hbase.alltypes.month, functional_hbase.alltypes.smallint_col, functional_hbase.alltypes.string_col, functional_hbase.alltypes.timestamp_col, functional_hbase.alltypes.tinyint_col, functional_hbase.alltypes.year
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HBASE [functional_hbase.alltypes]
@@ -1623,6 +1679,7 @@ Analyzed query: SELECT * FROM functional_hbase.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.17MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_hbase.alltypes.id, functional_hbase.alltypes.bigint_col, functional_hbase.alltypes.bool_col, functional_hbase.alltypes.date_string_col, functional_hbase.alltypes.double_col, functional_hbase.alltypes.float_col, functional_hbase.alltypes.int_col, functional_hbase.alltypes.month, functional_hbase.alltypes.smallint_col, functional_hbase.alltypes.string_col, functional_hbase.alltypes.timestamp_col, functional_hbase.alltypes.tinyint_col, functional_hbase.alltypes.year
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1649,6 +1706,7 @@ Analyzed query: SELECT * FROM functional_hbase.alltypes
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.25MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_hbase.alltypes.id, functional_hbase.alltypes.bigint_col, functional_hbase.alltypes.bool_col, functional_hbase.alltypes.date_string_col, functional_hbase.alltypes.double_col, functional_hbase.alltypes.float_col, functional_hbase.alltypes.int_col, functional_hbase.alltypes.month, functional_hbase.alltypes.smallint_col, functional_hbase.alltypes.string_col, functional_hbase.alltypes.timestamp_col, functional_hbase.alltypes.tinyint_col, functional_hbase.alltypes.year
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1677,6 +1735,7 @@ Analyzed query: SELECT * FROM functional_hbase.alltypessmall
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_hbase.alltypessmall.id, functional_hbase.alltypessmall.bigint_col, functional_hbase.alltypessmall.bool_col, functional_hbase.alltypessmall.date_string_col, functional_hbase.alltypessmall.double_col, functional_hbase.alltypessmall.float_col, functional_hbase.alltypessmall.int_col, functional_hbase.alltypessmall.month, functional_hbase.alltypessmall.smallint_col, functional_hbase.alltypessmall.string_col, functional_hbase.alltypessmall.timestamp_col, functional_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -1695,6 +1754,7 @@ Analyzed query: SELECT * FROM functional_hbase.alltypessmall
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_hbase.alltypessmall.id, functional_hbase.alltypessmall.bigint_col, functional_hbase.alltypessmall.bool_col, functional_hbase.alltypessmall.date_string_col, functional_hbase.alltypessmall.double_col, functional_hbase.alltypessmall.float_col, functional_hbase.alltypessmall.int_col, functional_hbase.alltypessmall.month, functional_hbase.alltypessmall.smallint_col, functional_hbase.alltypessmall.string_col, functional_hbase.alltypessmall.timestamp_col, functional_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1720,6 +1780,7 @@ Analyzed query: SELECT * FROM functional_hbase.alltypessmall
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=28.69KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_hbase.alltypessmall.id, functional_hbase.alltypessmall.bigint_col, functional_hbase.alltypessmall.bool_col, functional_hbase.alltypessmall.date_string_col, functional_hbase.alltypessmall.double_col, functional_hbase.alltypessmall.float_col, functional_hbase.alltypessmall.int_col, functional_hbase.alltypessmall.month, functional_hbase.alltypessmall.smallint_col, functional_hbase.alltypessmall.string_col, functional_hbase.alltypessmall.timestamp_col, functional_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1749,6 +1810,7 @@ Analyzed query: SELECT * FROM functional.alltypes_datasource
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.00GB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes_datasource.id, functional.alltypes_datasource.bool_col, functional.alltypes_datasource.tinyint_col, functional.alltypes_datasource.smallint_col, functional.alltypes_datasource.int_col, functional.alltypes_datasource.bigint_col, functional.alltypes_datasource.float_col, functional.alltypes_datasource.double_col, functional.alltypes_datasource.timestamp_col, functional.alltypes_datasource.string_col, functional.alltypes_datasource.dec_col1, functional.a [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
@@ -1765,6 +1827,7 @@ Analyzed query: SELECT * FROM functional.alltypes_datasource
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=686.41KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes_datasource.id, functional.alltypes_datasource.bool_col, functional.alltypes_datasource.tinyint_col, functional.alltypes_datasource.smallint_col, functional.alltypes_datasource.int_col, functional.alltypes_datasource.bigint_col, functional.alltypes_datasource.float_col, functional.alltypes_datasource.double_col, functional.alltypes_datasource.timestamp_col, functional.alltypes_datasource.string_col, functional.alltypes_datasource.dec_col1, functional.a [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1788,6 +1851,7 @@ Analyzed query: SELECT * FROM functional.alltypes_datasource
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=806.41KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes_datasource.id, functional.alltypes_datasource.bool_col, functional.alltypes_datasource.tinyint_col, functional.alltypes_datasource.smallint_col, functional.alltypes_datasource.int_col, functional.alltypes_datasource.bigint_col, functional.alltypes_datasource.float_col, functional.alltypes_datasource.double_col, functional.alltypes_datasource.timestamp_col, functional.alltypes_datasource.string_col, functional.alltypes_datasource.dec_col1, functional.a [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [UNPARTITIONED]
@@ -1815,6 +1879,7 @@ tpch.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:UNION
@@ -1851,6 +1916,7 @@ tpch.lineitem
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:EXCHANGE [UNPARTITIONED]
@@ -1894,6 +1960,7 @@ tpch.lineitem
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:EXCHANGE [UNPARTITIONED]
@@ -1942,6 +2009,7 @@ l_orderkey
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=114.00MB mem-reservation=38.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -1969,6 +2037,7 @@ l_orderkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=8.01MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -2017,6 +2086,7 @@ l_orderkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=8.07MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -2067,6 +2137,7 @@ Analyzed query: SELECT count(*) FROM tpch_parquet.lineitem
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.00MB mem-reservation=128.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -2092,6 +2163,7 @@ Analyzed query: SELECT count(*) FROM tpch_parquet.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -2130,6 +2202,7 @@ Analyzed query: SELECT count(*) FROM tpch_parquet.lineitem
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.02MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:AGGREGATE [FINALIZE]
@@ -2173,6 +2246,7 @@ Analyzed query: SELECT * FROM tpch_parquet.lineitem ORDER BY l_comment ASC
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=118.00MB mem-reservation=52.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -2198,6 +2272,7 @@ Analyzed query: SELECT * FROM tpch_parquet.lineitem ORDER BY l_comment ASC
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=30.69MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2231,6 +2306,7 @@ Analyzed query: SELECT * FROM tpch_parquet.lineitem ORDER BY l_comment ASC
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=61.38MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2271,6 +2347,7 @@ CAST(100 AS TINYINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=80.02MB mem-reservation=40.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:TOP-N [LIMIT=100]
@@ -2297,6 +2374,7 @@ CAST(100 AS TINYINT)
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=76.29KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2332,6 +2410,7 @@ CAST(100 AS TINYINT)
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=145.08KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
@@ -2371,6 +2450,7 @@ Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=445.94MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -2410,6 +2490,7 @@ Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -2463,6 +2544,7 @@ Analyzed query: SELECT * FROM tpch.lineitem INNER JOIN tpch.orders ON l_orderkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=12.40MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -2528,6 +2610,7 @@ tpch.orders ON l_orderkey = o_orderkey
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=445.94MB mem-reservation=51.00MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [INNER JOIN]
@@ -2567,6 +2650,7 @@ tpch.orders ON l_orderkey = o_orderkey
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:EXCHANGE [UNPARTITIONED]
@@ -2627,6 +2711,7 @@ tpch.orders ON l_orderkey = o_orderkey
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=12.40MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:EXCHANGE [UNPARTITIONED]
@@ -2698,6 +2783,7 @@ Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=420.49MB mem-reservation=16.00MB thread-reservation=3
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:NESTED LOOP JOIN [CROSS JOIN]
@@ -2732,6 +2818,7 @@ Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -2780,6 +2867,7 @@ Analyzed query: SELECT * FROM tpch.lineitem, tpch.orders
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=12.40MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey,  [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -2839,6 +2927,7 @@ Analyzed query: SELECT * FROM functional.alltypes WHERE FALSE
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:EMPTYSET
@@ -2854,6 +2943,7 @@ Analyzed query: SELECT * FROM functional.alltypes WHERE FALSE
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:EMPTYSET
@@ -2869,6 +2959,7 @@ Analyzed query: SELECT * FROM functional.alltypes WHERE FALSE
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:EMPTYSET
@@ -2889,6 +2980,7 @@ functional.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=26.00MB mem-reservation=10.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: max(tinyint_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:ANALYTIC
@@ -2924,6 +3016,7 @@ functional.alltypes
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=56.26KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: max(tinyint_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -2973,6 +3066,7 @@ functional.alltypes
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=98.26KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: max(tinyint_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -3029,6 +3123,7 @@ OVER (ORDER BY o_orderpriority ASC) rnum_priority FROM tpch_parquet.orders
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=68.00MB mem-reservation=48.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, row_number(), row_number(), row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 06:ANALYTIC
@@ -3092,6 +3187,7 @@ OVER (ORDER BY o_orderpriority ASC) rnum_priority FROM tpch_parquet.orders
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.34MB mem-reservation=32.00MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, row_number(), row_number(), row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 06:ANALYTIC
@@ -3163,6 +3259,7 @@ OVER (ORDER BY o_orderpriority ASC) rnum_priority FROM tpch_parquet.orders
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=68.68MB mem-reservation=32.00MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, row_number(), row_number(), row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 06:ANALYTIC
@@ -3252,6 +3349,7 @@ tpch_parquet.orders ON l_orderkey = o_orderkey WHERE l_shipmode = 'F'
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=140.16MB mem-reservation=78.00MB thread-reservation=3 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:UNION
@@ -3376,6 +3474,7 @@ tpch_parquet.orders ON l_orderkey = o_orderkey WHERE l_shipmode = 'F'
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.21MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 17:EXCHANGE [UNPARTITIONED]
@@ -3548,6 +3647,7 @@ tpch_parquet.orders ON l_orderkey = o_orderkey WHERE l_shipmode = 'F'
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.41MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 17:EXCHANGE [UNPARTITIONED]
@@ -3779,6 +3879,7 @@ o_orderdate ASC LIMIT CAST(100 AS TINYINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=419.25MB mem-reservation=111.50MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
+|  output exprs: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 09:TOP-N [LIMIT=100]
@@ -3880,6 +3981,7 @@ o_orderdate ASC LIMIT CAST(100 AS TINYINT)
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=31.12KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4039,6 +4141,7 @@ o_orderdate ASC LIMIT CAST(100 AS TINYINT)
 F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=59.24KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 17:MERGING-EXCHANGE [UNPARTITIONED]
@@ -4228,6 +4331,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.02MB mem-reservation=32.00KB thread-reservation=2
 WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false]
 |  partitions=1
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
 |  mem-estimate=24.52KB mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
@@ -4254,6 +4358,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.02MB mem-reservation=32.00KB thread-reservation=2
 WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false]
 |  partitions=1
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
 |  mem-estimate=24.52KB mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes, RANDOM]
@@ -4280,6 +4385,7 @@ F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
 |  Per-Host Resources: mem-estimate=32.02MB mem-reservation=64.00KB thread-reservation=2
 WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false]
 |  partitions=1
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
 |  mem-estimate=12.26KB mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes, RANDOM]
@@ -4307,6 +4413,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=106.00MB mem-reservation=20.00MB thread-reservation=2
 WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)]
 |  partitions=200516
+|  output exprs: l_comment, l_partkey
 |  mem-estimate=100.00KB mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -4333,6 +4440,7 @@ F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=28.15MB mem-reservation=12.00MB thread-reservation=1
 WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)]
 |  partitions=200516
+|  output exprs: l_comment, l_partkey
 |  mem-estimate=100.00KB mem-reservation=0B thread-reservation=0
 |
 02:SORT
@@ -4366,6 +4474,7 @@ F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=6
 |  Per-Host Resources: mem-estimate=56.59MB mem-reservation=24.00MB thread-reservation=2
 WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)]
 |  partitions=200516
+|  output exprs: l_comment, l_partkey
 |  mem-estimate=100.00KB mem-reservation=0B thread-reservation=0
 |
 02:SORT
@@ -4404,6 +4513,7 @@ F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=354.10MB mem-reservation=8.00MB thread-reservation=2
 WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)]
 |  partitions=200516
+|  output exprs: l_comment, l_partkey
 |  mem-estimate=266.10MB mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [tpch.lineitem]
@@ -4424,6 +4534,7 @@ F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=98.85MB mem-reservation=0B thread-reservation=1
 WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)]
 |  partitions=200516
+|  output exprs: l_comment, l_partkey
 |  mem-estimate=88.70MB mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [HASH(l_partkey)]
@@ -4451,6 +4562,7 @@ F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=6
 |  Per-Host Resources: mem-estimate=109.29MB mem-reservation=0B thread-reservation=2
 WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)]
 |  partitions=200516
+|  output exprs: l_comment, l_partkey
 |  mem-estimate=44.35MB mem-reservation=0B thread-reservation=0
 |
 01:EXCHANGE [HASH(l_partkey)]
@@ -4489,6 +4601,7 @@ ORDER BY o1.o_orderkey ASC LIMIT CAST(100 AS TINYINT)) v
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=345.94MB mem-reservation=85.94MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c_name, v.o_orderkey, v.o_orderstatus
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 09:AGGREGATE [FINALIZE]
@@ -4564,6 +4677,7 @@ ORDER BY o1.o_orderkey ASC LIMIT CAST(100 AS TINYINT)) v
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.16MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c_name, v.o_orderkey, v.o_orderstatus
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 12:EXCHANGE [UNPARTITIONED]
@@ -4659,6 +4773,7 @@ ORDER BY o1.o_orderkey ASC LIMIT CAST(100 AS TINYINT)) v
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.32MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c_name, v.o_orderkey, v.o_orderstatus
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 12:EXCHANGE [UNPARTITIONED]
@@ -4763,6 +4878,7 @@ o_orderpriority ASC) rnum_priority FROM c.c_orders) v
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=136.00MB mem-reservation=104.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, item.o_orderkey, item.o_orderstatus, item.o_totalprice, item.o_orderdate, item.o_orderpriority, item.o_clerk, item.o_shippriority, item.o_comment, row_number(), row_number(), row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SUBPLAN
@@ -4849,6 +4965,7 @@ o_orderpriority ASC) rnum_priority FROM c.c_orders) v
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.11MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, item.o_orderkey, item.o_orderstatus, item.o_totalprice, item.o_orderdate, item.o_orderpriority, item.o_clerk, item.o_shippriority, item.o_comment, row_number(), row_number(), row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 11:EXCHANGE [UNPARTITIONED]
@@ -4942,6 +5059,7 @@ o_orderpriority ASC) rnum_priority FROM c.c_orders) v
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=12.21MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c.c_custkey, c.c_name, c.c_address, c.c_nationkey, c.c_phone, c.c_acctbal, c.c_mktsegment, c.c_comment, item.o_orderkey, item.o_orderstatus, item.o_totalprice, item.o_orderdate, item.o_orderpriority, item.o_clerk, item.o_shippriority, item.o_comment, row_number(), row_number(), row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 11:EXCHANGE [UNPARTITIONED]
@@ -5051,6 +5169,7 @@ t1.o_orderkey
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=180.00MB mem-reservation=99.00MB thread-reservation=5 runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
+|  output exprs: t1.o_orderkey, t1.o_custkey, t1.o_orderstatus, t1.o_totalprice, t1.o_orderdate, t1.o_orderpriority, t1.o_clerk, t1.o_shippriority, t1.o_comment, t2.o_orderkey, t3.o_orderkey, t4.o_orderkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 06:HASH JOIN [INNER JOIN]
@@ -5132,6 +5251,7 @@ t1.o_orderkey
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.41MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: t1.o_orderkey, t1.o_custkey, t1.o_orderstatus, t1.o_totalprice, t1.o_orderdate, t1.o_orderpriority, t1.o_clerk, t1.o_shippriority, t1.o_comment, t2.o_orderkey, t3.o_orderkey, t4.o_orderkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 11:EXCHANGE [UNPARTITIONED]
@@ -5248,6 +5368,7 @@ t1.o_orderkey
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.82MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: t1.o_orderkey, t1.o_custkey, t1.o_orderstatus, t1.o_totalprice, t1.o_orderdate, t1.o_orderpriority, t1.o_clerk, t1.o_shippriority, t1.o_comment, t2.o_orderkey, t3.o_orderkey, t4.o_orderkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 11:EXCHANGE [UNPARTITIONED]
@@ -5403,6 +5524,7 @@ t4) v2) v1
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=137.99MB mem-reservation=176.00KB thread-reservation=5
 PLAN-ROOT SINK
+|  output exprs: t1.n_nationkey, t1.n_name, t1.n_regionkey, t1.n_comment, t2.n_nationkey, t3.n_nationkey, t4.s_suppkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 06:NESTED LOOP JOIN [CROSS JOIN]
@@ -5471,6 +5593,7 @@ t4) v2) v1
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.13MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: t1.n_nationkey, t1.n_name, t1.n_regionkey, t1.n_comment, t2.n_nationkey, t3.n_nationkey, t4.s_suppkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 10:EXCHANGE [UNPARTITIONED]
@@ -5567,6 +5690,7 @@ t4) v2) v1
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.27MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: t1.n_nationkey, t1.n_name, t1.n_regionkey, t1.n_comment, t2.n_nationkey, t3.n_nationkey, t4.s_suppkey
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 10:EXCHANGE [UNPARTITIONED]
@@ -5695,6 +5819,7 @@ functional.alltypesagg
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=34.00MB mem-reservation=18.12MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tinyint_col, smallint_col, sum(smallint_col), sum(smallint_col), sum(smallint_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:ANALYTIC
@@ -5752,6 +5877,7 @@ Analyzed query: SELECT * FROM functional_kudu.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.88MB mem-reservation=0B thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional_kudu.alltypes.id, functional_kudu.alltypes.bool_col, functional_kudu.alltypes.tinyint_col, functional_kudu.alltypes.smallint_col, functional_kudu.alltypes.int_col, functional_kudu.alltypes.bigint_col, functional_kudu.alltypes.float_col, functional_kudu.alltypes.double_col, functional_kudu.alltypes.date_string_col, functional_kudu.alltypes.string_col, functional_kudu.alltypes.timestamp_col, functional_kudu.alltypes.year, functional_kudu.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN KUDU [functional_kudu.alltypes]
@@ -5770,6 +5896,7 @@ Analyzed query: SELECT int_col FROM functional_kudu.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=384.00KB mem-reservation=0B thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: int_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN KUDU [functional_kudu.alltypes]
@@ -5788,6 +5915,7 @@ Analyzed query: SELECT count(*) FROM functional_kudu.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.00MB mem-reservation=0B thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -5812,6 +5940,7 @@ Analyzed query: SELECT * FROM tpch_kudu.nation
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.50MB mem-reservation=0B thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch_kudu.nation.n_nationkey, tpch_kudu.nation.n_name, tpch_kudu.nation.n_regionkey, tpch_kudu.nation.n_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN KUDU [tpch_kudu.nation]
@@ -5833,6 +5962,7 @@ TINYINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1.50GB mem-reservation=74.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:AGGREGATE [FINALIZE]
@@ -5860,6 +5990,7 @@ TINYINT)
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -5911,6 +6042,7 @@ Analyzed query: SELECT CAST(1 AS TINYINT) UNION SELECT CAST(1 AS TINYINT)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=20.00MB mem-reservation=0B thread-reservation=1
 INSERT INTO KUDU [functional_kudu.tinyinttable]
+|  output exprs: CAST(1 AS INT)
 |  mem-estimate=20.00MB mem-reservation=0B thread-reservation=0
 |
 00:UNION
@@ -5927,6 +6059,7 @@ Analyzed query: SELECT CAST(1 AS TINYINT) UNION SELECT CAST(1 AS TINYINT)
 F01:PLAN FRAGMENT [KUDU(KuduPartition(1))] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=22.02MB mem-reservation=2.00MB thread-reservation=1
 INSERT INTO KUDU [functional_kudu.tinyinttable]
+|  output exprs: CAST(1 AS INT)
 |  mem-estimate=20.00MB mem-reservation=0B thread-reservation=0
 |
 02:PARTIAL SORT
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/result-spooling.test b/testdata/workloads/functional-planner/queries/PlannerTest/result-spooling.test
index 6c29f59..f09a343 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/result-spooling.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/result-spooling.test
@@ -7,6 +7,7 @@ Per-Host Resource Estimates: Memory=20MB
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
@@ -40,6 +41,7 @@ Per-Host Resource Estimates: Memory=36MB
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
@@ -77,6 +79,7 @@ Per-Host Resource Estimates: Memory=257MB
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=130.69MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
@@ -108,6 +111,7 @@ Per-Host Resource Estimates: Memory=413MB
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=161.38MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
 |  mem-estimate=100.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
 |
 02:MERGING-EXCHANGE [UNPARTITIONED]
@@ -133,3 +137,4 @@ Per-Host Resources: mem-estimate=252.00MB mem-reservation=40.00MB thread-reserva
    mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=0
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
+====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test b/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
index a85b1a9..ce28957 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
@@ -4,6 +4,7 @@ select * from functional.alltypes order by random()
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -14,7 +15,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -30,6 +31,7 @@ select * from functional.alltypes order by abs(id) + abs(id)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -40,7 +42,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -56,6 +58,7 @@ select * from functional.alltypes order by tinyint_col + 1
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -65,7 +68,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -82,6 +85,7 @@ order by dayofweek(timestamp_col), true, id + 1, string_col = date_string_col, i
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -92,7 +96,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -109,6 +113,7 @@ from functional.alltypes
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=144.00MB mem-reservation=16.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: last_value(id)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:ANALYTIC
@@ -127,7 +132,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -144,6 +149,7 @@ regexp_replace(string_col, 'a.*b', 'c') limit 10
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:TOP-N [LIMIT=10]
@@ -154,7 +160,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -170,6 +176,7 @@ select * from functional.alltypes order by TestFn(double_col)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -180,7 +187,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -196,6 +203,7 @@ select concat(date_string_col, string_col) c from functional.alltypes order by c
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: concat(date_string_col, string_col)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 01:SORT
@@ -206,7 +214,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -226,6 +234,7 @@ order by id
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=42.00MB mem-reservation=16.00MB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id, row_number()
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:SORT
@@ -251,7 +260,7 @@ PLAN-ROOT SINK
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [functional.alltypestiny]
-   partitions=4/4 files=4 size=460B
+   HDFS partitions=4/4 files=4 size=460B
    stored statistics:
      table: rows=8 size=460B
      partitions: 4/4 rows=8
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
index 2b7bfd3..5538bd3 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
@@ -11,6 +11,7 @@ JOIN tpch_parquet.nation ON c_nationkey = n_nationkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.33MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.customer.c_acctbal, tpch_parquet.customer.c_mktsegment, tpch_parquet.customer.c_comment, tpch_parquet.nation.n_nationkey, tpch_parquet.nation.n_name, tpch_parquet.nation.n_regionkey, tpch_parquet.nation.n_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -36,9 +37,9 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=18.94MB thread-reservat
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.nation, RANDOM]
-|     partitions=1/1 files=1 size=2.75KB
+|     HDFS partitions=1/1 files=1 size=3.04KB
 |     stored statistics:
-|       table: rows=25 size=2.75KB
+|       table: rows=25 size=3.04KB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=25
 |     mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=1
@@ -46,10 +47,10 @@ Per-Host Resources: mem-estimate=26.95MB mem-reservation=18.94MB thread-reservat
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.customer, RANDOM]
-   partitions=1/1 files=1 size=12.31MB
+   HDFS partitions=1/1 files=1 size=12.34MB
    runtime filters: RF000[bloom] -> c_nationkey
    stored statistics:
-     table: rows=150.00K size=12.31MB
+     table: rows=150.00K size=12.34MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=150.00K
    mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1
@@ -64,6 +65,7 @@ JOIN tpch_parquet.nation ON c_nationkey = n_nationkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.65MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.customer.c_acctbal, tpch_parquet.customer.c_mktsegment, tpch_parquet.customer.c_comment, tpch_parquet.nation.n_nationkey, tpch_parquet.nation.n_name, tpch_parquet.nation.n_regionkey, tpch_parquet.nation.n_comment
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -97,9 +99,9 @@ Per-Host Resources: mem-estimate=53.88MB mem-reservation=37.88MB thread-reservat
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.nation, RANDOM]
-|     partitions=1/1 files=1 size=2.75KB
+|     HDFS partitions=1/1 files=1 size=3.04KB
 |     stored statistics:
-|       table: rows=25 size=2.75KB
+|       table: rows=25 size=3.04KB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=25
 |     mem-estimate=16.00MB mem-reservation=32.00KB thread-reservation=0
@@ -107,10 +109,10 @@ Per-Host Resources: mem-estimate=53.88MB mem-reservation=37.88MB thread-reservat
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.customer, RANDOM]
-   partitions=1/1 files=1 size=12.31MB
+   HDFS partitions=1/1 files=1 size=12.34MB
    runtime filters: RF000[bloom] -> c_nationkey
    stored statistics:
-     table: rows=150.00K size=12.31MB
+     table: rows=150.00K size=12.34MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=150.00K
    mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=0
@@ -130,6 +132,7 @@ OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.20MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -154,9 +157,9 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=74.00MB thread-reserva
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
-|     partitions=1/1 files=2 size=54.07MB
+|     HDFS partitions=1/1 files=2 size=54.21MB
 |     stored statistics:
-|       table: rows=1.50M size=54.07MB
+|       table: rows=1.50M size=54.21MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.18M
 |     mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
@@ -164,9 +167,9 @@ Per-Host Resources: mem-estimate=359.29MB mem-reservation=74.00MB thread-reserva
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -181,6 +184,7 @@ OUTER JOIN tpch_parquet.orders ON l_orderkey = o_orderkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=12.40MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -213,9 +217,9 @@ Per-Host Resources: mem-estimate=697.89MB mem-reservation=148.00MB thread-reserv
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
 |  Per-Host Resources: mem-estimate=80.00MB mem-reservation=48.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
-|     partitions=1/1 files=2 size=54.07MB
+|     HDFS partitions=1/1 files=2 size=54.21MB
 |     stored statistics:
-|       table: rows=1.50M size=54.07MB
+|       table: rows=1.50M size=54.21MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.18M
 |     mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=0
@@ -223,9 +227,9 @@ Per-Host Resources: mem-estimate=697.89MB mem-reservation=148.00MB thread-reserv
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=0
@@ -245,6 +249,7 @@ JOIN /* +shuffle */ tpch_parquet.customer ON o_custkey = c_custkey
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.77MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.orders.o_orderkey, tpch_parquet.orders.o_custkey, tpch_parquet.orders.o_orderstatus, tpch_parquet.orders.o_totalprice, tpch_parquet.orders.o_orderdate, tpch_parquet.orders.o_orderpriority, tpch_parquet.orders.o_clerk, tpch_parquet.orders.o_shippriority, tpch_parquet.orders.o_comment, tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.cu [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:EXCHANGE [UNPARTITIONED]
@@ -270,9 +275,9 @@ Per-Host Resources: mem-estimate=38.68MB mem-reservation=18.00MB thread-reservat
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
-|     partitions=1/1 files=1 size=12.31MB
+|     HDFS partitions=1/1 files=1 size=12.34MB
 |     stored statistics:
-|       table: rows=150.00K size=12.31MB
+|       table: rows=150.00K size=12.34MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=150.00K
 |     mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1
@@ -287,10 +292,10 @@ Per-Host Resources: mem-estimate=38.68MB mem-reservation=18.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 Per-Host Resources: mem-estimate=41.00MB mem-reservation=25.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
-   partitions=1/1 files=2 size=54.07MB
+   HDFS partitions=1/1 files=2 size=54.21MB
    runtime filters: RF000[bloom] -> o_custkey
    stored statistics:
-     table: rows=1.50M size=54.07MB
+     table: rows=1.50M size=54.21MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=1.18M
    mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
@@ -305,6 +310,7 @@ JOIN /* +shuffle */ tpch_parquet.customer ON o_custkey = c_custkey
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.55MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.orders.o_orderkey, tpch_parquet.orders.o_custkey, tpch_parquet.orders.o_orderstatus, tpch_parquet.orders.o_totalprice, tpch_parquet.orders.o_orderdate, tpch_parquet.orders.o_orderpriority, tpch_parquet.orders.o_clerk, tpch_parquet.orders.o_shippriority, tpch_parquet.orders.o_comment, tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.cu [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 05:EXCHANGE [UNPARTITIONED]
@@ -338,9 +344,9 @@ Per-Host Resources: mem-estimate=40.48MB mem-reservation=19.00MB thread-reservat
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
-|     partitions=1/1 files=1 size=12.31MB
+|     HDFS partitions=1/1 files=1 size=12.34MB
 |     stored statistics:
-|       table: rows=150.00K size=12.31MB
+|       table: rows=150.00K size=12.34MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=150.00K
 |     mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=0
@@ -355,10 +361,10 @@ Per-Host Resources: mem-estimate=40.48MB mem-reservation=19.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
 Per-Host Resources: mem-estimate=82.00MB mem-reservation=50.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
-   partitions=1/1 files=2 size=54.07MB
+   HDFS partitions=1/1 files=2 size=54.21MB
    runtime filters: RF000[bloom] -> o_custkey
    stored statistics:
-     table: rows=1.50M size=54.07MB
+     table: rows=1.50M size=54.21MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=1.18M
    mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=0
@@ -378,6 +384,7 @@ JOIN /* +broadcast */ tpch_parquet.customer ON o_custkey = c_custkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.77MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.orders.o_orderkey, tpch_parquet.orders.o_custkey, tpch_parquet.orders.o_orderstatus, tpch_parquet.orders.o_totalprice, tpch_parquet.orders.o_orderdate, tpch_parquet.orders.o_orderpriority, tpch_parquet.orders.o_clerk, tpch_parquet.orders.o_shippriority, tpch_parquet.orders.o_comment, tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.cu [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -403,9 +410,9 @@ Per-Host Resources: mem-estimate=85.45MB mem-reservation=59.00MB thread-reservat
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
-|     partitions=1/1 files=1 size=12.31MB
+|     HDFS partitions=1/1 files=1 size=12.34MB
 |     stored statistics:
-|       table: rows=150.00K size=12.31MB
+|       table: rows=150.00K size=12.34MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=150.00K
 |     mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=1
@@ -413,10 +420,10 @@ Per-Host Resources: mem-estimate=85.45MB mem-reservation=59.00MB thread-reservat
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
-   partitions=1/1 files=2 size=54.07MB
+   HDFS partitions=1/1 files=2 size=54.21MB
    runtime filters: RF000[bloom] -> o_custkey
    stored statistics:
-     table: rows=1.50M size=54.07MB
+     table: rows=1.50M size=54.21MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=1.18M
    mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=1
@@ -431,6 +438,7 @@ JOIN /* +broadcast */ tpch_parquet.customer ON o_custkey = c_custkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.55MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.orders.o_orderkey, tpch_parquet.orders.o_custkey, tpch_parquet.orders.o_orderstatus, tpch_parquet.orders.o_totalprice, tpch_parquet.orders.o_orderdate, tpch_parquet.orders.o_orderpriority, tpch_parquet.orders.o_clerk, tpch_parquet.orders.o_shippriority, tpch_parquet.orders.o_comment, tpch_parquet.customer.c_custkey, tpch_parquet.customer.c_name, tpch_parquet.customer.c_address, tpch_parquet.customer.c_nationkey, tpch_parquet.customer.c_phone, tpch_parquet.cu [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -464,9 +472,9 @@ Per-Host Resources: mem-estimate=150.47MB mem-reservation=118.00MB thread-reserv
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
-|     partitions=1/1 files=1 size=12.31MB
+|     HDFS partitions=1/1 files=1 size=12.34MB
 |     stored statistics:
-|       table: rows=150.00K size=12.31MB
+|       table: rows=150.00K size=12.34MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=150.00K
 |     mem-estimate=24.00MB mem-reservation=16.00MB thread-reservation=0
@@ -474,10 +482,10 @@ Per-Host Resources: mem-estimate=150.47MB mem-reservation=118.00MB thread-reserv
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
-   partitions=1/1 files=2 size=54.07MB
+   HDFS partitions=1/1 files=2 size=54.21MB
    runtime filters: RF000[bloom] -> o_custkey
    stored statistics:
-     table: rows=1.50M size=54.07MB
+     table: rows=1.50M size=54.21MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=1.18M
    mem-estimate=40.00MB mem-reservation=24.00MB thread-reservation=0
@@ -499,6 +507,7 @@ LEFT OUTER JOIN functional_parquet.alltypestiny ON alltypes.id = alltypestiny.id
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=503.95KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_parquet.alltypes.id, functional_parquet.alltypes.bool_col, functional_parquet.alltypes.tinyint_col, functional_parquet.alltypes.smallint_col, functional_parquet.alltypes.int_col, functional_parquet.alltypes.bigint_col, functional_parquet.alltypes.float_col, functional_parquet.alltypes.double_col, functional_parquet.alltypes.date_string_col, functional_parquet.alltypes.string_col, functional_parquet.alltypes.timestamp_col, functional_parquet.alltypes.year, func [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -523,7 +532,7 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.09MB thread-reservati
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB thread-reservation=2
 |  01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
-|     partitions=4/4 files=4 size=9.79KB
+|     HDFS partitions=4/4 files=4 size=11.67KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
@@ -534,7 +543,7 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.09MB thread-reservati
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   partitions=24/24 files=24 size=189.28KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/24 rows=unavailable
@@ -554,6 +563,7 @@ LEFT OUTER JOIN functional_parquet.alltypestiny ON alltypes.id = alltypestiny.id
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=1007.95KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional_parquet.alltypes.id, functional_parquet.alltypes.bool_col, functional_parquet.alltypes.tinyint_col, functional_parquet.alltypes.smallint_col, functional_parquet.alltypes.int_col, functional_parquet.alltypes.bigint_col, functional_parquet.alltypes.float_col, functional_parquet.alltypes.double_col, functional_parquet.alltypes.date_string_col, functional_parquet.alltypes.string_col, functional_parquet.alltypes.timestamp_col, functional_parquet.alltypes.year, func [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -586,7 +596,7 @@ Per-Host Resources: mem-estimate=4.03GB mem-reservation=68.17MB thread-reservati
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=176.00KB thread-reservation=2
 |  01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
-|     partitions=4/4 files=4 size=9.79KB
+|     HDFS partitions=4/4 files=4 size=11.67KB
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/4 rows=unavailable
@@ -597,7 +607,7 @@ Per-Host Resources: mem-estimate=4.03GB mem-reservation=68.17MB thread-reservati
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   partitions=24/24 files=24 size=189.28KB
+   HDFS partitions=24/24 files=24 size=200.33KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/24 rows=unavailable
@@ -620,6 +630,7 @@ GROUP BY c_nationkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c_nationkey, avg(c_acctbal)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -651,9 +662,9 @@ Per-Host Resources: mem-estimate=34.00MB mem-reservation=4.00MB thread-reservati
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.customer, RANDOM]
-   partitions=1/1 files=1 size=12.31MB
+   HDFS partitions=1/1 files=1 size=12.34MB
    stored statistics:
-     table: rows=150.00K size=12.31MB
+     table: rows=150.00K size=12.34MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=150.00K
    mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=1
@@ -668,6 +679,7 @@ GROUP BY c_nationkey
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: c_nationkey, avg(c_acctbal)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -699,9 +711,9 @@ Per-Host Resources: mem-estimate=68.00MB mem-reservation=8.00MB thread-reservati
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.customer, RANDOM]
-   partitions=1/1 files=1 size=12.31MB
+   HDFS partitions=1/1 files=1 size=12.34MB
    stored statistics:
-     table: rows=150.00K size=12.31MB
+     table: rows=150.00K size=12.34MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=150.00K
    mem-estimate=24.00MB mem-reservation=2.00MB thread-reservation=0
@@ -725,6 +737,7 @@ BIGINT)
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.10MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, o_orderstatus, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 08:EXCHANGE [UNPARTITIONED]
@@ -772,9 +785,9 @@ Per-Host Resources: mem-estimate=85.65MB mem-reservation=52.00MB thread-reservat
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
 |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
-|     partitions=1/1 files=2 size=54.07MB
+|     HDFS partitions=1/1 files=2 size=54.21MB
 |     stored statistics:
-|       table: rows=1.50M size=54.07MB
+|       table: rows=1.50M size=54.21MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.18M
 |     mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=1
@@ -789,10 +802,10 @@ Per-Host Resources: mem-estimate=85.65MB mem-reservation=52.00MB thread-reservat
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
 Per-Host Resources: mem-estimate=81.00MB mem-reservation=5.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=1
@@ -809,6 +822,7 @@ BIGINT)
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.19MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: l_orderkey, o_orderstatus, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 08:EXCHANGE [UNPARTITIONED]
@@ -864,9 +878,9 @@ Per-Host Resources: mem-estimate=107.20MB mem-reservation=87.00MB thread-reserva
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
 |  Per-Host Resources: mem-estimate=80.00MB mem-reservation=16.00MB thread-reservation=2
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
-|     partitions=1/1 files=2 size=54.07MB
+|     HDFS partitions=1/1 files=2 size=54.21MB
 |     stored statistics:
-|       table: rows=1.50M size=54.07MB
+|       table: rows=1.50M size=54.21MB
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=1.18M
 |     mem-estimate=40.00MB mem-reservation=8.00MB thread-reservation=0
@@ -881,10 +895,10 @@ Per-Host Resources: mem-estimate=107.20MB mem-reservation=87.00MB thread-reserva
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
 Per-Host Resources: mem-estimate=162.00MB mem-reservation=10.00MB thread-reservation=2 runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=4.00MB thread-reservation=0
@@ -902,6 +916,7 @@ Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.69MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -931,9 +946,9 @@ Per-Host Resources: mem-estimate=806.43MB mem-reservation=74.00MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=1
@@ -947,6 +962,7 @@ Analyzed query: SELECT DISTINCT * FROM tpch_parquet.lineitem
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=11.38MB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_ [...]
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -976,9 +992,9 @@ Per-Host Resources: mem-estimate=886.43MB mem-reservation=148.00MB thread-reserv
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.60MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.59MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=40.00MB thread-reservation=0
@@ -1000,6 +1016,7 @@ GROUP BY string_col
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=71.99KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: string_col, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -1031,7 +1048,7 @@ Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.01MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
-   partitions=4/4 files=4 size=9.79KB
+   HDFS partitions=4/4 files=4 size=11.67KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/4 rows=unavailable
@@ -1051,6 +1068,7 @@ GROUP BY string_col
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=143.99KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: string_col, count(*)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 04:EXCHANGE [UNPARTITIONED]
@@ -1082,7 +1100,7 @@ Per-Host Resources: mem-estimate=288.00MB mem-reservation=68.02MB thread-reserva
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
-   partitions=4/4 files=4 size=9.79KB
+   HDFS partitions=4/4 files=4 size=11.67KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/4 rows=unavailable
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test b/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
index e358e34..22d9144 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
@@ -4,10 +4,11 @@ select * from functional.alltypes tablesample system(10) repeatable(1234)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=3/24 files=3 size=60.68KB
+   HDFS partitions=3/24 files=3 size=60.68KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -23,10 +24,11 @@ select * from functional.alltypes tablesample system(50) repeatable(1234)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=80.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=12/24 files=12 size=239.26KB
+   HDFS partitions=12/24 files=12 size=239.26KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -44,10 +46,11 @@ where id < 10
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=80.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=12/24 files=12 size=239.26KB
+   HDFS partitions=12/24 files=12 size=239.26KB
    predicates: id < CAST(10 AS INT)
    stored statistics:
      table: rows=7.30K size=478.45KB
@@ -65,11 +68,12 @@ where year = 2009
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    partition predicates: `year` = CAST(2009 AS INT)
-   partitions=6/24 files=6 size=119.70KB
+   HDFS partitions=6/24 files=6 size=119.70KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 12/12 rows=3.65K
@@ -85,6 +89,7 @@ select * from functional.alltypes tablesample system(0)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
@@ -104,10 +109,11 @@ select * from functional.alltypes tablesample system(1) repeatable(1234)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=1/24 files=1 size=19.95KB
+   HDFS partitions=1/24 files=1 size=19.95KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -124,11 +130,12 @@ where year = 2010
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
    partition predicates: `year` = CAST(2010 AS INT)
-   partitions=1/24 files=1 size=20.36KB
+   HDFS partitions=1/24 files=1 size=20.36KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 12/12 rows=3.65K
@@ -144,10 +151,11 @@ select * from functional.alltypes tablesample system (100)
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id, functional.alltypes.bool_col, functional.alltypes.tinyint_col, functional.alltypes.smallint_col, functional.alltypes.int_col, functional.alltypes.bigint_col, functional.alltypes.float_col, functional.alltypes.double_col, functional.alltypes.date_string_col, functional.alltypes.string_col, functional.alltypes.timestamp_col, functional.alltypes.year, functional.alltypes.month
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
@@ -163,17 +171,18 @@ select id from functional_parquet.alltypes tablesample system(10) repeatable(123
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: id
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=3/24 files=3 size=23.96KB
+   HDFS partitions=3/24 files=3 size=25.50KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/24 rows=unavailable
      columns: unavailable
    extrapolated-rows=disabled max-scan-range-rows=unavailable
    mem-estimate=16.00MB mem-reservation=16.00KB thread-reservation=1
-   tuple-ids=0 row-size=4B cardinality=unavailable
+   tuple-ids=0 row-size=4B cardinality=1.27K
    in pipelines: 00(GETNEXT)
 ====
 # Sampling in a subquery.
@@ -184,6 +193,7 @@ select id from functional.alltypes t1 where exists (
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=162.94MB mem-reservation=2.98MB thread-reservation=3 runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
+|  output exprs: id
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 02:HASH JOIN [LEFT SEMI JOIN]
@@ -194,7 +204,7 @@ PLAN-ROOT SINK
 |  in pipelines: 00(GETNEXT), 01(OPEN)
 |
 |--01:SCAN HDFS [functional.alltypessmall t2]
-|     partitions=1/4 files=1 size=1.57KB
+|     HDFS partitions=1/4 files=1 size=1.57KB
 |     stored statistics:
 |       table: rows=100 size=6.32KB
 |       partitions: 4/4 rows=100
@@ -205,7 +215,7 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [functional.alltypes t1]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000[bloom] -> t1.id
    stored statistics:
      table: rows=7.30K size=478.45KB
@@ -223,10 +233,11 @@ select id from t
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB thread-reservation=2
 PLAN-ROOT SINK
+|  output exprs: functional.alltypes.id
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 00:SCAN HDFS [functional.alltypes]
-   partitions=3/24 files=3 size=60.68KB
+   HDFS partitions=3/24 files=3 size=60.68KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/union.test b/testdata/workloads/functional-planner/queries/PlannerTest/union.test
index 098e1a4..8cb122b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/union.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/union.test
@@ -17,7 +17,7 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypessmall b]
 |     partition predicates: b.`month` = 1
-|     partitions=1/4 files=1 size=1.57KB
+|     HDFS partitions=1/4 files=1 size=1.57KB
 |     row-size=89B cardinality=25
 |
 00:UNION
@@ -26,13 +26,13 @@ PLAN-ROOT SINK
 |
 |--02:SCAN HDFS [functional.alltypes]
 |     partition predicates: functional.alltypes.month = 1
-|     partitions=2/24 files=2 size=40.32KB
+|     HDFS partitions=2/24 files=2 size=40.32KB
 |     runtime filters: RF000 -> functional.alltypes.month
 |     row-size=8B cardinality=620
 |
 01:SCAN HDFS [functional.alltypes]
    partition predicates: functional.alltypes.month = 1
-   partitions=2/24 files=2 size=40.32KB
+   HDFS partitions=2/24 files=2 size=40.32KB
    runtime filters: RF000 -> functional.alltypes.month
    row-size=8B cardinality=620
 ---- DISTRIBUTEDPLAN
@@ -49,7 +49,7 @@ PLAN-ROOT SINK
 |  |
 |  03:SCAN HDFS [functional.alltypessmall b]
 |     partition predicates: b.`month` = 1
-|     partitions=1/4 files=1 size=1.57KB
+|     HDFS partitions=1/4 files=1 size=1.57KB
 |     row-size=89B cardinality=25
 |
 00:UNION
@@ -58,13 +58,13 @@ PLAN-ROOT SINK
 |
 |--02:SCAN HDFS [functional.alltypes]
 |     partition predicates: functional.alltypes.month = 1
-|     partitions=2/24 files=2 size=40.32KB
+|     HDFS partitions=2/24 files=2 size=40.32KB
 |     runtime filters: RF000 -> functional.alltypes.month
 |     row-size=8B cardinality=620
 |
 01:SCAN HDFS [functional.alltypes]
    partition predicates: functional.alltypes.month = 1
-   partitions=2/24 files=2 size=40.32KB
+   HDFS partitions=2/24 files=2 size=40.32KB
    runtime filters: RF000 -> functional.alltypes.month
    row-size=8B cardinality=620
 ====
@@ -83,17 +83,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -113,17 +113,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Only UNION ALL with limit inside operands. One of the operands also has an order by.
@@ -141,7 +141,7 @@ PLAN-ROOT SINK
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     limit: 1
 |     row-size=89B cardinality=1
 |
@@ -151,12 +151,12 @@ PLAN-ROOT SINK
 |  |
 |  02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    limit: 1
    row-size=89B cardinality=1
 ---- SCANRANGELOCATIONS
@@ -178,7 +178,7 @@ PLAN-ROOT SINK
 |  |
 |  04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     limit: 1
 |     row-size=89B cardinality=1
 |
@@ -192,7 +192,7 @@ PLAN-ROOT SINK
 |  |
 |  02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 05:EXCHANGE [UNPARTITIONED]
@@ -200,7 +200,7 @@ PLAN-ROOT SINK
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    limit: 1
    row-size=89B cardinality=1
 ====
@@ -223,17 +223,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -263,17 +263,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Only UNION ALL, mixed selects with and without from clauses, no nested unions
@@ -294,12 +294,12 @@ PLAN-ROOT SINK
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -318,12 +318,12 @@ PLAN-ROOT SINK
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Only UNION DISTINCT, mixed selects with and without from clauses, no nested unions
@@ -348,12 +348,12 @@ PLAN-ROOT SINK
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -382,12 +382,12 @@ PLAN-ROOT SINK
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Mixed UNION ALL/DISTINCT but effectively only UNION DISTINCT, no nested unions,
@@ -417,22 +417,22 @@ PLAN-ROOT SINK
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -470,22 +470,22 @@ PLAN-ROOT SINK
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Mixed UNION ALL/DISTINCT, no nested unions, with order by and limit
@@ -518,22 +518,22 @@ PLAN-ROOT SINK
 |  |
 |  |--02:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--06:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 05:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -575,22 +575,22 @@ PLAN-ROOT SINK
 |  |
 |  |--02:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--06:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 05:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Mixed UNION ALL/DISTINCT, no nested unions, with order by and limit
@@ -623,22 +623,22 @@ PLAN-ROOT SINK
 |  |
 |  |--03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  |--02:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 06:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -680,22 +680,22 @@ PLAN-ROOT SINK
 |  |
 |  |--03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  |--02:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 06:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: Only UNION ALL, first operand is nested
@@ -713,17 +713,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -743,17 +743,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: Only UNION ALL, second operand is nested
@@ -771,17 +771,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -801,17 +801,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: Only UNION DISTINCT, first operand is nested
@@ -833,17 +833,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -873,17 +873,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: Only UNION DISTINCT, second operand is nested
@@ -905,17 +905,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -945,17 +945,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION ALL doesn't absorb nested union with DISTINCT,
@@ -974,7 +974,7 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 04:AGGREGATE [FINALIZE]
@@ -987,12 +987,12 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 02:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 2:
@@ -1012,7 +1012,7 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 07:AGGREGATE [FINALIZE]
@@ -1031,12 +1031,12 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 02:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION ALL doesn't absorb nested union with DISTINCT,
@@ -1063,17 +1063,17 @@ PLAN-ROOT SINK
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1107,17 +1107,17 @@ PLAN-ROOT SINK
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION ALL absorbs the children but not directly the operands
@@ -1138,7 +1138,7 @@ PLAN-ROOT SINK
 |
 |--06:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--05:AGGREGATE [FINALIZE]
@@ -1151,17 +1151,17 @@ PLAN-ROOT SINK
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1183,7 +1183,7 @@ PLAN-ROOT SINK
 |
 |--06:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--08:AGGREGATE [FINALIZE]
@@ -1202,17 +1202,17 @@ PLAN-ROOT SINK
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION ALL absorbs the children but not directly the operands
@@ -1241,22 +1241,22 @@ PLAN-ROOT SINK
 |  |
 |  |--05:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1292,22 +1292,22 @@ PLAN-ROOT SINK
 |  |
 |  |--05:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION ALL doesn't absorb the children of a nested union
@@ -1342,22 +1342,22 @@ PLAN-ROOT SINK
 |  |  |
 |  |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |  |     partition predicates: `year` = 2009, `month` = 2
-|  |  |     partitions=1/4 files=1 size=115B
+|  |  |     HDFS partitions=1/4 files=1 size=115B
 |  |  |     row-size=89B cardinality=2
 |  |  |
 |  |  03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  07:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1403,22 +1403,22 @@ PLAN-ROOT SINK
 |  |  |
 |  |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |  |     partition predicates: `year` = 2009, `month` = 2
-|  |  |     partitions=1/4 files=1 size=115B
+|  |  |     HDFS partitions=1/4 files=1 size=115B
 |  |  |     row-size=89B cardinality=2
 |  |  |
 |  |  03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  07:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION ALL doesn't absorb nested union with order by and limit,
@@ -1438,7 +1438,7 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 04:TOP-N [LIMIT=3]
@@ -1451,12 +1451,12 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 02:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 2:
@@ -1476,7 +1476,7 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 07:EXCHANGE [RANDOM]
@@ -1495,12 +1495,12 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 02:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION ALL doesn't absorb nested union with order by and limit,
@@ -1528,17 +1528,17 @@ PLAN-ROOT SINK
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1572,17 +1572,17 @@ PLAN-ROOT SINK
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION DISTINCT absorbs nested union with ALL
@@ -1605,17 +1605,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1645,17 +1645,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION DISTINCT absorbs nested union with ALL,
@@ -1678,17 +1678,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1718,17 +1718,17 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION DISTINCT absorbs nested union with mixed ALL/DISTINCT,
@@ -1753,22 +1753,22 @@ PLAN-ROOT SINK
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1800,22 +1800,22 @@ PLAN-ROOT SINK
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION DISTINCT absorbs nested union with mixed ALL/DISTINCT,
@@ -1840,22 +1840,22 @@ PLAN-ROOT SINK
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -1887,22 +1887,22 @@ PLAN-ROOT SINK
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION DISTINCT doesn't absorb nested union with order by and limit,
@@ -1926,7 +1926,7 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 04:TOP-N [LIMIT=3]
@@ -1939,12 +1939,12 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 02:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 2:
@@ -1974,7 +1974,7 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 08:EXCHANGE [RANDOM]
@@ -1993,12 +1993,12 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 02:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Union unnesting: UNION DISTINCT doesn't absorb nested union with order by and limit
@@ -2030,17 +2030,17 @@ PLAN-ROOT SINK
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -2084,17 +2084,17 @@ PLAN-ROOT SINK
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Complex union unnesting: Multiple levels of UNION ALL, fully unnestable
@@ -2116,27 +2116,27 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 3
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -2160,27 +2160,27 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 3
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Complex union unnesting: Multiple levels of UNION DISTINCT, fully unnestable
@@ -2206,27 +2206,27 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 3
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -2260,27 +2260,27 @@ PLAN-ROOT SINK
 |
 |--05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 3
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--04:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Complex union unnesting: Partially unnestable up to 2nd level
@@ -2323,27 +2323,27 @@ PLAN-ROOT SINK
 |  |  |
 |  |  |--07:SCAN HDFS [functional.alltypestiny]
 |  |  |     partition predicates: `year` = 2009, `month` = 3
-|  |  |     partitions=1/4 files=1 size=115B
+|  |  |     HDFS partitions=1/4 files=1 size=115B
 |  |  |     row-size=89B cardinality=2
 |  |  |
 |  |  06:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -2405,27 +2405,27 @@ PLAN-ROOT SINK
 |  |  |
 |  |  |--07:SCAN HDFS [functional.alltypestiny]
 |  |  |     partition predicates: `year` = 2009, `month` = 3
-|  |  |     partitions=1/4 files=1 size=115B
+|  |  |     HDFS partitions=1/4 files=1 size=115B
 |  |  |     row-size=89B cardinality=2
 |  |  |
 |  |  06:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  |--04:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Complex union unnesting: Partially unnestable up to 1st level
@@ -2464,27 +2464,27 @@ PLAN-ROOT SINK
 |  |
 |  |--06:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 3
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -2542,27 +2542,27 @@ PLAN-ROOT SINK
 |  |
 |  |--06:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 3
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  05:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // Complex union unnesting: Multiple nested unions to test all rules in a single query
@@ -2610,27 +2610,27 @@ PLAN-ROOT SINK
 |  |  |
 |  |  |--06:SCAN HDFS [functional.alltypestiny]
 |  |  |     partition predicates: `year` = 2009, `month` = 3
-|  |  |     partitions=1/4 files=1 size=115B
+|  |  |     HDFS partitions=1/4 files=1 size=115B
 |  |  |     row-size=89B cardinality=2
 |  |  |
 |  |  05:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  |--03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  |--02:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--15:TOP-N [LIMIT=3]
@@ -2648,17 +2648,17 @@ PLAN-ROOT SINK
 |  |
 |  13:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 4
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--11:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 4
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 10:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 3
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 1:
@@ -2717,27 +2717,27 @@ PLAN-ROOT SINK
 |  |  |
 |  |  |--06:SCAN HDFS [functional.alltypestiny]
 |  |  |     partition predicates: `year` = 2009, `month` = 3
-|  |  |     partitions=1/4 files=1 size=115B
+|  |  |     HDFS partitions=1/4 files=1 size=115B
 |  |  |     row-size=89B cardinality=2
 |  |  |
 |  |  05:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  |--03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 2
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  |--02:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--21:EXCHANGE [RANDOM]
@@ -2761,17 +2761,17 @@ PLAN-ROOT SINK
 |  |
 |  13:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 4
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 |--11:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 4
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 10:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 3
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // UNION ALL in subquery
@@ -2799,17 +2799,17 @@ PLAN-ROOT SINK
 |  |
 |  |--03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 04:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 2:
@@ -2839,17 +2839,17 @@ PLAN-ROOT SINK
 |  |
 |  |--03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 04:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // UNION DISTINCT in subquery
@@ -2885,17 +2885,17 @@ PLAN-ROOT SINK
 |  |
 |  |--03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 05:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ---- SCANRANGELOCATIONS
 NODE 2:
@@ -2945,17 +2945,17 @@ PLAN-ROOT SINK
 |  |
 |  |--03:SCAN HDFS [functional.alltypestiny]
 |  |     partition predicates: `year` = 2009, `month` = 1
-|  |     partitions=1/4 files=1 size=115B
+|  |     HDFS partitions=1/4 files=1 size=115B
 |  |     row-size=89B cardinality=2
 |  |
 |  02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=89B cardinality=2
 |
 05:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=89B cardinality=2
 ====
 // UNION ALL in subquery with a WHERE condition in the outer select.
@@ -2980,7 +2980,7 @@ PLAN-ROOT SINK
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
 |     row-size=5B cardinality=1
 |
@@ -2991,7 +2991,7 @@ PLAN-ROOT SINK
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
    row-size=5B cardinality=1
 ---- SCANRANGELOCATIONS
@@ -3022,7 +3022,7 @@ PLAN-ROOT SINK
 |  |
 |  03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
 |     row-size=5B cardinality=1
 |
@@ -3040,7 +3040,7 @@ PLAN-ROOT SINK
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
    row-size=5B cardinality=1
 ====
@@ -3115,11 +3115,11 @@ PLAN-ROOT SINK
 |  |  row-size=21B cardinality=10
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=22B cardinality=8
 |
 04:SCAN HDFS [functional.alltypestiny]
-   partitions=4/4 files=4 size=460B
+   HDFS partitions=4/4 files=4 size=460B
    row-size=22B cardinality=8
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
@@ -3146,11 +3146,11 @@ PLAN-ROOT SINK
 |  |  row-size=21B cardinality=10
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=22B cardinality=8
 |
 04:SCAN HDFS [functional.alltypestiny]
-   partitions=4/4 files=4 size=460B
+   HDFS partitions=4/4 files=4 size=460B
    row-size=22B cardinality=8
 ====
 // UNION ALL/DISTINCT with mixed constant selects and non-constant selects and nested unions
@@ -3184,7 +3184,7 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=4B cardinality=2
 |
 02:TOP-N [LIMIT=3]
@@ -3193,7 +3193,7 @@ PLAN-ROOT SINK
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=4B cardinality=2
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
@@ -3222,7 +3222,7 @@ PLAN-ROOT SINK
 |
 |--03:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, `month` = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     row-size=4B cardinality=2
 |
 07:EXCHANGE [RANDOM]
@@ -3237,7 +3237,7 @@ PLAN-ROOT SINK
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, `month` = 2
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    row-size=4B cardinality=2
 ====
 // UNION ALL with only values statements
@@ -3311,11 +3311,11 @@ PLAN-ROOT SINK
 |  |  row-size=21B cardinality=10
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=22B cardinality=8
 |
 04:SCAN HDFS [functional.alltypessmall]
-   partitions=4/4 files=4 size=6.32KB
+   HDFS partitions=4/4 files=4 size=6.32KB
    row-size=22B cardinality=100
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
@@ -3342,11 +3342,11 @@ PLAN-ROOT SINK
 |  |  row-size=21B cardinality=10
 |  |
 |  01:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=22B cardinality=8
 |
 04:SCAN HDFS [functional.alltypessmall]
-   partitions=4/4 files=4 size=6.32KB
+   HDFS partitions=4/4 files=4 size=6.32KB
    row-size=22B cardinality=100
 ====
 # all union output slots are being materialized even though none is referenced by
@@ -3386,11 +3386,11 @@ PLAN-ROOT SINK
 |  row-size=89B cardinality=14.60K
 |
 |--02:SCAN HDFS [functional.alltypes]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     row-size=89B cardinality=7.30K
 |
 01:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    row-size=89B cardinality=7.30K
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
@@ -3421,11 +3421,11 @@ PLAN-ROOT SINK
 |  row-size=89B cardinality=14.60K
 |
 |--02:SCAN HDFS [functional.alltypes]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     row-size=89B cardinality=7.30K
 |
 01:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    row-size=89B cardinality=7.30K
 ====
 # UNION ALL in subquery with a WHERE condition in the outer select;
@@ -3444,13 +3444,13 @@ PLAN-ROOT SINK
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, functional.alltypestiny.month = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
 |     row-size=9B cardinality=1
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, functional.alltypestiny.month = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
    row-size=9B cardinality=1
 ---- DISTRIBUTEDPLAN
@@ -3464,13 +3464,13 @@ PLAN-ROOT SINK
 |
 |--02:SCAN HDFS [functional.alltypestiny]
 |     partition predicates: `year` = 2009, functional.alltypestiny.month = 1
-|     partitions=1/4 files=1 size=115B
+|     HDFS partitions=1/4 files=1 size=115B
 |     predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
 |     row-size=9B cardinality=1
 |
 01:SCAN HDFS [functional.alltypestiny]
    partition predicates: `year` = 2009, functional.alltypestiny.month = 1
-   partitions=1/4 files=1 size=115B
+   HDFS partitions=1/4 files=1 size=115B
    predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
    row-size=9B cardinality=1
 ====
@@ -3490,12 +3490,12 @@ PLAN-ROOT SINK
 |  row-size=0B cardinality=731
 |
 |--02:SCAN HDFS [functional.alltypes]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     predicates: functional.alltypes.bigint_col > 50, functional.alltypes.smallint_col > 20, functional.alltypes.tinyint_col + functional.alltypes.int_col < 100
 |     row-size=15B cardinality=730
 |
 01:SCAN HDFS [functional.alltypestiny]
-   partitions=4/4 files=4 size=460B
+   HDFS partitions=4/4 files=4 size=460B
    predicates: functional.alltypestiny.bigint_col > 50, functional.alltypestiny.int_col > 20
    row-size=12B cardinality=1
 ====
@@ -3535,7 +3535,7 @@ PLAN-ROOT SINK
 |
 00:UNION
 |  constant-operands=1
-|  row-size=12B cardinality=27
+|  row-size=16B cardinality=27
 |
 |--06:HASH JOIN [INNER JOIN]
 |  |  hash predicates: a.id = b.id
@@ -3543,11 +3543,11 @@ PLAN-ROOT SINK
 |  |  row-size=16B cardinality=8
 |  |
 |  |--05:SCAN HDFS [functional.alltypestiny b]
-|  |     partitions=4/4 files=4 size=460B
+|  |     HDFS partitions=4/4 files=4 size=460B
 |  |     row-size=4B cardinality=8
 |  |
 |  04:SCAN HDFS [functional.alltypestiny a]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     runtime filters: RF000 -> a.id
 |     row-size=12B cardinality=8
 |
@@ -3558,11 +3558,11 @@ PLAN-ROOT SINK
 |  |
 |  02:SCAN HDFS [functional.alltypes]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/24 files=1 size=18.12KB
+|     HDFS partitions=1/24 files=1 size=18.12KB
 |     row-size=12B cardinality=280
 |
 01:SCAN HDFS [functional.alltypestiny]
-   partitions=4/4 files=4 size=460B
+   HDFS partitions=4/4 files=4 size=460B
    row-size=12B cardinality=8
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
@@ -3571,7 +3571,7 @@ PLAN-ROOT SINK
 |
 00:UNION
 |  constant-operands=1
-|  row-size=12B cardinality=27
+|  row-size=16B cardinality=27
 |
 |--06:HASH JOIN [INNER JOIN, PARTITIONED]
 |  |  hash predicates: a.id = b.id
@@ -3581,13 +3581,13 @@ PLAN-ROOT SINK
 |  |--10:EXCHANGE [HASH(b.id)]
 |  |  |
 |  |  05:SCAN HDFS [functional.alltypestiny b]
-|  |     partitions=4/4 files=4 size=460B
+|  |     HDFS partitions=4/4 files=4 size=460B
 |  |     row-size=4B cardinality=8
 |  |
 |  09:EXCHANGE [HASH(a.id)]
 |  |
 |  04:SCAN HDFS [functional.alltypestiny a]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     runtime filters: RF000 -> a.id
 |     row-size=12B cardinality=8
 |
@@ -3605,11 +3605,11 @@ PLAN-ROOT SINK
 |  |
 |  02:SCAN HDFS [functional.alltypes]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/24 files=1 size=18.12KB
+|     HDFS partitions=1/24 files=1 size=18.12KB
 |     row-size=12B cardinality=280
 |
 01:SCAN HDFS [functional.alltypestiny]
-   partitions=4/4 files=4 size=460B
+   HDFS partitions=4/4 files=4 size=460B
    row-size=12B cardinality=8
 ====
 # Test union merging only unpartitioned inputs.
@@ -3630,7 +3630,7 @@ PLAN-ROOT SINK
 |  |  row-size=12B cardinality=10
 |  |
 |  03:SCAN HDFS [functional.alltypessmall]
-|     partitions=4/4 files=4 size=6.32KB
+|     HDFS partitions=4/4 files=4 size=6.32KB
 |     row-size=12B cardinality=100
 |
 02:AGGREGATE [FINALIZE]
@@ -3638,7 +3638,7 @@ PLAN-ROOT SINK
 |  row-size=16B cardinality=1
 |
 01:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    row-size=12B cardinality=7.30K
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
@@ -3656,7 +3656,7 @@ PLAN-ROOT SINK
 |  |  row-size=12B cardinality=10
 |  |
 |  03:SCAN HDFS [functional.alltypessmall]
-|     partitions=4/4 files=4 size=6.32KB
+|     HDFS partitions=4/4 files=4 size=6.32KB
 |     row-size=12B cardinality=100
 |
 06:AGGREGATE [FINALIZE]
@@ -3670,7 +3670,7 @@ PLAN-ROOT SINK
 |  row-size=16B cardinality=1
 |
 01:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    row-size=12B cardinality=7.30K
 ====
 # Test union merging mixed unpartitioned and partitioned inputs.
@@ -3702,11 +3702,11 @@ PLAN-ROOT SINK
 |  |  row-size=16B cardinality=8
 |  |
 |  |--09:SCAN HDFS [functional.alltypestiny b]
-|  |     partitions=4/4 files=4 size=460B
+|  |     HDFS partitions=4/4 files=4 size=460B
 |  |     row-size=4B cardinality=8
 |  |
 |  08:SCAN HDFS [functional.alltypestiny a]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     runtime filters: RF000 -> a.id
 |     row-size=12B cardinality=8
 |
@@ -3717,11 +3717,11 @@ PLAN-ROOT SINK
 |  |
 |  06:SCAN HDFS [functional.alltypes]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/24 files=1 size=18.12KB
+|     HDFS partitions=1/24 files=1 size=18.12KB
 |     row-size=12B cardinality=280
 |
 |--05:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=12B cardinality=8
 |
 |--04:TOP-N [LIMIT=10]
@@ -3729,7 +3729,7 @@ PLAN-ROOT SINK
 |  |  row-size=12B cardinality=10
 |  |
 |  03:SCAN HDFS [functional.alltypessmall]
-|     partitions=4/4 files=4 size=6.32KB
+|     HDFS partitions=4/4 files=4 size=6.32KB
 |     row-size=12B cardinality=100
 |
 02:AGGREGATE [FINALIZE]
@@ -3737,7 +3737,7 @@ PLAN-ROOT SINK
 |  row-size=16B cardinality=1
 |
 01:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    row-size=12B cardinality=7.30K
 ---- DISTRIBUTEDPLAN
 PLAN-ROOT SINK
@@ -3756,13 +3756,13 @@ PLAN-ROOT SINK
 |  |--17:EXCHANGE [HASH(b.id)]
 |  |  |
 |  |  09:SCAN HDFS [functional.alltypestiny b]
-|  |     partitions=4/4 files=4 size=460B
+|  |     HDFS partitions=4/4 files=4 size=460B
 |  |     row-size=4B cardinality=8
 |  |
 |  16:EXCHANGE [HASH(a.id)]
 |  |
 |  08:SCAN HDFS [functional.alltypestiny a]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     runtime filters: RF000 -> a.id
 |     row-size=12B cardinality=8
 |
@@ -3780,11 +3780,11 @@ PLAN-ROOT SINK
 |  |
 |  06:SCAN HDFS [functional.alltypes]
 |     partition predicates: `year` = 2009, `month` = 2
-|     partitions=1/24 files=1 size=18.12KB
+|     HDFS partitions=1/24 files=1 size=18.12KB
 |     row-size=12B cardinality=280
 |
 |--05:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=12B cardinality=8
 |
 |--19:EXCHANGE [RANDOM]
@@ -3798,7 +3798,7 @@ PLAN-ROOT SINK
 |  |  row-size=12B cardinality=10
 |  |
 |  03:SCAN HDFS [functional.alltypessmall]
-|     partitions=4/4 files=4 size=6.32KB
+|     HDFS partitions=4/4 files=4 size=6.32KB
 |     row-size=12B cardinality=100
 |
 18:EXCHANGE [RANDOM]
@@ -3814,7 +3814,7 @@ PLAN-ROOT SINK
 |  row-size=16B cardinality=1
 |
 01:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    row-size=12B cardinality=7.30K
 ====
 # IMPALA-3450: limits on union nodes are reflected in cardinality estimates. The test for
@@ -3830,11 +3830,11 @@ PLAN-ROOT SINK
 |  row-size=231B cardinality=1
 |
 |--02:SCAN HDFS [tpch.lineitem]
-|     partitions=1/1 files=1 size=718.94MB
+|     HDFS partitions=1/1 files=1 size=718.94MB
 |     row-size=231B cardinality=6.00M
 |
 01:SCAN HDFS [tpch.lineitem]
-   partitions=1/1 files=1 size=718.94MB
+   HDFS partitions=1/1 files=1 size=718.94MB
    row-size=231B cardinality=6.00M
 ====
 select l_orderkey from tpch.lineitem UNION DISTINCT (select l_orderkey from tpch.lineitem) LIMIT 1
@@ -3851,11 +3851,11 @@ PLAN-ROOT SINK
 |  row-size=8B cardinality=12.00M
 |
 |--02:SCAN HDFS [tpch.lineitem]
-|     partitions=1/1 files=1 size=718.94MB
+|     HDFS partitions=1/1 files=1 size=718.94MB
 |     row-size=8B cardinality=6.00M
 |
 01:SCAN HDFS [tpch.lineitem]
-   partitions=1/1 files=1 size=718.94MB
+   HDFS partitions=1/1 files=1 size=718.94MB
    row-size=8B cardinality=6.00M
 ====
 # IMPALA-3678: Predicates migrated into a union operand should be placed into
@@ -3885,7 +3885,7 @@ PLAN-ROOT SINK
 |  |  row-size=89B cardinality=8
 |  |
 |  06:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=89B cardinality=8
 |
 |--05:SELECT
@@ -3897,16 +3897,16 @@ PLAN-ROOT SINK
 |  |  row-size=89B cardinality=10
 |  |
 |  03:SCAN HDFS [functional.alltypessmall]
-|     partitions=4/4 files=4 size=6.32KB
+|     HDFS partitions=4/4 files=4 size=6.32KB
 |     row-size=89B cardinality=100
 |
 |--02:SCAN HDFS [functional.alltypes]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     predicates: functional.alltypes.id < 10, functional.alltypes.int_col > 20
 |     row-size=89B cardinality=730
 |
 01:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    predicates: functional.alltypes.id < 10, functional.alltypes.int_col > 20
    row-size=89B cardinality=730
 ====
@@ -3940,7 +3940,7 @@ PLAN-ROOT SINK
 |  |  row-size=89B cardinality=8
 |  |
 |  06:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=89B cardinality=8
 |
 |--05:SELECT
@@ -3952,16 +3952,16 @@ PLAN-ROOT SINK
 |  |  row-size=89B cardinality=10
 |  |
 |  03:SCAN HDFS [functional.alltypessmall]
-|     partitions=4/4 files=4 size=6.32KB
+|     HDFS partitions=4/4 files=4 size=6.32KB
 |     row-size=89B cardinality=100
 |
 |--02:SCAN HDFS [functional.alltypes]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     predicates: functional.alltypes.id < 10, functional.alltypes.int_col > 20
 |     row-size=89B cardinality=730
 |
 01:SCAN HDFS [functional.alltypes]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    predicates: functional.alltypes.id < 10, functional.alltypes.int_col > 20
    row-size=89B cardinality=730
 ====
@@ -3982,7 +3982,7 @@ PLAN-ROOT SINK
 |  |  row-size=8B cardinality=1
 |  |
 |  02:SCAN HDFS [functional.alltypestiny]
-|     partitions=4/4 files=4 size=460B
+|     HDFS partitions=4/4 files=4 size=460B
 |     row-size=0B cardinality=8
 |
 01:SCAN KUDU [functional_kudu.alltypesagg_idx]
@@ -4019,11 +4019,11 @@ PLAN-ROOT SINK
 |  |  row-size=16B cardinality=5.84K
 |  |
 |  |--08:SCAN HDFS [functional.alltypestiny t1]
-|  |     partitions=4/4 files=4 size=460B
+|  |     HDFS partitions=4/4 files=4 size=460B
 |  |     row-size=8B cardinality=8
 |  |
 |  09:SCAN HDFS [functional.alltypes t2]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     runtime filters: RF004 -> t2.bigint_col
 |     row-size=8B cardinality=7.30K
 |
@@ -4033,11 +4033,11 @@ PLAN-ROOT SINK
 |  |  row-size=16B cardinality=5.84K
 |  |
 |  |--05:SCAN HDFS [functional.alltypestiny t1]
-|  |     partitions=4/4 files=4 size=460B
+|  |     HDFS partitions=4/4 files=4 size=460B
 |  |     row-size=8B cardinality=8
 |  |
 |  06:SCAN HDFS [functional.alltypes t2]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     runtime filters: RF002 -> t2.bigint_col
 |     row-size=8B cardinality=7.30K
 |
@@ -4047,16 +4047,16 @@ PLAN-ROOT SINK
 |  |  row-size=8B cardinality=8
 |  |
 |  |--02:SCAN HDFS [functional.alltypestiny t1]
-|  |     partitions=4/4 files=4 size=460B
+|  |     HDFS partitions=4/4 files=4 size=460B
 |  |     row-size=8B cardinality=8
 |  |
 |  03:SCAN HDFS [functional.alltypes t2]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     runtime filters: RF000 -> t2.bigint_col
 |     row-size=8B cardinality=7.30K
 |
 01:SCAN HDFS [functional.alltypestiny]
-   partitions=4/4 files=4 size=460B
+   HDFS partitions=4/4 files=4 size=460B
    row-size=8B cardinality=8
 ====
 # IMPALA-3678: union in a subplan - passthrough should be disabled.
@@ -4103,7 +4103,7 @@ PLAN-ROOT SINK
 |     row-size=0B cardinality=10
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
-   partitions=1/1 files=4 size=288.99MB
+   HDFS partitions=1/1 files=4 size=288.99MB
    row-size=32B cardinality=150.00K
 ====
 # IMPALA-6388: Verify that the order of the union operands does not impact the
@@ -4117,6 +4117,7 @@ explain_level=2
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=33.50KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: f2
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:EXCHANGE [UNPARTITIONED]
@@ -4133,7 +4134,7 @@ Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB thread-reserva
 |  in pipelines: 01(GETNEXT), 02(GETNEXT)
 |
 |--02:SCAN HDFS [functional.alltypes, RANDOM]
-|     partitions=24/24 files=24 size=478.45KB
+|     HDFS partitions=24/24 files=24 size=478.45KB
 |     stored statistics:
 |       table: rows=7.30K size=478.45KB
 |       partitions: 24/24 rows=7.30K
@@ -4165,6 +4166,7 @@ explain_level=2
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=33.50KB mem-reservation=0B thread-reservation=1
 PLAN-ROOT SINK
+|  output exprs: int_col
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 03:EXCHANGE [UNPARTITIONED]
@@ -4192,7 +4194,7 @@ Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB thread-reserva
 |     in pipelines: 02(GETNEXT)
 |
 01:SCAN HDFS [functional.alltypes, RANDOM]
-   partitions=24/24 files=24 size=478.45KB
+   HDFS partitions=24/24 files=24 size=478.45KB
    stored statistics:
      table: rows=7.30K size=478.45KB
      partitions: 24/24 rows=7.30K
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
index 887001e..5f93c0b 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
@@ -13,6 +13,7 @@ row_regex:.*Per-Host Resource Estimates: Memory=[0-9.]*MB.*
 'F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 row_regex:.*Per-Host Resources: mem-estimate=[0-9.]*[A-Z]*B mem-reservation=[0-9.]*[A-Z]*B thread-reservation=1
 'PLAN-ROOT SINK'
+'|  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custkey, [...]
 row_regex:.*mem-estimate=[0-9.]*[A-Z]*B mem-reservation=[0-9.]*[A-Z]*B thread-reservation=0
 '|'
 '04:EXCHANGE [UNPARTITIONED]'
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
index a4bffe4..02339da 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
@@ -13,6 +13,7 @@ row_regex:.*Per-Host Resource Estimates: Memory=[0-9.]*MB.*
 'F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 row_regex:.*Per-Host Resources: mem-estimate=[0-9.]*[A-Z]*B mem-reservation=[0-9.]*[A-Z]*B thread-reservation=1
 '  PLAN-ROOT SINK'
+'  |  output exprs: tpch.lineitem.l_orderkey, tpch.lineitem.l_partkey, tpch.lineitem.l_suppkey, tpch.lineitem.l_linenumber, tpch.lineitem.l_quantity, tpch.lineitem.l_extendedprice, tpch.lineitem.l_discount, tpch.lineitem.l_tax, tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, tpch.lineitem.l_shipdate, tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate, tpch.lineitem.l_shipinstruct, tpch.lineitem.l_shipmode, tpch.lineitem.l_comment, tpch.orders.o_orderkey, tpch.orders.o_custke [...]
 row_regex:.*mem-estimate=[0-9.]*[A-Z]*B mem-reservation=[0-9.]*[A-Z]*B thread-reservation=0
 '  |'
 '  04:EXCHANGE [UNPARTITIONED]'
diff --git a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
index 6183f91..eac7916 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
@@ -73,6 +73,7 @@ row_regex:.*Analyzed query: SELECT id FROM test_stats_extrapolation_.*.alltypes.
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 row_regex:.*Per-Host Resources: mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
+'|  output exprs: id'
 row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN $FILESYSTEM_NAME [$DATABASE.alltypes]'
@@ -101,6 +102,7 @@ row_regex:.*Analyzed query: SELECT id FROM test_stats_extrapolation_.*.alltypes
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 row_regex:.*mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
+'|  output exprs: id'
 row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN $FILESYSTEM_NAME [$DATABASE.alltypes]'
@@ -130,6 +132,7 @@ row_regex:.*Analyzed query: SELECT id FROM test_stats_extrapolation_.*.alltypes.
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 row_regex:.*Per-Host Resources: mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
+'|  output exprs: id'
 row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN $FILESYSTEM_NAME [$DATABASE.alltypes]'
@@ -160,6 +163,7 @@ row_regex:.*Analyzed query: SELECT id FROM test_stats_extrapolation_.*.alltypes
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 row_regex:.*Per-Host Resources: mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
+'|  output exprs: id'
 row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN $FILESYSTEM_NAME [$DATABASE.alltypes]'
@@ -190,6 +194,7 @@ row_regex:.*Analyzed query: SELECT id FROM test_stats_extrapolation_.*.alltypes
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 row_regex:.*Per-Host Resources: mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
+'|  output exprs: id'
 row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN $FILESYSTEM_NAME [$DATABASE.alltypes]'
@@ -236,4 +241,4 @@ row_regex:.* extrapolated-rows=unavailable.*
 '   in pipelines: 00(GETNEXT)'
 ---- TYPES
 STRING
-====
\ No newline at end of file
+====


[impala] 03/03: IMPALA-8969: Grouping aggregator can cause segmentation fault when doing multiple aggregations

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

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

commit ab975c95171bdc0892326133a63e467398d5bbe2
Author: Zoltan Borok-Nagy <bo...@cloudera.com>
AuthorDate: Tue Sep 24 12:14:34 2019 +0200

    IMPALA-8969: Grouping aggregator can cause segmentation fault when doing multiple aggregations
    
    Grouping aggregator always tried to serialize the 0th tuple regardless
    of the aggregation index. This could lead to a segmentation fault
    because the 0th tuple might be null.
    
    Testing:
    Added a query that triggers the error to multiple-distinct-aggs.test
    
    Change-Id: I7acdd40c63166cd4986e546a992c0816f94823d5
    Reviewed-on: http://gerrit.cloudera.org:8080/14290
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exec/grouping-aggregator-ir.cc                  |  2 +-
 .../queries/QueryTest/multiple-distinct-aggs.test      | 18 +++++++++++++++++-
 2 files changed, 18 insertions(+), 2 deletions(-)

diff --git a/be/src/exec/grouping-aggregator-ir.cc b/be/src/exec/grouping-aggregator-ir.cc
index 9f7c399..d3c7f1a 100644
--- a/be/src/exec/grouping-aggregator-ir.cc
+++ b/be/src/exec/grouping-aggregator-ir.cc
@@ -201,7 +201,7 @@ Status GroupingAggregator::AddBatchStreamingImpl(int agg_idx, bool needs_seriali
 ret:
   if (needs_serialize) {
     FOREACH_ROW(out_batch, 0, out_batch_iter) {
-      AggFnEvaluator::Serialize(agg_fn_evals_, out_batch_iter.Get()->GetTuple(0));
+      AggFnEvaluator::Serialize(agg_fn_evals_, out_batch_iter.Get()->GetTuple(agg_idx));
     }
   }
 
diff --git a/testdata/workloads/functional-query/queries/QueryTest/multiple-distinct-aggs.test b/testdata/workloads/functional-query/queries/QueryTest/multiple-distinct-aggs.test
index 5ae87d8..ab7fd4e 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/multiple-distinct-aggs.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/multiple-distinct-aggs.test
@@ -436,4 +436,20 @@ where 1 not in
 BIGINT,BIGINT,BIGINT
 ---- RESULTS
 1464556,1500000,2406
-====
\ No newline at end of file
+====
+---- QUERY
+# IMPALA-8969: Grouping aggregator can cause segmentation fault when doing multiple
+# aggregations.
+select sum(len_orderkey), sum(len_comment)
+from (
+  select
+    length(group_concat(distinct cast(l_orderkey as string))) len_orderkey,
+    length(group_concat(distinct(l_comment))) len_comment
+    from tpch.lineitem
+    group by l_comment
+  ) v
+---- TYPES
+BIGINT,BIGINT
+---- RESULTS
+43737923,135857609
+====


[impala] 01/03: IMPALA-7637: Add more hash table stats to profile

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

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

commit 803323bb39aee3a3bd38e7cca544cef72500b8b6
Author: Yongzhi Chen <yc...@cloudera.com>
AuthorDate: Fri Sep 13 12:13:04 2019 -0400

    IMPALA-7637: Add more hash table stats to profile
    
    Add hash table counters(probes, travel and resizes) to profile.
    Put hash table stats into the child profile "hash table".
    
    Tests:
    Add new test test_query_profle_hashtable.
    Ran exhaustive tests.
    
    Profile Sample:
      Hash Join Builder (join_node_id=2):
            ...
            Runtime filters: 1 of 1 Runtime Filter Published
            - BuildRowsPartitionTime: 157.960us
            - BuildRowsPartitioned: 100 (100)
            - HashTablesBuildTime: 298.817us
            - LargestPartitionPercent: 7 (7)
            - MaxPartitionLevel: 0 (0)
            - NumRepartitions: 0 (0)
            - PartitionsCreated: 16 (16)
            - PeakMemoryUsage: 17.12 KB (17536)
            - RepartitionTime: 0.000ns
            - SpilledPartitions: 0 (0)
            Hash Table:
            - HashBuckets: 256 (256)
            - HashCollisions: 0 (0)
            - Probes: 2.52K (2520)
            - Resizes: 0 (0)
            - Travel: 1.79K (178
    
    Change-Id: I1fd875dd1af8031242fd5f5ff554d3a71aaa6f87
    Reviewed-on: http://gerrit.cloudera.org:8080/14234
    Reviewed-by: Sahil Takiar <st...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exec/grouping-aggregator-partition.cc |  5 +++-
 be/src/exec/grouping-aggregator.cc           |  5 ++--
 be/src/exec/grouping-aggregator.h            |  4 +--
 be/src/exec/hash-table.cc                    | 25 +++++++++++++++++
 be/src/exec/hash-table.h                     | 40 +++++++++++++++++++++++++---
 be/src/exec/partitioned-hash-join-builder.cc | 10 +++----
 be/src/exec/partitioned-hash-join-builder.h  | 11 ++++----
 tests/query_test/test_observability.py       | 28 +++++++++++++++++++
 8 files changed, 108 insertions(+), 20 deletions(-)

diff --git a/be/src/exec/grouping-aggregator-partition.cc b/be/src/exec/grouping-aggregator-partition.cc
index 03f54d8..10b24c5 100644
--- a/be/src/exec/grouping-aggregator-partition.cc
+++ b/be/src/exec/grouping-aggregator-partition.cc
@@ -214,7 +214,10 @@ void GroupingAggregator::Partition::Close(bool finalize_rows) {
     }
     aggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
   }
-  if (hash_tbl.get() != nullptr) hash_tbl->Close();
+  if (hash_tbl.get() != nullptr) {
+    hash_tbl->StatsCountersAdd(parent->ht_stats_profile_.get());
+    hash_tbl->Close();
+  }
   if (unaggregated_row_stream.get() != nullptr) {
     unaggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
   }
diff --git a/be/src/exec/grouping-aggregator.cc b/be/src/exec/grouping-aggregator.cc
index 5238c43..d94604c 100644
--- a/be/src/exec/grouping-aggregator.cc
+++ b/be/src/exec/grouping-aggregator.cc
@@ -134,7 +134,7 @@ Status GroupingAggregator::Prepare(RuntimeState* state) {
 
   ht_resize_timer_ = ADD_TIMER(runtime_profile(), "HTResizeTime");
   get_results_timer_ = ADD_TIMER(runtime_profile(), "GetResultsTime");
-  num_hash_buckets_ = ADD_COUNTER(runtime_profile(), "HashBuckets", TUnit::UNIT);
+  ht_stats_profile_ = HashTable::AddHashTableCounters(runtime_profile());
   partitions_created_ = ADD_COUNTER(runtime_profile(), "PartitionsCreated", TUnit::UNIT);
   largest_partition_percent_ =
       runtime_profile()->AddHighWaterMarkCounter("LargestPartitionPercent", TUnit::UNIT);
@@ -722,7 +722,8 @@ Status GroupingAggregator::NextPartition() {
 
   output_partition_ = partition;
   output_iterator_ = output_partition_->hash_tbl->Begin(ht_ctx_.get());
-  COUNTER_ADD(num_hash_buckets_, output_partition_->hash_tbl->num_buckets());
+  COUNTER_ADD(this->ht_stats_profile_->num_hash_buckets_,
+      output_partition_->hash_tbl->num_buckets());
   return Status::OK();
 }
 
diff --git a/be/src/exec/grouping-aggregator.h b/be/src/exec/grouping-aggregator.h
index dea7e3f..bdcae14 100644
--- a/be/src/exec/grouping-aggregator.h
+++ b/be/src/exec/grouping-aggregator.h
@@ -245,8 +245,8 @@ class GroupingAggregator : public Aggregator {
   /// Time spent returning the aggregated rows
   RuntimeProfile::Counter* get_results_timer_ = nullptr;
 
-  /// Total number of hash buckets across all partitions.
-  RuntimeProfile::Counter* num_hash_buckets_ = nullptr;
+  /// Counters and profile objects for HashTable stats
+  std::unique_ptr<HashTableStatsProfile> ht_stats_profile_;
 
   /// Total number of partitions created.
   RuntimeProfile::Counter* partitions_created_ = nullptr;
diff --git a/be/src/exec/hash-table.cc b/be/src/exec/hash-table.cc
index 556434b..6ea5fdf 100644
--- a/be/src/exec/hash-table.cc
+++ b/be/src/exec/hash-table.cc
@@ -416,6 +416,24 @@ Status HashTable::Init(bool* got_memory) {
   return Status::OK();
 }
 
+unique_ptr<HashTableStatsProfile> HashTable::AddHashTableCounters(
+    RuntimeProfile* parent_profile) {
+  unique_ptr<HashTableStatsProfile> stats_profile(new HashTableStatsProfile());
+  RuntimeProfile *hashtable_profile = stats_profile->hashtable_profile =
+      parent_profile->CreateChild("Hash Table", true, true);
+  stats_profile->num_hash_probes_ =
+      ADD_COUNTER(hashtable_profile, "Probes", TUnit::UNIT);
+  stats_profile->num_hash_travels_ =
+      ADD_COUNTER(hashtable_profile, "Travel", TUnit::UNIT);
+  stats_profile->num_hash_collisions_ =
+      ADD_COUNTER(hashtable_profile, "HashCollisions", TUnit::UNIT);
+  stats_profile->num_hash_buckets_ =
+      ADD_COUNTER(hashtable_profile, "HashBuckets", TUnit::UNIT);
+  stats_profile->num_hash_resizes_ =
+      ADD_COUNTER(hashtable_profile, "Resizes", TUnit::UNIT);
+  return stats_profile;
+}
+
 void HashTable::Close() {
   // Print statistics only for the large or heavily used hash tables.
   // TODO: Tweak these numbers/conditions, or print them always?
@@ -428,6 +446,13 @@ void HashTable::Close() {
   if (bucket_allocation_ != nullptr) allocator_->Free(move(bucket_allocation_));
 }
 
+void HashTable::StatsCountersAdd(HashTableStatsProfile* profile) {
+  COUNTER_ADD(profile->num_hash_collisions_, num_hash_collisions_);
+  COUNTER_ADD(profile->num_hash_probes_, num_probes_);
+  COUNTER_ADD(profile->num_hash_travels_, travel_length_);
+  COUNTER_ADD(profile->num_hash_resizes_, this->num_resizes_);
+}
+
 Status HashTable::CheckAndResize(
     uint64_t buckets_to_fill, const HashTableCtx* ht_ctx, bool* got_memory) {
   uint64_t shift = 0;
diff --git a/be/src/exec/hash-table.h b/be/src/exec/hash-table.h
index b4a6905..c4df52c 100644
--- a/be/src/exec/hash-table.h
+++ b/be/src/exec/hash-table.h
@@ -34,6 +34,7 @@
 #include "runtime/tuple-row.h"
 #include "util/bitmap.h"
 #include "util/hash-util.h"
+#include "util/runtime-profile.h"
 
 namespace llvm {
   class Function;
@@ -528,6 +529,30 @@ class HashTableCtx {
   MemPool* probe_expr_results_pool_;
 };
 
+/// HashTableStatsProfile encapsulates hash tables stats. It tracks the stats of all the
+/// hash tables created by a node. It should be created, stored by the node, and be
+/// released when the node is released.
+struct HashTableStatsProfile {
+  /// Profile object for HashTable Stats
+  RuntimeProfile* hashtable_profile = nullptr;
+
+  /// Number of hash collisions - unequal rows that have identical hash values
+  RuntimeProfile::Counter* num_hash_collisions_ = nullptr;
+
+  /// Number of hash table probes.
+  RuntimeProfile::Counter* num_hash_probes_ = nullptr;
+
+  /// Total distance traveled for each hash table probe.
+  RuntimeProfile::Counter* num_hash_travels_ = nullptr;
+
+  /// Number of hash table resized
+  RuntimeProfile::Counter* num_hash_resizes_ = nullptr;
+
+  /// Total number of hash buckets across all partitions.
+  RuntimeProfile::Counter* num_hash_buckets_ = nullptr;
+
+};
+
 /// The hash table consists of a contiguous array of buckets that contain a pointer to the
 /// data, the hash value and three flags: whether this bucket is filled, whether this
 /// entry has been matched (used in right and full joins) and whether this entry has
@@ -618,9 +643,21 @@ class HashTable {
   /// enough memory for the initial buckets was allocated from the Suballocator.
   Status Init(bool* got_memory) WARN_UNUSED_RESULT;
 
+  /// Create the counters for HashTable stats and put them into the child profile
+  /// "Hash Table".
+  /// Returns a HashTableStatsProfile object.
+  static std::unique_ptr<HashTableStatsProfile> AddHashTableCounters(
+      RuntimeProfile* parent_profile);
+
   /// Call to cleanup any resources. Must be called once.
   void Close();
 
+  /// Add operations stats of this hash table to the counters in profile.
+  /// This method should only be called once for each HashTable and be called during
+  /// closing the owner object of the HashTable. Not all the counters are added with the
+  /// method, only counters for Probes, travels, collisions and resizes are affected.
+  void StatsCountersAdd(HashTableStatsProfile* profile);
+
   /// Inserts the row to the hash table. The caller is responsible for ensuring that the
   /// table has free buckets. Returns true if the insertion was successful. Always
   /// returns true if the table has free buckets and the key is not a duplicate. If the
@@ -737,9 +774,6 @@ class HashTable {
   /// Update and print some statistics that can be used for performance debugging.
   std::string PrintStats() const;
 
-  /// Number of hash collisions so far in the lifetime of this object
-  int64_t NumHashCollisions() const { return num_hash_collisions_; }
-
   /// stl-like iterator interface.
   class Iterator {
    private:
diff --git a/be/src/exec/partitioned-hash-join-builder.cc b/be/src/exec/partitioned-hash-join-builder.cc
index 175d7d8..2f9017d 100644
--- a/be/src/exec/partitioned-hash-join-builder.cc
+++ b/be/src/exec/partitioned-hash-join-builder.cc
@@ -71,8 +71,6 @@ PhjBuilder::PhjBuilder(int join_node_id, const string& join_node_label,
     largest_partition_percent_(NULL),
     max_partition_level_(NULL),
     num_build_rows_partitioned_(NULL),
-    num_hash_collisions_(NULL),
-    num_hash_buckets_(NULL),
     num_spilled_partitions_(NULL),
     num_repartitions_(NULL),
     partition_build_rows_timer_(NULL),
@@ -133,8 +131,7 @@ Status PhjBuilder::Prepare(RuntimeState* state, MemTracker* parent_mem_tracker)
       profile()->AddHighWaterMarkCounter("MaxPartitionLevel", TUnit::UNIT);
   num_build_rows_partitioned_ =
       ADD_COUNTER(profile(), "BuildRowsPartitioned", TUnit::UNIT);
-  num_hash_collisions_ = ADD_COUNTER(profile(), "HashCollisions", TUnit::UNIT);
-  num_hash_buckets_ = ADD_COUNTER(profile(), "HashBuckets", TUnit::UNIT);
+  ht_stats_profile_ = HashTable::AddHashTableCounters(profile());
   num_spilled_partitions_ = ADD_COUNTER(profile(), "SpilledPartitions", TUnit::UNIT);
   num_repartitions_ = ADD_COUNTER(profile(), "NumRepartitions", TUnit::UNIT);
   partition_build_rows_timer_ = ADD_TIMER(profile(), "BuildRowsPartitionTime");
@@ -609,7 +606,7 @@ void PhjBuilder::Partition::Close(RowBatch* batch) {
   if (IsClosed()) return;
 
   if (hash_tbl_ != NULL) {
-    COUNTER_ADD(parent_->num_hash_collisions_, hash_tbl_->NumHashCollisions());
+    hash_tbl_->StatsCountersAdd(parent_->ht_stats_profile_.get());
     hash_tbl_->Close();
   }
 
@@ -711,7 +708,8 @@ Status PhjBuilder::Partition::BuildHashTable(bool* built) {
   DCHECK(*built);
   DCHECK(hash_tbl_ != NULL);
   is_spilled_ = false;
-  COUNTER_ADD(parent_->num_hash_buckets_, hash_tbl_->num_buckets());
+  COUNTER_ADD(parent_->ht_stats_profile_->num_hash_buckets_,
+      hash_tbl_->num_buckets());
   return Status::OK();
 
 not_built:
diff --git a/be/src/exec/partitioned-hash-join-builder.h b/be/src/exec/partitioned-hash-join-builder.h
index 9acc7a7..bbdc472 100644
--- a/be/src/exec/partitioned-hash-join-builder.h
+++ b/be/src/exec/partitioned-hash-join-builder.h
@@ -133,6 +133,8 @@ class PhjBuilder : public DataSink {
   /// depends on the join type and the equijoin conjuncts.
   bool HashTableStoresNulls() const;
 
+  void AddHashTableStatsToProfile(RuntimeProfile* profile);
+
   /// Accessor functions, mainly required to expose state to PartitionedHashJoinNode.
   inline bool non_empty_build() const { return non_empty_build_; }
   inline const std::vector<bool>& is_not_distinct_from() const {
@@ -423,6 +425,9 @@ class PhjBuilder : public DataSink {
   /// The level is set to the same level as 'hash_partitions_'.
   boost::scoped_ptr<HashTableCtx> ht_ctx_;
 
+  /// Counters and profile objects for HashTable stats
+  std::unique_ptr<HashTableStatsProfile> ht_stats_profile_;
+
   /// Total number of partitions created.
   RuntimeProfile::Counter* partitions_created_;
 
@@ -436,12 +441,6 @@ class PhjBuilder : public DataSink {
   /// Number of build rows that have been partitioned.
   RuntimeProfile::Counter* num_build_rows_partitioned_;
 
-  /// Number of hash collisions - unequal rows that have identical hash values
-  RuntimeProfile::Counter* num_hash_collisions_;
-
-  /// Total number of hash buckets across all partitions.
-  RuntimeProfile::Counter* num_hash_buckets_;
-
   /// Number of partitions that have been spilled.
   RuntimeProfile::Counter* num_spilled_partitions_;
 
diff --git a/tests/query_test/test_observability.py b/tests/query_test/test_observability.py
index 09674fd..463f7a5 100644
--- a/tests/query_test/test_observability.py
+++ b/tests/query_test/test_observability.py
@@ -709,3 +709,31 @@ class TestObservability(ImpalaTestSuite):
     # Call the second time, no metastore loading needed.
     runtime_profile = self.execute_query(query).runtime_profile
     assert storageLoadTime not in runtime_profile
+
+  def __verify_hashtable_stats_profile(self, runtime_profile):
+    assert "Hash Table" in runtime_profile
+    assert "Probes:" in runtime_profile
+    assert "Travel:" in runtime_profile
+    assert "HashCollisions:" in runtime_profile
+    assert "Resizes:" in runtime_profile
+    nprobes = re.search('Probes:.*\((\d+)\)', runtime_profile)
+    # Probes and travel can be 0. The number can be an integer or float with K.
+    assert nprobes and len(nprobes.groups()) == 1 and nprobes.group(1) >= 0
+    ntravel = re.search('Travel:.*\((\d+)\)', runtime_profile)
+    assert ntravel and len(ntravel.groups()) == 1 and ntravel.group(1) >= 0
+
+  def test_query_profle_hashtable(self):
+    """Test that the profile for join/aggregate contains hash table related
+    information."""
+    # Join
+    query = """select a.int_col, a.string_col from functional.alltypes a
+        inner join functional.alltypessmall b on a.id = b.id"""
+    result = self.execute_query(query)
+    assert result.success
+    self.__verify_hashtable_stats_profile(result.runtime_profile)
+    # Group by
+    query = """select year, count(*) from
+        functional.alltypesagg where int_col < 7 and year = 2010 group by year"""
+    result = self.execute_query(query)
+    assert result.success
+    self.__verify_hashtable_stats_profile(result.runtime_profile)