You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2017/04/19 01:20:28 UTC

[1/6] incubator-impala git commit: IMPALA-5080: OutOfMemory PermGen space

Repository: incubator-impala
Updated Branches:
  refs/heads/master 955b257cf -> 9a29dfc91


IMPALA-5080: OutOfMemory PermGen space

The problem is that over time our usage of PermGen space has
gradually gone up, recently hitting a point where we may run
out during testing. The fix is to increase the allowed max
perm gen space.

The default is 64mb on 32bit systems and 82mb on 64bit systems.
After bumping it up to 128mb, I haven't seen any failures on my
machine.

I also verified that this is not a leak by running the test,
test_java_udfs, 10 times, checking the perm gen usage with jmap,
running it a further 300 times, and checking that the perm gen
usage remained the same.

Change-Id: Iccfb69cdf7958e053890089db6a82ad491fb5b9d
Reviewed-on: http://gerrit.cloudera.org:8080/6642
Tested-by: Impala Public Jenkins
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/1fa33153
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/1fa33153
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/1fa33153

Branch: refs/heads/master
Commit: 1fa331537acf4c2460f3787ba81a5a64cac89c99
Parents: 955b257
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Fri Apr 14 13:28:52 2017 -0700
Committer: Matthew Jacobs <mj...@cloudera.com>
Committed: Tue Apr 18 20:02:44 2017 +0000

----------------------------------------------------------------------
 bin/impala-config.sh | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/1fa33153/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 3ee12ad..91d9106 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -432,6 +432,9 @@ export LIBHDFS_OPTS="${LIBHDFS_OPTS:-} -Djava.library.path=${HADOOP_LIB_DIR}/nat
 # TODO: Consider having cmake scripts change this value depending on
 # the build type.
 LIBHDFS_OPTS="${LIBHDFS_OPTS}:${IMPALA_HOME}/be/build/debug/service"
+# IMPALA-5080: Our use of PermGen space sometimes exceeds the default maximum while
+# running tests that load UDF jars.
+LIBHDFS_OPTS="${LIBHDFS_OPTS} -XX:MaxPermSize=128mb"
 
 export ARTISTIC_STYLE_OPTIONS="$IMPALA_BE_DIR/.astylerc"
 


[5/6] incubator-impala git commit: IMPALA-3748: minimum buffer requirements in planner

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/PlanNode.java
----------------------------------------------------------------------
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 978e0ac..d5c72a0 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -65,6 +65,10 @@ import com.google.common.math.LongMath;
 abstract public class PlanNode extends TreeNode<PlanNode> {
   private final static Logger LOG = LoggerFactory.getLogger(PlanNode.class);
 
+  // The size of buffer used in spilling nodes. Used in computeResourceProfile().
+  // TODO: IMPALA-3200: get from query option
+  protected final static long SPILLABLE_BUFFER_BYTES = 8L * 1024L * 1024L;
+
   // String used for this node in getExplainString().
   protected String displayName_;
 
@@ -110,13 +114,13 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   // set in computeStats(); invalid: -1
   protected int numNodes_;
 
+  // resource requirements and estimates for this plan node.
+  // set in computeResourceProfile().
+  protected ResourceProfile resourceProfile_ = null;
+
   // sum of tupleIds_' avgSerializedSizes; set in computeStats()
   protected float avgRowSize_;
 
-  // estimated per-host memory requirement for this node;
-  // set in computeCosts(); invalid: -1
-  protected long perHostMemCost_ = -1;
-
   // If true, disable codegen for this plan node.
   protected boolean disableCodegen_;
 
@@ -187,9 +191,9 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   }
   public long getLimit() { return limit_; }
   public boolean hasLimit() { return limit_ > -1; }
-  public long getPerHostMemCost() { return perHostMemCost_; }
   public long getCardinality() { return cardinality_; }
   public int getNumNodes() { return numNodes_; }
+  public ResourceProfile getResourceProfile() { return resourceProfile_; }
   public float getAvgRowSize() { return avgRowSize_; }
   public void setFragment(PlanFragment fragment) { fragment_ = fragment; }
   public PlanFragment getFragment() { return fragment_; }
@@ -235,8 +239,8 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     conjuncts_.clear();
   }
 
-  public String getExplainString() {
-    return getExplainString("", "", TExplainLevel.VERBOSE);
+  public String getExplainString(TQueryOptions queryOptions) {
+    return getExplainString("", "", queryOptions, TExplainLevel.VERBOSE);
   }
 
   protected void setDisplayName(String s) { displayName_ = s; }
@@ -269,7 +273,7 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
    * output will be prefixed by prefix.
    */
   protected final String getExplainString(String rootPrefix, String prefix,
-      TExplainLevel detailLevel) {
+      TQueryOptions queryOptions, TExplainLevel detailLevel) {
     StringBuilder expBuilder = new StringBuilder();
     String detailPrefix = prefix;
     String filler;
@@ -302,11 +306,12 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
     // Output cardinality, cost estimates and tuple Ids only when explain plan level
     // is extended or above.
     if (detailLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
-      // Print estimated output cardinality and memory cost.
-      expBuilder.append(PrintUtils.printHosts(detailPrefix, numNodes_));
-      expBuilder.append(PrintUtils.printMemCost(" ", perHostMemCost_) + "\n");
+      // Print resource profile.
+      expBuilder.append(detailPrefix);
+      expBuilder.append(resourceProfile_.getExplainString());
+      expBuilder.append("\n");
 
-      // Print tuple ids and row size.
+      // Print tuple ids, row size and cardinality.
       expBuilder.append(detailPrefix + "tuple-ids=");
       for (int i = 0; i < tupleIds_.size(); ++i) {
         TupleId tupleId = tupleIds_.get(i);
@@ -331,15 +336,23 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
           // we're crossing a fragment boundary
           expBuilder.append(
               child.fragment_.getExplainString(
-                childHeadlinePrefix, childDetailPrefix, detailLevel));
+                childHeadlinePrefix, childDetailPrefix, queryOptions, detailLevel));
         } else {
-          expBuilder.append(
-              child.getExplainString(childHeadlinePrefix, childDetailPrefix,
-                  detailLevel));
+          expBuilder.append(child.getExplainString(childHeadlinePrefix,
+              childDetailPrefix, queryOptions, detailLevel));
         }
         if (printFiller) expBuilder.append(filler + "\n");
       }
-      expBuilder.append(children_.get(0).getExplainString(prefix, prefix, detailLevel));
+      PlanFragment childFragment = children_.get(0).fragment_;
+      if (fragment_ != childFragment && detailLevel == TExplainLevel.EXTENDED) {
+        // we're crossing a fragment boundary - print the fragment header.
+        expBuilder.append(prefix);
+        expBuilder.append(
+            childFragment.getFragmentHeaderString(queryOptions.getMt_dop()));
+        expBuilder.append("\n");
+      }
+      expBuilder.append(
+          children_.get(0).getExplainString(prefix, prefix, queryOptions, detailLevel));
     }
     return expBuilder.toString();
   }
@@ -379,7 +392,7 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
 
     TExecStats estimatedStats = new TExecStats();
     estimatedStats.setCardinality(cardinality_);
-    estimatedStats.setMemory_used(perHostMemCost_);
+    estimatedStats.setMemory_used(resourceProfile_.getMemEstimateBytes());
     msg.setLabel(getDisplayLabel());
     msg.setLabel_detail(getDisplayLabelDetail());
     msg.setEstimated_stats(estimatedStats);
@@ -605,13 +618,12 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   public boolean isBlockingNode() { return false; }
 
   /**
-   * Estimates the cost of executing this PlanNode. Currently only sets perHostMemCost_.
-   * May only be called after this PlanNode has been placed in a PlanFragment because
-   * the cost computation is dependent on the enclosing fragment's data partition.
+   * Compute resources consumed when executing this PlanNode, initializing
+   * 'resource_profile_'. May only be called after this PlanNode has been placed in a
+   * PlanFragment because the cost computation is dependent on the enclosing fragment's
+   * data partition.
    */
-  public void computeCosts(TQueryOptions queryOptions) {
-    perHostMemCost_ = 0;
-  }
+  public abstract void computeResourceProfile(TQueryOptions queryOptions);
 
   /**
    * The input cardinality is the sum of output cardinalities of its children.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
----------------------------------------------------------------------
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 a199f54..fba9149 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
@@ -20,6 +20,7 @@ package org.apache.impala.planner;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExplainLevel;
+import org.apache.impala.thrift.TQueryOptions;
 
 /**
  * Sink for the root of a query plan that produces result rows. Allows coordination
@@ -28,9 +29,15 @@ import org.apache.impala.thrift.TExplainLevel;
  */
 public class PlanRootSink extends DataSink {
 
-  public String getExplainString(String prefix, String detailPrefix,
-      TExplainLevel explainLevel) {
-    return String.format("%sPLAN-ROOT SINK\n", prefix);
+  public void appendSinkExplainString(String prefix, String detailPrefix,
+      TQueryOptions queryOptions, TExplainLevel explainLevel, StringBuilder output) {
+    output.append(String.format("%sPLAN-ROOT SINK\n", prefix));
+  }
+
+  @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add a memory estimate
+    resourceProfile_ = new ResourceProfile(0, 0);
   }
 
   protected TDataSink toThrift() {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/Planner.java
----------------------------------------------------------------------
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 8842c9c..ad5bba5 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -40,6 +40,7 @@ import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TQueryCtx;
 import org.apache.impala.thrift.TQueryExecRequest;
+import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TRuntimeFilterMode;
 import org.apache.impala.thrift.TTableName;
 import org.apache.impala.util.MaxRowsProcessedVisitor;
@@ -266,11 +267,14 @@ public class Planner {
       TQueryExecRequest request, TExplainLevel explainLevel) {
     StringBuilder str = new StringBuilder();
     boolean hasHeader = false;
-    if (request.isSetPer_host_mem_req() && request.isSetPer_host_vcores()) {
-      str.append(
-          String.format("Estimated Per-Host Requirements: Memory=%s VCores=%s\n",
-          PrintUtils.printBytes(request.getPer_host_mem_req()),
-          request.per_host_vcores));
+    if (request.isSetPer_host_min_reservation()) {
+      str.append(String.format("Per-Host Resource Reservation: Memory=%s\n",
+              PrintUtils.printBytes(request.getPer_host_min_reservation()))) ;
+      hasHeader = true;
+    }
+    if (request.isSetPer_host_mem_estimate()) {
+      str.append(String.format("Per-Host Resource Estimates: Memory=%s\n",
+          PrintUtils.printBytes(request.getPer_host_mem_estimate())));
       hasHeader = true;
     }
 
@@ -324,12 +328,12 @@ public class Planner {
 
     if (explainLevel.ordinal() < TExplainLevel.VERBOSE.ordinal()) {
       // Print the non-fragmented parallel plan.
-      str.append(fragments.get(0).getExplainString(explainLevel));
+      str.append(fragments.get(0).getExplainString(ctx_.getQueryOptions(), explainLevel));
     } else {
       // Print the fragmented parallel plan.
       for (int i = 0; i < fragments.size(); ++i) {
         PlanFragment fragment = fragments.get(i);
-        str.append(fragment.getExplainString(explainLevel));
+        str.append(fragment.getExplainString(ctx_.getQueryOptions(), explainLevel));
         if (i < fragments.size() - 1) str.append("\n");
       }
     }
@@ -337,91 +341,46 @@ public class Planner {
   }
 
   /**
-   * Returns true if the fragments are for a trivial, coordinator-only query:
-   * Case 1: Only an EmptySetNode, e.g. query has a limit 0.
-   * Case 2: Query has only constant exprs.
-   */
-  private static boolean isTrivialCoordOnlyPlan(List<PlanFragment> fragments) {
-    Preconditions.checkNotNull(fragments);
-    Preconditions.checkState(!fragments.isEmpty());
-    if (fragments.size() > 1) return false;
-    PlanNode root = fragments.get(0).getPlanRoot();
-    if (root instanceof EmptySetNode) return true;
-    if (root instanceof UnionNode && ((UnionNode) root).isConstantUnion()) return true;
-    return false;
-  }
-
-  /**
-   * Estimates the per-host memory and CPU requirements for the given plan fragments,
-   * and sets the results in request.
-   * Optionally excludes the requirements for unpartitioned fragments.
+   * Estimates the per-host resource requirements for the given plans, and sets the
+   * results in request.
    * TODO: The LOG.warn() messages should eventually become Preconditions checks
    * once resource estimation is more robust.
-   * TODO: Revisit and possibly remove during MT work, particularly references to vcores.
    */
-  public void computeResourceReqs(List<PlanFragment> fragments,
-      boolean excludeUnpartitionedFragments,
+  public void computeResourceReqs(List<PlanFragment> planRoots,
       TQueryExecRequest request) {
-    Preconditions.checkState(!fragments.isEmpty());
+    Preconditions.checkState(!planRoots.isEmpty());
     Preconditions.checkNotNull(request);
 
-    // Compute pipelined plan node sets.
-    ArrayList<PipelinedPlanNodeSet> planNodeSets =
-        PipelinedPlanNodeSet.computePlanNodeSets(fragments.get(0).getPlanRoot());
-
-    // Compute the max of the per-host mem and vcores requirement.
-    // Note that the max mem and vcores may come from different plan node sets.
-    long maxPerHostMem = Long.MIN_VALUE;
-    int maxPerHostVcores = Integer.MIN_VALUE;
-    for (PipelinedPlanNodeSet planNodeSet: planNodeSets) {
-      if (!planNodeSet.computeResourceEstimates(
-          excludeUnpartitionedFragments, ctx_.getQueryOptions())) {
-        continue;
-      }
-      long perHostMem = planNodeSet.getPerHostMem();
-      int perHostVcores = planNodeSet.getPerHostVcores();
-      if (perHostMem > maxPerHostMem) maxPerHostMem = perHostMem;
-      if (perHostVcores > maxPerHostVcores) maxPerHostVcores = perHostVcores;
-    }
-
-    // Do not ask for more cores than are in the RuntimeEnv.
-    maxPerHostVcores = Math.min(maxPerHostVcores, RuntimeEnv.INSTANCE.getNumCores());
-
-    // Special case for some trivial coordinator-only queries (IMPALA-3053, IMPALA-1092).
-    if (isTrivialCoordOnlyPlan(fragments)) {
-      maxPerHostMem = 1024;
-      maxPerHostVcores = 1;
-    }
-
-    // Set costs to zero if there are only unpartitioned fragments and
-    // excludeUnpartitionedFragments is true.
-    // TODO: handle this case with a better indication for unknown, e.g. -1 or not set.
-    if (maxPerHostMem == Long.MIN_VALUE || maxPerHostVcores == Integer.MIN_VALUE) {
-      boolean allUnpartitioned = true;
-      for (PlanFragment fragment: fragments) {
-        if (fragment.isPartitioned()) {
-          allUnpartitioned = false;
-          break;
-        }
+    // Compute the sum over all plans.
+    // TODO: Revisit during MT work - scheduling of fragments will change and computing
+    // the sum may not be correct or optimal.
+    ResourceProfile totalResources = ResourceProfile.invalid();
+    for (PlanFragment planRoot: planRoots) {
+      ResourceProfile planMaxResources = ResourceProfile.invalid();
+      ArrayList<PlanFragment> fragments = planRoot.getNodesPreOrder();
+      // Compute pipelined plan node sets.
+      ArrayList<PipelinedPlanNodeSet> planNodeSets =
+          PipelinedPlanNodeSet.computePlanNodeSets(fragments.get(0).getPlanRoot());
+
+      // Compute the max of the per-host resources requirement.
+      // Note that the different maxes may come from different plan node sets.
+      for (PipelinedPlanNodeSet planNodeSet : planNodeSets) {
+        TQueryOptions queryOptions = ctx_.getQueryOptions();
+        ResourceProfile perHostResources =
+            planNodeSet.computePerHostResources(queryOptions);
+        if (!perHostResources.isValid()) continue;
+        planMaxResources = ResourceProfile.max(planMaxResources, perHostResources);
       }
-      if (allUnpartitioned && excludeUnpartitionedFragments) {
-        maxPerHostMem = 0;
-        maxPerHostVcores = 0;
-      }
-    }
-
-    if (maxPerHostMem < 0 || maxPerHostMem == Long.MIN_VALUE) {
-      LOG.warn("Invalid per-host memory requirement: " + maxPerHostMem);
-    }
-    if (maxPerHostVcores < 0 || maxPerHostVcores == Integer.MIN_VALUE) {
-      LOG.warn("Invalid per-host virtual cores requirement: " + maxPerHostVcores);
+      totalResources = ResourceProfile.sum(totalResources, planMaxResources);
     }
-    request.setPer_host_mem_req(maxPerHostMem);
-    request.setPer_host_vcores((short) maxPerHostVcores);
 
+    Preconditions.checkState(totalResources.getMemEstimateBytes() >= 0);
+    Preconditions.checkState(totalResources.getMinReservationBytes() >= 0);
+    request.setPer_host_mem_estimate(totalResources.getMemEstimateBytes());
+    request.setPer_host_min_reservation(totalResources.getMinReservationBytes());
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Estimated per-host peak memory requirement: " + maxPerHostMem);
-      LOG.trace("Estimated per-host virtual cores requirement: " + maxPerHostVcores);
+      LOG.trace("Per-host min buffer : " + totalResources.getMinReservationBytes());
+      LOG.trace("Estimated per-host memory: " + totalResources.getMemEstimateBytes());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java b/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java
new file mode 100644
index 0000000..c0dc607
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/ResourceProfile.java
@@ -0,0 +1,83 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.planner;
+
+import org.apache.impala.common.PrintUtils;
+
+/**
+ * The resources that will be consumed by a set of plan nodes.
+ */
+public class ResourceProfile {
+  // If the computed values are valid.
+  private final boolean isValid_;
+
+  // Estimated memory consumption in bytes.
+  // TODO: IMPALA-5013: currently we are inconsistent about how these estimates are
+  // derived or what they mean. Re-evaluate what they mean and either deprecate or
+  // fix them.
+  private final long memEstimateBytes_;
+
+  // Minimum buffer reservation required to execute in bytes.
+  private final long minReservationBytes_;
+
+  private ResourceProfile(boolean isValid, long memEstimateBytes, long minReservationBytes) {
+    isValid_ = isValid;
+    memEstimateBytes_ = memEstimateBytes;
+    minReservationBytes_ = minReservationBytes;
+  }
+
+  public ResourceProfile(long memEstimateBytes, long minReservationBytes) {
+    this(true, memEstimateBytes, minReservationBytes);
+  }
+
+  public static ResourceProfile invalid() {
+    return new ResourceProfile(false, -1, -1);
+  }
+
+  public boolean isValid() { return isValid_; }
+  public long getMemEstimateBytes() { return memEstimateBytes_; }
+  public long getMinReservationBytes() { return minReservationBytes_; }
+
+  // Return a string with the resource profile information suitable for display in an
+  // explain plan in a format like: "resource1=value resource2=value"
+  public String getExplainString() {
+    StringBuilder output = new StringBuilder();
+    output.append("mem-estimate=");
+    output.append(isValid_ ? PrintUtils.printBytes(memEstimateBytes_) : "invalid");
+    output.append(" mem-reservation=");
+    output.append(isValid_ ? PrintUtils.printBytes(minReservationBytes_) : "invalid");
+    return output.toString();
+  }
+
+  // Returns a profile with the max of each value in 'p1' and 'p2'.
+  public static ResourceProfile max(ResourceProfile p1, ResourceProfile p2) {
+    if (!p1.isValid()) return p2;
+    if (!p2.isValid()) return p1;
+    return new ResourceProfile(
+        Math.max(p1.getMemEstimateBytes(), p2.getMemEstimateBytes()),
+        Math.max(p1.getMinReservationBytes(), p2.getMinReservationBytes()));
+  }
+
+  // Returns a profile with the sum of each value in 'p1' and 'p2'.
+  public static ResourceProfile sum(ResourceProfile p1, ResourceProfile p2) {
+    if (!p1.isValid()) return p2;
+    if (!p2.isValid()) return p1;
+    return new ResourceProfile(p1.getMemEstimateBytes() + p2.getMemEstimateBytes(),
+        p1.getMinReservationBytes() + p2.getMinReservationBytes());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/SelectNode.java
----------------------------------------------------------------------
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 e09d572..c346df9 100644
--- a/fe/src/main/java/org/apache/impala/planner/SelectNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SelectNode.java
@@ -27,6 +27,8 @@ import org.apache.impala.analysis.Expr;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
+import org.apache.impala.thrift.TQueryOptions;
+
 import com.google.common.base.Preconditions;
 
 /**
@@ -80,6 +82,12 @@ public class SelectNode extends PlanNode {
   }
 
   @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add an estimate
+    resourceProfile_ = new ResourceProfile(0, 0);
+  }
+
+  @Override
   protected String getNodeExplainString(String prefix, String detailPrefix,
       TExplainLevel detailLevel) {
     StringBuilder output = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java b/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
index 5b66d18..82a1c41 100644
--- a/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SingularRowSrcNode.java
@@ -22,6 +22,8 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
+import org.apache.impala.thrift.TQueryOptions;
+
 import com.google.common.base.Preconditions;
 
 /**
@@ -64,6 +66,12 @@ public class SingularRowSrcNode extends PlanNode {
   }
 
   @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add an estimate
+    resourceProfile_ = new ResourceProfile(0, 0);
+  }
+
+  @Override
   protected String getNodeExplainString(String prefix, String detailPrefix,
       TExplainLevel detailLevel) {
     StringBuilder output = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/SortNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/SortNode.java b/fe/src/main/java/org/apache/impala/planner/SortNode.java
index 0533b22..ef05499 100644
--- a/fe/src/main/java/org/apache/impala/planner/SortNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SortNode.java
@@ -205,10 +205,12 @@ public class SortNode extends PlanNode {
   }
 
   @Override
-  public void computeCosts(TQueryOptions queryOptions) {
+  public void computeResourceProfile(TQueryOptions queryOptions) {
     Preconditions.checkState(hasValidStats());
     if (useTopN_) {
-      perHostMemCost_ = (long) Math.ceil((cardinality_ + offset_) * avgRowSize_);
+      long perInstanceMemEstimate =
+              (long) Math.ceil((cardinality_ + offset_) * avgRowSize_);
+      resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0);
       return;
     }
 
@@ -233,7 +235,15 @@ public class SortNode extends PlanNode {
     // doubles the block size when there are var-len columns present.
     if (hasVarLenSlots) blockSize *= 2;
     double numInputBlocks = Math.ceil(fullInputSize / blockSize);
-    perHostMemCost_ = blockSize * (long) Math.ceil(Math.sqrt(numInputBlocks));
+    long perInstanceMemEstimate = blockSize * (long) Math.ceil(Math.sqrt(numInputBlocks));
+
+    // Must be kept in sync with min_buffers_required in Sorter in be.
+    long perInstanceMinReservation = 3 * SPILLABLE_BUFFER_BYTES;
+    if (info_.getSortTupleDescriptor().hasVarLenSlots()) {
+      perInstanceMinReservation *= 2;
+    }
+    resourceProfile_ =
+        new ResourceProfile(perInstanceMemEstimate, perInstanceMinReservation);
   }
 
   private static String getDisplayName(boolean isTopN, boolean isMergeOnly) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
----------------------------------------------------------------------
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 6143255..cbe7087 100644
--- a/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SubplanNode.java
@@ -22,6 +22,8 @@ import org.apache.impala.common.InternalException;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
+import org.apache.impala.thrift.TQueryOptions;
+
 import com.google.common.base.Preconditions;
 
 /**
@@ -91,6 +93,12 @@ public class SubplanNode extends PlanNode {
   }
 
   @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add an estimate
+    resourceProfile_ = new ResourceProfile(0, 0);
+  }
+
+  @Override
   protected String getNodeExplainString(String prefix, String detailPrefix,
       TExplainLevel detailLevel) {
     StringBuilder output = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/UnionNode.java
----------------------------------------------------------------------
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 d724c59..6b8d331 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnionNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
@@ -30,6 +30,7 @@ import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TExpr;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
+import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TUnionNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -127,6 +128,12 @@ public class UnionNode extends PlanNode {
     }
   }
 
+  @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add an estimate
+    resourceProfile_ = new ResourceProfile(0, 0);
+  }
+
   /**
    * Returns true if rows from the child with 'childTupleIds' and 'childResultExprs' can
    * be returned directly by the union node (without materialization into a new tuple).

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
----------------------------------------------------------------------
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 35abc55..5847e62 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnnestNode.java
@@ -24,6 +24,7 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
+import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TUnnestNode;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -73,6 +74,12 @@ public class UnnestNode extends PlanNode {
   }
 
   @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add an estimate
+    resourceProfile_ = new ResourceProfile(0, 0);
+  }
+
+  @Override
   protected String getNodeExplainString(String prefix, String detailPrefix,
       TExplainLevel detailLevel) {
     StringBuilder output = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index d3cefa6..1348129 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -962,6 +962,9 @@ public class Frontend {
       }
     }
 
+    // Clear pre-existing lists to avoid adding duplicate entries in FE tests.
+    queryCtx.unsetTables_missing_stats();
+    queryCtx.unsetTables_with_corrupt_stats();
     for (TTableName tableName: tablesMissingStats) {
       queryCtx.addToTables_missing_stats(tableName);
     }
@@ -972,16 +975,6 @@ public class Frontend {
       queryCtx.addToTables_missing_diskids(tableName);
     }
 
-    // Compute resource requirements after scan range locations because the cost
-    // estimates of scan nodes rely on them.
-    try {
-      planner.computeResourceReqs(fragments, true, queryExecRequest);
-    } catch (Exception e) {
-      // Turn exceptions into a warning to allow the query to execute.
-      LOG.error("Failed to compute resource requirements for query\n" +
-          queryCtx.client_request.getStmt(), e);
-    }
-
     // The fragment at this point has all state set, serialize it to thrift.
     for (PlanFragment fragment: fragments) {
       TPlanFragment thriftFragment = fragment.toThrift();
@@ -1020,6 +1013,10 @@ public class Frontend {
           createPlanExecInfo(planRoot, planner, queryCtx, result));
     }
 
+    // Compute resource requirements after scan range locations because the cost
+    // estimates of scan nodes rely on them.
+    planner.computeResourceReqs(planRoots, result);
+
     // Optionally disable spilling in the backend. Allow spilling if there are plan hints
     // or if all tables have stats.
     boolean disableSpilling =

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index 5148f68..363c59c 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -369,4 +369,14 @@ public class PlannerTest extends PlannerTestBase {
     // Check that the effective MT_DOP is as expected.
     Assert.assertEquals(actualMtDop, expectedMtDop);
   }
+
+  @Test
+  public void testResourceRequirements() {
+    // Tests the resource requirement computation from the planner.
+    TQueryOptions options = defaultQueryOptions();
+    options.setExplain_level(TExplainLevel.EXTENDED);
+    options.setNum_scanner_threads(1); // Required so that output doesn't vary by machine
+    runPlannerTestFile("resource-requirements", options, false);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java b/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
index d354897..eceaeca 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
@@ -390,7 +390,8 @@ public class PlannerTestBase extends FrontendTestBase {
    * of 'testCase'.
    */
   private void runTestCase(TestCase testCase, StringBuilder errorLog,
-      StringBuilder actualOutput, String dbName, TQueryOptions options)
+      StringBuilder actualOutput, String dbName, TQueryOptions options,
+      boolean ignoreExplainHeader)
       throws CatalogException {
     if (options == null) {
       options = defaultQueryOptions();
@@ -408,16 +409,18 @@ public class PlannerTestBase extends FrontendTestBase {
         dbName, System.getProperty("user.name"));
     queryCtx.client_request.query_options = options;
     // Test single node plan, scan range locations, and column lineage.
-    TExecRequest singleNodeExecRequest =
-        testPlan(testCase, Section.PLAN, queryCtx, errorLog, actualOutput);
+    TExecRequest singleNodeExecRequest = testPlan(testCase, Section.PLAN, queryCtx,
+        ignoreExplainHeader, errorLog, actualOutput);
     validateTableIds(singleNodeExecRequest);
     checkScanRangeLocations(testCase, singleNodeExecRequest, errorLog, actualOutput);
     checkColumnLineage(testCase, singleNodeExecRequest, errorLog, actualOutput);
     checkLimitCardinality(query, singleNodeExecRequest, errorLog);
     // Test distributed plan.
-    testPlan(testCase, Section.DISTRIBUTEDPLAN, queryCtx, errorLog, actualOutput);
+    testPlan(testCase, Section.DISTRIBUTEDPLAN, queryCtx, ignoreExplainHeader, errorLog,
+        actualOutput);
     // test parallel plans
-    testPlan(testCase, Section.PARALLELPLANS, queryCtx, errorLog, actualOutput);
+    testPlan(testCase, Section.PARALLELPLANS, queryCtx, ignoreExplainHeader, errorLog,
+        actualOutput);
   }
 
   /**
@@ -471,19 +474,26 @@ public class PlannerTestBase extends FrontendTestBase {
    *
    * Returns the produced exec request or null if there was an error generating
    * the plan.
+   *
+   * If ignoreExplainHeader is true, the explain header with warnings and resource
+   * estimates is stripped out.
    */
   private TExecRequest testPlan(TestCase testCase, Section section,
-      TQueryCtx queryCtx, StringBuilder errorLog, StringBuilder actualOutput) {
+      TQueryCtx queryCtx, boolean ignoreExplainHeader,
+      StringBuilder errorLog, StringBuilder actualOutput) {
     String query = testCase.getQuery();
     queryCtx.client_request.setStmt(query);
+    TQueryOptions queryOptions = queryCtx.client_request.getQuery_options();
     if (section == Section.PLAN) {
-      queryCtx.client_request.getQuery_options().setNum_nodes(1);
+      queryOptions.setNum_nodes(1);
     } else {
       // for distributed and parallel execution we want to run on all available nodes
-      queryCtx.client_request.getQuery_options().setNum_nodes(
+      queryOptions.setNum_nodes(
           ImpalaInternalServiceConstants.NUM_NODES_ALL);
     }
-    if (section == Section.PARALLELPLANS) {
+    if (section == Section.PARALLELPLANS
+        && (!queryOptions.isSetMt_dop() || queryOptions.getMt_dop() == 0)) {
+      // Set mt_dop to force production of parallel plans.
       queryCtx.client_request.query_options.setMt_dop(2);
     }
     ArrayList<String> expectedPlan = testCase.getSectionContents(section);
@@ -507,7 +517,8 @@ public class PlannerTestBase extends FrontendTestBase {
     // Failed to produce an exec request.
     if (execRequest == null) return null;
 
-    String explainStr = removeExplainHeader(explainBuilder.toString());
+    String explainStr = explainBuilder.toString();
+    if (ignoreExplainHeader) explainStr = removeExplainHeader(explainStr);
     actualOutput.append(explainStr);
     LOG.info(section.toString() + ":" + explainStr);
     if (expectedErrorMsg != null) {
@@ -702,10 +713,16 @@ public class PlannerTestBase extends FrontendTestBase {
   }
 
   protected void runPlannerTestFile(String testFile, TQueryOptions options) {
-    runPlannerTestFile(testFile, "default", options);
+    runPlannerTestFile(testFile, options, true);
+  }
+
+  protected void runPlannerTestFile(String testFile, TQueryOptions options,
+      boolean ignoreExplainHeader) {
+    runPlannerTestFile(testFile, "default", options, ignoreExplainHeader);
   }
 
-  private void runPlannerTestFile(String testFile, String dbName, TQueryOptions options) {
+  private void runPlannerTestFile(String testFile, String dbName, TQueryOptions options,
+      boolean ignoreExplainHeader) {
     String fileName = testDir_ + "/" + testFile + ".test";
     TestFileParser queryFileParser = new TestFileParser(fileName);
     StringBuilder actualOutput = new StringBuilder();
@@ -716,7 +733,8 @@ public class PlannerTestBase extends FrontendTestBase {
       actualOutput.append(testCase.getSectionAsString(Section.QUERY, true, "\n"));
       actualOutput.append("\n");
       try {
-        runTestCase(testCase, errorLog, actualOutput, dbName, options);
+        runTestCase(testCase, errorLog, actualOutput, dbName, options,
+                ignoreExplainHeader);
       } catch (CatalogException e) {
         errorLog.append(String.format("Failed to plan query\n%s\n%s",
             testCase.getQuery(), e.getMessage()));
@@ -743,10 +761,10 @@ public class PlannerTestBase extends FrontendTestBase {
   }
 
   protected void runPlannerTestFile(String testFile) {
-    runPlannerTestFile(testFile, "default", null);
+    runPlannerTestFile(testFile, "default", null, true);
   }
 
   protected void runPlannerTestFile(String testFile, String dbName) {
-    runPlannerTestFile(testFile, dbName, null);
+    runPlannerTestFile(testFile, dbName, null, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
index 3a9855d..7effd9b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
@@ -4,42 +4,44 @@ from tpch_nested_parquet.customer c, c.c_orders o, o.o_lineitems
 where 5 + 5 < c_custkey and o_orderkey = (2 + 2)
   and (coalesce(2, 3, 4) * 10) + l_linenumber < (0 * 1)
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 01:SUBPLAN
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=2,1,0 row-size=52B cardinality=1500000
 |
 |--08:NESTED LOOP JOIN [CROSS JOIN]
-|  |  hosts=3 per-host-mem=unavailable
+|  |  mem-estimate=24B mem-reservation=0B
 |  |  tuple-ids=2,1,0 row-size=52B cardinality=100
 |  |
 |  |--02:SINGULAR ROW SRC
 |  |     parent-subplan=01
-|  |     hosts=3 per-host-mem=unavailable
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=0 row-size=24B cardinality=1
 |  |
 |  04:SUBPLAN
-|  |  hosts=3 per-host-mem=unavailable
+|  |  mem-estimate=0B mem-reservation=0B
 |  |  tuple-ids=2,1 row-size=28B cardinality=100
 |  |
 |  |--07:NESTED LOOP JOIN [CROSS JOIN]
-|  |  |  hosts=3 per-host-mem=unavailable
+|  |  |  mem-estimate=24B mem-reservation=0B
 |  |  |  tuple-ids=2,1 row-size=28B cardinality=10
 |  |  |
 |  |  |--05:SINGULAR ROW SRC
 |  |  |     parent-subplan=04
-|  |  |     hosts=3 per-host-mem=unavailable
+|  |  |     mem-estimate=0B mem-reservation=0B
 |  |  |     tuple-ids=1 row-size=24B cardinality=1
 |  |  |
 |  |  06:UNNEST [o.o_lineitems]
 |  |     parent-subplan=04
-|  |     hosts=3 per-host-mem=unavailable
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=2 row-size=0B cardinality=10
 |  |
 |  03:UNNEST [c.c_orders o]
 |     parent-subplan=01
-|     hosts=3 per-host-mem=unavailable
+|     mem-estimate=0B mem-reservation=0B
 |     tuple-ids=1 row-size=0B cardinality=10
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
@@ -51,7 +53,7 @@ PLAN-ROOT SINK
    columns missing stats: c_orders
    parquet statistics predicates: c_custkey > 10
    parquet dictionary predicates: c_custkey > 10
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=176.00MB mem-reservation=0B
    tuple-ids=0 row-size=24B cardinality=15000
 ====
 # Test HBase scan node.
@@ -59,7 +61,9 @@ select * from functional_hbase.stringids
 where string_col = cast(4 as string) and 2 + 3 = tinyint_col
   and id between concat('1', '0') and upper('20')
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 10
@@ -68,19 +72,21 @@ PLAN-ROOT SINK
    predicates: tinyint_col = 5, string_col = '4'
    table stats: 10000 rows total
    column stats: all
-   hosts=100 per-host-mem=unavailable
+   mem-estimate=1.00GB mem-reservation=0B
    tuple-ids=0 row-size=119B cardinality=1
 ====
 # Test datasource scan node.
 select * from functional.alltypes_datasource
 where tinyint_col < (pow(2, 8)) and float_col != 0 and 1 + 1 > int_col
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
 data source predicates: tinyint_col < 256, int_col < 2
 predicates: float_col != 0
-   hosts=1 per-host-mem=unavailable
+   mem-estimate=1.00GB mem-reservation=0B
    tuple-ids=0 row-size=116B cardinality=500
 ====
 # Test aggregation.
@@ -91,20 +97,22 @@ having 1024 * 1024 * count(*) % 2 = 0
   and (sm > 1 or sm > 1)
   and (sm between 5 and 10)
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 01:AGGREGATE [FINALIZE]
 |  output: sum(2 + id), count(*)
 |  group by: timestamp_col = TIMESTAMP '2016-11-15 00:00:00'
 |  having: sum(2 + id) <= 10, sum(2 + id) > 1, sum(2 + id) >= 5, 1048576 * count(*) % 2 = 0
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=10.00MB mem-reservation=264.00MB
 |  tuple-ids=1 row-size=17B cardinality=0
 |
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
    table stats: 7300 rows total
    column stats: all
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=20B cardinality=7300
 ====
 # Test hash join.
@@ -114,13 +122,15 @@ left outer join functional.alltypes b
       a.int_col between 0 + 0 + 0 + b.bigint_col and b.bigint_col + ascii('a'))
 where round(1.11 + 2.22 + 3.33 + 4.44, 1) < cast(b.double_col as decimal(3, 2))
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: 2 + a.id = b.id - 2
 |  other join predicates: a.int_col <= b.bigint_col + 97, a.int_col >= 0 + b.bigint_col
 |  other predicates: CAST(b.double_col AS DECIMAL(3,2)) > 11.1
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=15.68KB mem-reservation=136.00MB
 |  tuple-ids=0,1N row-size=28B cardinality=7300
 |
 |--01:SCAN HDFS [functional.alltypes b]
@@ -129,14 +139,14 @@ PLAN-ROOT SINK
 |     table stats: 7300 rows total
 |     column stats: all
 |     parquet dictionary predicates: CAST(b.double_col AS DECIMAL(3,2)) > 11.1
-|     hosts=3 per-host-mem=unavailable
+|     mem-estimate=128.00MB mem-reservation=0B
 |     tuple-ids=1 row-size=20B cardinality=730
 |
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    table stats: 7300 rows total
    column stats: all
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=8B cardinality=7300
 ====
 # Test nested-loop join. Same as above but and with a disjunction in the On clause.
@@ -147,12 +157,14 @@ left outer join functional.alltypes b
       a.int_col between 0 + 0 + 0 + b.bigint_col and b.bigint_col + ascii('a'))
 where cast(b.double_col as decimal(3, 2)) > round(1.11 + 2.22 + 3.33 + 4.44, 1)
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 02:NESTED LOOP JOIN [LEFT OUTER JOIN]
 |  join predicates: (2 + a.id = b.id - 2 OR a.int_col >= 0 + b.bigint_col AND a.int_col <= b.bigint_col + 97)
 |  predicates: CAST(b.double_col AS DECIMAL(3,2)) > 11.1
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=14.26KB mem-reservation=0B
 |  tuple-ids=0,1N row-size=28B cardinality=7300
 |
 |--01:SCAN HDFS [functional.alltypes b]
@@ -161,14 +173,14 @@ PLAN-ROOT SINK
 |     table stats: 7300 rows total
 |     column stats: all
 |     parquet dictionary predicates: CAST(b.double_col AS DECIMAL(3,2)) > 11.1
-|     hosts=3 per-host-mem=unavailable
+|     mem-estimate=128.00MB mem-reservation=0B
 |     tuple-ids=1 row-size=20B cardinality=730
 |
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    table stats: 7300 rows total
    column stats: all
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=8B cardinality=7300
 ====
 # Test distinct aggregation with grouping.
@@ -177,26 +189,28 @@ from functional.alltypes
 group by timestamp_col = cast('2015-11-15' as timestamp) + interval 1 year
 having 1024 * 1024 * count(*) % 2 = 0
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 02:AGGREGATE [FINALIZE]
 |  output: sum(2 + id), count:merge(*)
 |  group by: timestamp_col = TIMESTAMP '2016-11-15 00:00:00'
 |  having: 1048576 * count(*) % 2 = 0
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=10.00MB mem-reservation=264.00MB
 |  tuple-ids=2 row-size=17B cardinality=0
 |
 01:AGGREGATE
 |  output: count(*)
 |  group by: timestamp_col = TIMESTAMP '2016-11-15 00:00:00', 2 + id
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=10.00MB mem-reservation=264.00MB
 |  tuple-ids=1 row-size=17B cardinality=7300
 |
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
    table stats: 7300 rows total
    column stats: all
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=20B cardinality=7300
 ====
 # Test non-grouping distinct aggregation.
@@ -204,25 +218,27 @@ select sum(distinct 1 + 1 + id)
 from functional.alltypes
 having 1024 * 1024 * count(*) % 2 = 0
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 02:AGGREGATE [FINALIZE]
 |  output: sum(2 + id), count:merge(*)
 |  having: 1048576 * zeroifnull(count(*)) % 2 = 0
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=10.00MB mem-reservation=0B
 |  tuple-ids=2 row-size=16B cardinality=0
 |
 01:AGGREGATE
 |  output: count(*)
 |  group by: 2 + id
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=10.00MB mem-reservation=264.00MB
 |  tuple-ids=1 row-size=16B cardinality=7300
 |
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
    table stats: 7300 rows total
    column stats: all
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=4B cardinality=7300
 ====
 # Test analytic eval node.
@@ -231,44 +247,48 @@ select first_value(1 + 1 + int_col - (1 - 1)) over
    order by greatest(greatest(10, 20), bigint_col))
 from functional.alltypes
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 02:ANALYTIC
 |  functions: first_value(2 + int_col - 0)
 |  partition by: concat('ab', string_col)
 |  order by: greatest(20, bigint_col) ASC
 |  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=16.00MB
 |  tuple-ids=3,2 row-size=37B cardinality=7300
 |
 01:SORT
 |  order by: concat('ab', string_col) ASC NULLS FIRST, greatest(20, bigint_col) ASC
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=16.00MB mem-reservation=48.00MB
 |  tuple-ids=3 row-size=29B cardinality=7300
 |
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
    table stats: 7300 rows total
    column stats: all
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=29B cardinality=7300
 ====
 # Test sort node.
 select int_col from functional.alltypes
 order by id * abs((factorial(5) / power(2, 4)))
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 01:SORT
 |  order by: id * 7.5 ASC
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=8.00MB mem-reservation=24.00MB
 |  tuple-ids=1 row-size=8B cardinality=7300
 |
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
    table stats: 7300 rows total
    column stats: all
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=8B cardinality=7300
 ====
 # Test HDFS table sink.
@@ -276,15 +296,16 @@ insert into functional.alltypes (id, int_col) partition(year,month)
 select id, int_col, cast(1 + 1 + 1 + year as int), cast(month - (1 - 1 - 1) as int)
 from functional.alltypessmall
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(CAST(3 + year AS INT),CAST(month - -1 AS INT))]
 |  partitions=4
-|  hosts=1 per-host-mem=unavailable
+|  mem-estimate=1.56KB mem-reservation=0B
 |
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
    table stats: 100 rows total
    column stats: all
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=32.00MB mem-reservation=0B
    tuple-ids=0 row-size=16B cardinality=100
 ====
 # Constant folding does not work across query blocks.
@@ -295,11 +316,13 @@ select sum(id + c3) from
     ) v2
   ) v3
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 01:AGGREGATE [FINALIZE]
 |  output: sum(id + 10 + 20 + 30)
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=10.00MB mem-reservation=0B
 |  tuple-ids=4 row-size=8B cardinality=1
 |
 00:SCAN HDFS [functional.alltypes]
@@ -307,6 +330,6 @@ PLAN-ROOT SINK
    table stats: 7300 rows total
    column stats: all
    limit: 2
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=4B cardinality=2
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
index b064d2b..bad3299 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
@@ -1,100 +1,112 @@
 select * from functional_kudu.zipcode_incomes where id = '8600000US00601'
 ---- PLAN
-F00:PLAN FRAGMENT [UNPARTITIONED]
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      kudu predicates: id = '8600000US00601'
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=1
 ---- DISTRIBUTEDPLAN
-F01:PLAN FRAGMENT [UNPARTITIONED]
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   01:EXCHANGE [UNPARTITIONED]
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=1
 
-F00:PLAN FRAGMENT [RANDOM]
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
   DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
+  |  mem-estimate=0B mem-reservation=0B
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      kudu predicates: id = '8600000US00601'
-     hosts=3 per-host-mem=0B
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=1
 ====
 # The cardinality from "zip = '2'" should dominate.
 select * from functional_kudu.zipcode_incomes where id != '1' and zip = '2'
 ---- PLAN
-F00:PLAN FRAGMENT [UNPARTITIONED]
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      predicates: id != '1'
      kudu predicates: zip = '2'
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=1
 ---- DISTRIBUTEDPLAN
-F01:PLAN FRAGMENT [UNPARTITIONED]
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   01:EXCHANGE [UNPARTITIONED]
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=1
 
-F00:PLAN FRAGMENT [RANDOM]
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
   DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
+  |  mem-estimate=0B mem-reservation=0B
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      predicates: id != '1'
      kudu predicates: zip = '2'
-     hosts=3 per-host-mem=0B
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=1
 ====
 select * from functional_kudu.zipcode_incomes where id > '1' and zip > '2'
 ---- PLAN
-F00:PLAN FRAGMENT [UNPARTITIONED]
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      kudu predicates: zip > '2', id > '1'
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=3317
 ---- DISTRIBUTEDPLAN
-F01:PLAN FRAGMENT [UNPARTITIONED]
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   01:EXCHANGE [UNPARTITIONED]
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=3317
 
-F00:PLAN FRAGMENT [RANDOM]
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
   DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
+  |  mem-estimate=0B mem-reservation=0B
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      kudu predicates: zip > '2', id > '1'
-     hosts=3 per-host-mem=0B
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=3317
 ====
 select * from functional_kudu.zipcode_incomes where id = '1' or id = '2'
 ---- PLAN
-F00:PLAN FRAGMENT [UNPARTITIONED]
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      predicates: id = '1' OR id = '2'
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=2
 ---- DISTRIBUTEDPLAN
-F01:PLAN FRAGMENT [UNPARTITIONED]
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   01:EXCHANGE [UNPARTITIONED]
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=2
 
-F00:PLAN FRAGMENT [RANDOM]
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
   DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
+  |  mem-estimate=0B mem-reservation=0B
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      predicates: id = '1' OR id = '2'
-     hosts=3 per-host-mem=0B
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=124B cardinality=2
 ====
 select * from functional_kudu.alltypes where
@@ -121,13 +133,14 @@ double_col in (cast('inf' as double)) and
 string_col not in ("bar") and
 id in (int_col)
 ---- PLAN
-F00:PLAN FRAGMENT [UNPARTITIONED]
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   00:SCAN KUDU [functional_kudu.alltypes]
      predicates: id IN (int_col), string_col NOT IN ('bar'), bigint_col IN (9999999999999999999), double_col IN (CAST('inf' AS DOUBLE)), float_col IN (CAST('NaN' AS FLOAT)), int_col IN (9999999999), smallint_col IN (99999, 2), tinyint_col IN (1, 999), bool_col IN (1)
      kudu predicates: double_col IN (0.0), float_col IN (0.0), bigint_col IN (1, 2), int_col IN (1, 2), smallint_col IN (0, 2), string_col IN ('foo', 'foo       '), tinyint_col IN (1, 2), bool_col IN (TRUE)
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=126B cardinality=4
 ====
 select * from functional_kudu.alltypes where
@@ -135,12 +148,13 @@ tinyint_col is not null and
 smallint_col is null and
 cast(date_string_col as tinyint) is null
 ---- PLAN
-F00:PLAN FRAGMENT [UNPARTITIONED]
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
   PLAN-ROOT SINK
+  |  mem-estimate=0B mem-reservation=0B
   |
   00:SCAN KUDU [functional_kudu.alltypes]
      predicates: CAST(date_string_col AS TINYINT) IS NULL
      kudu predicates: smallint_col IS NULL, tinyint_col IS NOT NULL
-     hosts=3 per-host-mem=unavailable
+     mem-estimate=0B mem-reservation=0B
      tuple-ids=0 row-size=126B cardinality=730
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
----------------------------------------------------------------------
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 4d9544d..e3dd297 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
@@ -39,17 +39,19 @@ group by bigint_col
 order by cnt, bigint_col
 limit 10
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 02:TOP-N [LIMIT=10]
 |  order by: count(int_col) ASC, bigint_col ASC
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=160B mem-reservation=0B
 |  tuple-ids=2 row-size=16B cardinality=10
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(int_col)
 |  group by: bigint_col
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=128.00MB mem-reservation=264.00MB
 |  tuple-ids=1 row-size=16B cardinality=unavailable
 |
 00:SCAN HDFS [functional_parquet.alltypes]
@@ -59,36 +61,40 @@ PLAN-ROOT SINK
    column stats: unavailable
    parquet statistics predicates: id < 10
    parquet dictionary predicates: id < 10
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=16.00MB mem-reservation=0B
    tuple-ids=0 row-size=16B cardinality=unavailable
 ---- PARALLELPLANS
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(int_col) ASC, bigint_col ASC
 |  limit: 10
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=2 row-size=16B cardinality=10
 |
+F01:PLAN FRAGMENT [HASH(bigint_col)] hosts=3 instances=9
 02:TOP-N [LIMIT=10]
 |  order by: count(int_col) ASC, bigint_col ASC
-|  hosts=3 per-host-mem=160B
+|  mem-estimate=160B mem-reservation=0B
 |  tuple-ids=2 row-size=16B cardinality=10
 |
 04:AGGREGATE [FINALIZE]
 |  output: count:merge(int_col)
 |  group by: bigint_col
-|  hosts=3 per-host-mem=128.00MB
+|  mem-estimate=128.00MB mem-reservation=264.00MB
 |  tuple-ids=1 row-size=16B cardinality=unavailable
 |
 03:EXCHANGE [HASH(bigint_col)]
-|  hosts=3 per-host-mem=0B
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=1 row-size=16B cardinality=unavailable
 |
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
 01:AGGREGATE [STREAMING]
 |  output: count(int_col)
 |  group by: bigint_col
-|  hosts=3 per-host-mem=128.00MB
+|  mem-estimate=128.00MB mem-reservation=0B
 |  tuple-ids=1 row-size=16B cardinality=unavailable
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
@@ -98,27 +104,29 @@ PLAN-ROOT SINK
    column stats: unavailable
    parquet statistics predicates: id < 10
    parquet dictionary predicates: id < 10
-   hosts=3 per-host-mem=16.00MB
+   mem-estimate=16.00MB mem-reservation=0B
    tuple-ids=0 row-size=16B cardinality=unavailable
 ====
-# Single-table scan/filter/analysic should work.
+# Single-table scan/filter/analytic should work.
 select row_number() over(partition by int_col order by id)
 from functional_parquet.alltypes
 where id < 10
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 02:ANALYTIC
 |  functions: row_number()
 |  partition by: int_col
 |  order by: id ASC
 |  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=16.00MB
 |  tuple-ids=4,3 row-size=16B cardinality=unavailable
 |
 01:SORT
 |  order by: int_col ASC NULLS FIRST, id ASC
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=24.00MB
 |  tuple-ids=4 row-size=8B cardinality=unavailable
 |
 00:SCAN HDFS [functional_parquet.alltypes]
@@ -128,32 +136,36 @@ PLAN-ROOT SINK
    column stats: unavailable
    parquet statistics predicates: id < 10
    parquet dictionary predicates: id < 10
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=16.00MB mem-reservation=0B
    tuple-ids=0 row-size=8B cardinality=unavailable
 ---- PARALLELPLANS
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 04:EXCHANGE [UNPARTITIONED]
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=4,3 row-size=16B cardinality=unavailable
 |
+F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=9
 02:ANALYTIC
 |  functions: row_number()
 |  partition by: int_col
 |  order by: id ASC
 |  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
-|  hosts=3 per-host-mem=0B
+|  mem-estimate=0B mem-reservation=16.00MB
 |  tuple-ids=4,3 row-size=16B cardinality=unavailable
 |
 01:SORT
 |  order by: int_col ASC NULLS FIRST, id ASC
-|  hosts=3 per-host-mem=0B
+|  mem-estimate=0B mem-reservation=24.00MB
 |  tuple-ids=4 row-size=8B cardinality=unavailable
 |
 03:EXCHANGE [HASH(int_col)]
-|  hosts=3 per-host-mem=0B
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=0 row-size=8B cardinality=unavailable
 |
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
    partitions=24/24 files=24 size=156.57KB
    predicates: id < 10
@@ -161,7 +173,7 @@ PLAN-ROOT SINK
    column stats: unavailable
    parquet statistics predicates: id < 10
    parquet dictionary predicates: id < 10
-   hosts=3 per-host-mem=16.00MB
+   mem-estimate=16.00MB mem-reservation=0B
    tuple-ids=0 row-size=8B cardinality=unavailable
 ====
 # Nested-loop join in a subplan should work.
@@ -169,42 +181,44 @@ select *
 from tpch_nested_parquet.customer c, c.c_orders o, o.o_lineitems
 where c_custkey < 10 and o_orderkey < 5 and l_linenumber < 3
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 01:SUBPLAN
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=2,1,0 row-size=562B cardinality=1500000
 |
 |--08:NESTED LOOP JOIN [CROSS JOIN]
-|  |  hosts=3 per-host-mem=unavailable
+|  |  mem-estimate=254B mem-reservation=0B
 |  |  tuple-ids=2,1,0 row-size=562B cardinality=100
 |  |
 |  |--02:SINGULAR ROW SRC
 |  |     parent-subplan=01
-|  |     hosts=3 per-host-mem=unavailable
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=0 row-size=254B cardinality=1
 |  |
 |  04:SUBPLAN
-|  |  hosts=3 per-host-mem=unavailable
+|  |  mem-estimate=0B mem-reservation=0B
 |  |  tuple-ids=2,1 row-size=308B cardinality=100
 |  |
 |  |--07:NESTED LOOP JOIN [CROSS JOIN]
-|  |  |  hosts=3 per-host-mem=unavailable
+|  |  |  mem-estimate=124B mem-reservation=0B
 |  |  |  tuple-ids=2,1 row-size=308B cardinality=10
 |  |  |
 |  |  |--05:SINGULAR ROW SRC
 |  |  |     parent-subplan=04
-|  |  |     hosts=3 per-host-mem=unavailable
+|  |  |     mem-estimate=0B mem-reservation=0B
 |  |  |     tuple-ids=1 row-size=124B cardinality=1
 |  |  |
 |  |  06:UNNEST [o.o_lineitems]
 |  |     parent-subplan=04
-|  |     hosts=3 per-host-mem=unavailable
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=2 row-size=0B cardinality=10
 |  |
 |  03:UNNEST [c.c_orders o]
 |     parent-subplan=01
-|     hosts=3 per-host-mem=unavailable
+|     mem-estimate=0B mem-reservation=0B
 |     tuple-ids=1 row-size=0B cardinality=10
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
@@ -216,49 +230,52 @@ PLAN-ROOT SINK
    columns missing stats: c_orders
    parquet statistics predicates: c_custkey < 10
    parquet dictionary predicates: c_custkey < 10
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=88.00MB mem-reservation=0B
    tuple-ids=0 row-size=254B cardinality=15000
 ---- PARALLELPLANS
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 09:EXCHANGE [UNPARTITIONED]
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=2,1,0 row-size=562B cardinality=1500000
 |
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
 01:SUBPLAN
-|  hosts=3 per-host-mem=0B
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=2,1,0 row-size=562B cardinality=1500000
 |
 |--08:NESTED LOOP JOIN [CROSS JOIN]
-|  |  hosts=3 per-host-mem=254B
+|  |  mem-estimate=254B mem-reservation=0B
 |  |  tuple-ids=2,1,0 row-size=562B cardinality=100
 |  |
 |  |--02:SINGULAR ROW SRC
 |  |     parent-subplan=01
-|  |     hosts=3 per-host-mem=0B
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=0 row-size=254B cardinality=1
 |  |
 |  04:SUBPLAN
-|  |  hosts=3 per-host-mem=0B
+|  |  mem-estimate=0B mem-reservation=0B
 |  |  tuple-ids=2,1 row-size=308B cardinality=100
 |  |
 |  |--07:NESTED LOOP JOIN [CROSS JOIN]
-|  |  |  hosts=3 per-host-mem=124B
+|  |  |  mem-estimate=124B mem-reservation=0B
 |  |  |  tuple-ids=2,1 row-size=308B cardinality=10
 |  |  |
 |  |  |--05:SINGULAR ROW SRC
 |  |  |     parent-subplan=04
-|  |  |     hosts=3 per-host-mem=0B
+|  |  |     mem-estimate=0B mem-reservation=0B
 |  |  |     tuple-ids=1 row-size=124B cardinality=1
 |  |  |
 |  |  06:UNNEST [o.o_lineitems]
 |  |     parent-subplan=04
-|  |     hosts=3 per-host-mem=0B
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=2 row-size=0B cardinality=10
 |  |
 |  03:UNNEST [c.c_orders o]
 |     parent-subplan=01
-|     hosts=3 per-host-mem=0B
+|     mem-estimate=0B mem-reservation=0B
 |     tuple-ids=1 row-size=0B cardinality=10
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM]
@@ -270,7 +287,7 @@ PLAN-ROOT SINK
    columns missing stats: c_orders
    parquet statistics predicates: c_custkey < 10
    parquet dictionary predicates: c_custkey < 10
-   hosts=3 per-host-mem=88.00MB
+   mem-estimate=88.00MB mem-reservation=0B
    tuple-ids=0 row-size=254B cardinality=15000
 ====
 # Hash-join in a subplan should work.
@@ -278,34 +295,36 @@ select c.*
 from tpch_nested_parquet.customer c, c.c_orders o1, c.c_orders o2
 where o1.o_orderkey = o2.o_orderkey + 2 and o1.o_orderkey < 5
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 01:SUBPLAN
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=1,0,2 row-size=286B cardinality=1500000
 |
 |--06:HASH JOIN [INNER JOIN]
 |  |  hash predicates: o1.o_orderkey = o2.o_orderkey + 2
-|  |  hosts=3 per-host-mem=unavailable
+|  |  mem-estimate=0B mem-reservation=136.00MB
 |  |  tuple-ids=1,0,2 row-size=286B cardinality=10
 |  |
 |  |--04:UNNEST [c.c_orders o2]
 |  |     parent-subplan=01
-|  |     hosts=3 per-host-mem=unavailable
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=2 row-size=0B cardinality=10
 |  |
 |  05:NESTED LOOP JOIN [CROSS JOIN]
-|  |  hosts=3 per-host-mem=unavailable
+|  |  mem-estimate=270B mem-reservation=0B
 |  |  tuple-ids=1,0 row-size=278B cardinality=10
 |  |
 |  |--02:SINGULAR ROW SRC
 |  |     parent-subplan=01
-|  |     hosts=3 per-host-mem=unavailable
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=0 row-size=270B cardinality=1
 |  |
 |  03:UNNEST [c.c_orders o1]
 |     parent-subplan=01
-|     hosts=3 per-host-mem=unavailable
+|     mem-estimate=0B mem-reservation=0B
 |     tuple-ids=1 row-size=0B cardinality=10
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c]
@@ -314,41 +333,44 @@ PLAN-ROOT SINK
    predicates on o1: o1.o_orderkey < 5
    table stats: 150000 rows total
    columns missing stats: c_orders, c_orders
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=88.00MB mem-reservation=0B
    tuple-ids=0 row-size=270B cardinality=150000
 ---- PARALLELPLANS
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 07:EXCHANGE [UNPARTITIONED]
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=1,0,2 row-size=286B cardinality=1500000
 |
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
 01:SUBPLAN
-|  hosts=3 per-host-mem=0B
+|  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=1,0,2 row-size=286B cardinality=1500000
 |
 |--06:HASH JOIN [INNER JOIN]
 |  |  hash predicates: o1.o_orderkey = o2.o_orderkey + 2
-|  |  hosts=3 per-host-mem=0B
+|  |  mem-estimate=0B mem-reservation=136.00MB
 |  |  tuple-ids=1,0,2 row-size=286B cardinality=10
 |  |
 |  |--04:UNNEST [c.c_orders o2]
 |  |     parent-subplan=01
-|  |     hosts=3 per-host-mem=0B
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=2 row-size=0B cardinality=10
 |  |
 |  05:NESTED LOOP JOIN [CROSS JOIN]
-|  |  hosts=3 per-host-mem=270B
+|  |  mem-estimate=270B mem-reservation=0B
 |  |  tuple-ids=1,0 row-size=278B cardinality=10
 |  |
 |  |--02:SINGULAR ROW SRC
 |  |     parent-subplan=01
-|  |     hosts=3 per-host-mem=0B
+|  |     mem-estimate=0B mem-reservation=0B
 |  |     tuple-ids=0 row-size=270B cardinality=1
 |  |
 |  03:UNNEST [c.c_orders o1]
 |     parent-subplan=01
-|     hosts=3 per-host-mem=0B
+|     mem-estimate=0B mem-reservation=0B
 |     tuple-ids=1 row-size=0B cardinality=10
 |
 00:SCAN HDFS [tpch_nested_parquet.customer c, RANDOM]
@@ -357,6 +379,6 @@ PLAN-ROOT SINK
    predicates on o1: o1.o_orderkey < 5
    table stats: 150000 rows total
    columns missing stats: c_orders, c_orders
-   hosts=3 per-host-mem=88.00MB
+   mem-estimate=88.00MB mem-reservation=0B
    tuple-ids=0 row-size=270B cardinality=150000
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
index 4712b96..df5f99d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
@@ -7,21 +7,23 @@ select count(*) from functional_parquet.alltypes
 where int_col > 1 and int_col * rand() > 50 and int_col is null
 and int_col > tinyint_col;
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=10.00MB mem-reservation=0B
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=165.17KB
+   partitions=24/24 files=24 size=156.57KB
    predicates: int_col IS NULL, int_col > 1, int_col > tinyint_col, int_col * rand() > 50
    table stats: unavailable
    column stats: unavailable
    parquet statistics predicates: int_col > 1
    parquet dictionary predicates: int_col > 1
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=32.00MB mem-reservation=0B
    tuple-ids=0 row-size=5B cardinality=unavailable
 ====
 # Test a variety of types
@@ -32,20 +34,22 @@ and double_col > 100.00 and date_string_col > '1993-10-01' and string_col > 'aaa
 and timestamp_cmp(timestamp_col, '2016-11-20 00:00:00') = 1
 and year > 2000 and month < 12;
 ---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
 |
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
-|  hosts=3 per-host-mem=unavailable
+|  mem-estimate=10.00MB mem-reservation=0B
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=22/24 files=22 size=151.24KB
+   partitions=22/24 files=22 size=143.36KB
    predicates: bool_col, bigint_col < 5000, double_col > 100.00, float_col > 50.00, id = 1, smallint_col > 50, tinyint_col < 50, string_col > 'aaaa', mod(int_col, 2) = 1, timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = 1, date_string_col > '1993-10-01'
    table stats: unavailable
    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
    parquet statistics predicates: bigint_col < 5000, double_col > 100.00, float_col > 50.00, id = 1, smallint_col > 50, tinyint_col < 50, string_col > 'aaaa', date_string_col > '1993-10-01'
    parquet dictionary predicates: bool_col, bigint_col < 5000, double_col > 100.00, float_col > 50.00, id = 1, smallint_col > 50, tinyint_col < 50, string_col > 'aaaa', mod(int_col, 2) = 1, timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = 1, date_string_col > '1993-10-01'
-   hosts=3 per-host-mem=unavailable
+   mem-estimate=128.00MB mem-reservation=0B
    tuple-ids=0 row-size=80B cardinality=unavailable
 ====


[2/6] incubator-impala git commit: IMPALA-5173: crash with hash join feeding directly into nlj

Posted by ta...@apache.org.
IMPALA-5173: crash with hash join feeding directly into nlj

The background for this bug is that we can't transfer ownership
of BufferdBlockMgr::Blocks that are attached to RowBatches.

The NestedLoopJoinNode accumulates row batches on its right
side and tries to take ownership of the memory, which doesn't
work as expected in this case.

The fix is to copy the data when we encounter one of these
(likely very rare) cases.

Testing:
Added a regression test that produces a crash before the fix and
succeeds after the fix.

Change-Id: I0c04952e591d17e5ff7e994884be4c4c899ae192
Reviewed-on: http://gerrit.cloudera.org:8080/6568
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/96316e3b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/96316e3b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/96316e3b

Branch: refs/heads/master
Commit: 96316e3b34e1102d0df2714e617d9847b853a4c4
Parents: 1fa3315
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Apr 5 16:26:44 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Apr 18 20:11:23 2017 +0000

----------------------------------------------------------------------
 be/src/exec/nested-loop-join-builder.cc         |  8 ++++-
 .../queries/QueryTest/spilling.test             | 38 +++++++++++++++++++-
 2 files changed, 44 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/96316e3b/be/src/exec/nested-loop-join-builder.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/nested-loop-join-builder.cc b/be/src/exec/nested-loop-join-builder.cc
index 5631df4..86848f5 100644
--- a/be/src/exec/nested-loop-join-builder.cc
+++ b/be/src/exec/nested-loop-join-builder.cc
@@ -45,10 +45,16 @@ Status NljBuilder::Send(RuntimeState* state, RowBatch* batch) {
   build_batch->AcquireState(batch);
 
   AddBuildBatch(build_batch);
-  if (build_batch->needs_deep_copy()) {
+  if (build_batch->needs_deep_copy() || build_batch->num_blocks() > 0
+      || build_batch->num_buffers() > 0) {
     // This batch and earlier batches may refer to resources passed from the child
     // that aren't owned by the row batch itself. Deep copying ensures that the row
     // batches are backed by memory owned by this node that is safe to hold on to.
+    //
+    // Acquiring ownership of attached Blocks or Buffers does not correctly update the
+    // accounting, so also copy data in that cases to avoid stealing reservation
+    // from whoever created the Block/Buffer. TODO: remove workaround when IMPALA-4179
+    // is fixed.
     RETURN_IF_ERROR(DeepCopyBuildBatches(state));
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/96316e3b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling.test b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
index 89668e8..aa524a3 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
@@ -689,4 +689,40 @@ BIGINT
 5995258
 ---- RUNTIME_PROFILE
 row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
-====
\ No newline at end of file
+====
+---- QUERY
+# IMPALA-5171: spilling hash join feeding into right side of nested loop join.
+# Equivalent to:
+#   select *
+#   from lineitem
+#   where l1.l_quantity = 31.0 and l1.l_tax = 0.03 and l1.l_orderkey <= 100000
+# order by l_orderkey, l_partkey, l_suppkey, l_linenumber
+# limit 5
+set max_block_mgr_memory=7m;
+set num_nodes=1;
+select straight_join l.*
+from
+   (select *
+    from tpch_parquet.orders limit 1) o,
+   (select l2.*
+     from tpch_parquet.lineitem l1
+        inner join tpch_parquet.lineitem l2 on l1.l_orderkey = l2.l_orderkey
+            and l1.l_partkey = l2.l_partkey
+            and l1.l_suppkey = l2.l_suppkey and l1.l_linenumber = l2.l_linenumber
+     where
+        # Include a selective post-join predicate so that the RHS of the nested loop join
+        # doesn't consume too much memory.
+        (l1.l_quantity != l2.l_quantity or l1.l_quantity = 31.0 and l1.l_tax = 0.03)
+        # Reduce the data size to get the test to execute quicker
+         and l1.l_orderkey <= 100000) l
+order by l_orderkey, l_partkey, l_suppkey, l_linenumber
+limit 5;
+---- TYPES
+bigint,bigint,bigint,int,decimal,decimal,decimal,decimal,string,string,string,string,string,string,string,string
+---- RESULTS
+288,50641,8157,1,31.00,49340.84,0.00,0.03,'N','O','1997-03-17','1997-04-28','1997-04-06','TAKE BACK RETURN','AIR','instructions wa'
+418,18552,1054,1,31.00,45587.05,0.00,0.03,'N','F','1995-06-05','1995-06-18','1995-06-26','COLLECT COD','FOB','final theodolites. fluffil'
+482,61141,6154,3,31.00,34166.34,0.04,0.03,'N','O','1996-06-01','1996-05-06','1996-06-17','NONE','MAIL',' blithe pin'
+1382,156162,6163,5,31.00,37762.96,0.07,0.03,'R','F','1993-10-26','1993-10-15','1993-11-09','TAKE BACK RETURN','FOB','hely regular dependencies. f'
+1509,186349,3904,6,31.00,44495.54,0.04,0.03,'A','F','1993-07-14','1993-08-21','1993-08-06','COLLECT COD','SHIP','ic deposits cajole carefully. quickly bold '
+====


[3/6] incubator-impala git commit: IMPALA-4858: add more info to MemLimitExceeded errors

Posted by ta...@apache.org.
IMPALA-4858: add more info to MemLimitExceeded errors

This improves the usefulness of MemLimitExceeded errors:
- The host name is included.
- The memory left in the query and process limits is included.
- The approach for deciding whether to print the query or process
  MemTracker hierarchy is more robust: we show the query hierarchy
  only when we're closer to the query limit than the process limit.
  Previously if we were near but not over the process limit it
  printed the query output, which was often useless.
- The detailed output is included in the main error message, rather
  than in the "details" or merged errors. This reduces the impact
  of not logging those details - IMPALA-4697.

Note that the output still does not exactly reflect the state of
the world when the memory limit was exceeded because other threads
will concurrently allocate and release memory.

This also refactors the various related methods slightly to remove
RuntimeState::LogMemLimitExceeded() and reduce the number of
different methods that can log memory limit errors.

Sample output:
--------------
  Memory limit exceeded: Cannot perform aggregation at node with id 1. Failed to allocate 252 bytes for intermediate tuple.
  Fragment ab4765ecacca4d01:23edbb0800000002 could not allocate 252.00 B without exceeding limit.
  Error occurred on backend tarmstrong-box:22001 by fragment ab4765ecacca4d01:23edbb0800000002
  Memory left in process limit: 7.94 GB
  Memory left in query limit: 1.02 MB
  Query(ab4765ecacca4d01:23edbb0800000000): Limit=300.00 MB Total=298.98 MB Peak=299.98 MB
    Fragment ab4765ecacca4d01:23edbb0800000002: Total=105.32 MB Peak=114.14 MB
      AGGREGATION_NODE (id=1): Total=11.46 MB Peak=11.84 MB
      HDFS_SCAN_NODE (id=0): Total=93.73 MB Peak=102.09 MB
      DataStreamSender (dst_id=2): Total=70.02 KB Peak=86.02 KB
      CodeGen: Total=24.83 KB Peak=1.22 MB
    Block Manager: Limit=200.00 MB Total=185.00 MB Peak=186.00 MB
    Fragment ab4765ecacca4d01:23edbb0800000005: Total=185.66 MB Peak=194.66 MB
      AGGREGATION_NODE (id=3): Total=185.48 MB Peak=194.48 MB
      EXCHANGE_NODE (id=2): Total=0 Peak=0
      DataStreamRecvr: Total=160.24 KB Peak=435.45 KB
      DataStreamSender (dst_id=4): Total=688.00 B Peak=688.00 B
      CodeGen: Total=24.28 KB Peak=963.50 KB

Change-Id: Id4ab9d162cf7cd4508bce1efbfccfb4ba97e7355
Reviewed-on: http://gerrit.cloudera.org:8080/6537
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/c0e30604
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/c0e30604
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/c0e30604

Branch: refs/heads/master
Commit: c0e306047de57c39c43c9b7fbe06e64a4b824ca9
Parents: 96316e3
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Mon Apr 3 15:42:59 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Apr 18 20:34:50 2017 +0000

----------------------------------------------------------------------
 be/src/common/status.cc                         |  5 +++
 be/src/common/status.h                          |  1 +
 be/src/runtime/collection-value-builder-test.cc |  3 ++
 be/src/runtime/mem-tracker.cc                   | 34 ++++++++++++++++--
 be/src/runtime/runtime-state.cc                 | 37 ++++----------------
 be/src/runtime/runtime-state.h                  |  9 ++---
 be/src/util/debug-util.cc                       |  7 ++++
 be/src/util/debug-util.h                        |  3 ++
 8 files changed, 58 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0e30604/be/src/common/status.cc
----------------------------------------------------------------------
diff --git a/be/src/common/status.cc b/be/src/common/status.cc
index 6803103..3d3e6e7 100644
--- a/be/src/common/status.cc
+++ b/be/src/common/status.cc
@@ -37,6 +37,11 @@ Status Status::MemLimitExceeded() {
   return Status(TErrorCode::MEM_LIMIT_EXCEEDED, "Memory limit exceeded");
 }
 
+Status Status::MemLimitExceeded(const std::string& details) {
+  return Status(TErrorCode::MEM_LIMIT_EXCEEDED,
+        Substitute("Memory limit exceeded: $0", details));
+}
+
 Status::Status(TErrorCode::type code)
     : msg_(new ErrorMsg(code)) {
   VLOG(1) << msg_->msg() << "\n" << GetStackTrace();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0e30604/be/src/common/status.h
----------------------------------------------------------------------
diff --git a/be/src/common/status.h b/be/src/common/status.h
index 6cd9035..ff54dc9 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -90,6 +90,7 @@ class Status {
 
   // Return a MEM_LIMIT_EXCEEDED error status.
   static Status MemLimitExceeded();
+  static Status MemLimitExceeded(const std::string& details);
 
   static const Status CANCELLED;
   static const Status DEPRECATED_RPC;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0e30604/be/src/runtime/collection-value-builder-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/collection-value-builder-test.cc b/be/src/runtime/collection-value-builder-test.cc
index 613132a..af710ce 100644
--- a/be/src/runtime/collection-value-builder-test.cc
+++ b/be/src/runtime/collection-value-builder-test.cc
@@ -17,6 +17,7 @@
 
 #include "runtime/collection-value-builder.h"
 #include "runtime/mem-tracker.h"
+#include "runtime/test-env.h"
 #include "service/fe-support.h"
 #include "service/frontend.h"
 #include "testutil/desc-tbl-builder.h"
@@ -30,6 +31,8 @@ using namespace impala;
 static scoped_ptr<Frontend> fe;
 
 TEST(CollectionValueBuilderTest, MaxBufferSize) {
+  TestEnv test_env;
+  ASSERT_OK(test_env.Init());
   ObjectPool obj_pool;
   DescriptorTblBuilder builder(fe.get(), &obj_pool);
   builder.DeclareTuple() << TYPE_TINYINT << TYPE_TINYINT << TYPE_TINYINT;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0e30604/be/src/runtime/mem-tracker.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.cc b/be/src/runtime/mem-tracker.cc
index 9e3775d..cfd1881 100644
--- a/be/src/runtime/mem-tracker.cc
+++ b/be/src/runtime/mem-tracker.cc
@@ -286,9 +286,37 @@ string MemTracker::LogUsage(const string& prefix, const list<MemTracker*>& track
 
 Status MemTracker::MemLimitExceeded(RuntimeState* state, const std::string& details,
     int64_t failed_allocation_size) {
-  Status status = Status::MemLimitExceeded();
-  status.AddDetail(details);
-  if (state != NULL) state->LogMemLimitExceeded(this, failed_allocation_size);
+  DCHECK_GE(failed_allocation_size, 0);
+  stringstream ss;
+  if (details.size() != 0) ss << details << endl;
+  if (failed_allocation_size != 0) {
+    ss << label() << " could not allocate "
+       << PrettyPrinter::Print(failed_allocation_size, TUnit::BYTES)
+       << " without exceeding limit." << endl;
+  }
+  ss << "Error occurred on backend " << GetBackendString();
+  if (state != nullptr) ss << " by fragment " << state->fragment_instance_id();
+  ss << endl;
+  ExecEnv* exec_env = ExecEnv::GetInstance();
+  MemTracker* process_tracker = exec_env->process_mem_tracker();
+  const int64_t process_capacity = process_tracker->SpareCapacity();
+  ss << "Memory left in process limit: "
+     << PrettyPrinter::Print(process_capacity, TUnit::BYTES) << endl;
+
+  // Choose which tracker to log the usage of. Default to the process tracker so we can
+  // get the full view of memory consumption.
+  MemTracker* tracker_to_log = process_tracker;
+  if (state != nullptr && state->query_mem_tracker()->has_limit()) {
+    MemTracker* query_tracker = state->query_mem_tracker();
+    const int64_t query_capacity = query_tracker->limit() - query_tracker->consumption();
+    ss << "Memory left in query limit: "
+       << PrettyPrinter::Print(query_capacity, TUnit::BYTES) << endl;
+    // Log the query tracker only if the query limit was closer to being exceeded.
+    if (query_capacity < process_capacity) tracker_to_log = query_tracker;
+  }
+  ss << tracker_to_log->LogUsage();
+  Status status = Status::MemLimitExceeded(ss.str());
+  if (state != nullptr) state->LogError(status.msg());
   return status;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0e30604/be/src/runtime/runtime-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.cc b/be/src/runtime/runtime-state.cc
index c7b916b..62d0737 100644
--- a/be/src/runtime/runtime-state.cc
+++ b/be/src/runtime/runtime-state.cc
@@ -224,39 +224,15 @@ Status RuntimeState::LogOrReturnError(const ErrorMsg& message) {
   return Status::OK();
 }
 
-void RuntimeState::LogMemLimitExceeded(
-    const MemTracker* tracker, int64_t failed_allocation_size) {
-  DCHECK_GE(failed_allocation_size, 0);
-  DCHECK(query_mem_tracker_ != NULL);
-  stringstream ss;
-  ss << "Memory Limit Exceeded by fragment: " << fragment_instance_id() << endl;
-  if (failed_allocation_size != 0) {
-    DCHECK(tracker != NULL);
-    ss << "  " << tracker->label() << " could not allocate "
-       << PrettyPrinter::Print(failed_allocation_size, TUnit::BYTES)
-       << " without exceeding limit." << endl;
-  }
-
-  if (exec_env_->process_mem_tracker()->LimitExceeded()) {
-    ss << exec_env_->process_mem_tracker()->LogUsage();
-  } else {
-    ss << query_mem_tracker_->LogUsage();
-  }
-  LogError(ErrorMsg(TErrorCode::GENERAL, ss.str()));
-}
-
-Status RuntimeState::SetMemLimitExceeded(MemTracker* tracker,
+void RuntimeState::SetMemLimitExceeded(MemTracker* tracker,
     int64_t failed_allocation_size, const ErrorMsg* msg) {
+  Status status = tracker->MemLimitExceeded(this, msg == nullptr ? "" : msg->msg(),
+      failed_allocation_size);
   {
     lock_guard<SpinLock> l(query_status_lock_);
-    if (query_status_.ok()) {
-      query_status_ = Status::MemLimitExceeded();
-      if (msg != NULL) query_status_.MergeStatus(*msg);
-    } else {
-      return query_status_;
-    }
+    if (query_status_.ok()) query_status_ = status;
   }
-  LogMemLimitExceeded(tracker, failed_allocation_size);
+  LogError(status.msg());
   // Add warning about missing stats except for compute stats child queries.
   if (!query_ctx().__isset.parent_query_id &&
       query_ctx().__isset.tables_missing_stats &&
@@ -265,13 +241,12 @@ Status RuntimeState::SetMemLimitExceeded(MemTracker* tracker,
         GetTablesMissingStatsWarning(query_ctx().tables_missing_stats)));
   }
   DCHECK(query_status_.IsMemLimitExceeded());
-  return query_status_;
 }
 
 Status RuntimeState::CheckQueryState() {
   if (instance_mem_tracker_ != nullptr
       && UNLIKELY(instance_mem_tracker_->AnyLimitExceeded())) {
-    return SetMemLimitExceeded();
+    SetMemLimitExceeded(instance_mem_tracker_.get());
   }
   return GetQueryStatus();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0e30604/be/src/runtime/runtime-state.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-state.h b/be/src/runtime/runtime-state.h
index 009fee5..db9948f 100644
--- a/be/src/runtime/runtime-state.h
+++ b/be/src/runtime/runtime-state.h
@@ -278,13 +278,8 @@ class RuntimeState {
     query_status_ = Status(err_msg);
   }
 
-  /// Function for logging memory usages to the error log when memory limit is exceeded.
-  /// If 'failed_allocation_size' is greater than zero, logs the allocation size. If
-  /// 'failed_allocation_size' is zero, nothing about the allocation size is logged.
-  void LogMemLimitExceeded(const MemTracker* tracker, int64_t failed_allocation_size);
-
   /// Sets query_status_ to MEM_LIMIT_EXCEEDED and logs all the registered trackers.
-  /// Subsequent calls to this will be no-ops. Returns query_status_.
+  /// Subsequent calls to this will be no-ops.
   /// If 'failed_allocation_size' is not 0, then it is the size of the allocation (in
   /// bytes) that would have exceeded the limit allocated for 'tracker'.
   /// This value and tracker are only used for error reporting.
@@ -292,7 +287,7 @@ class RuntimeState {
   /// generic "Memory limit exceeded" error.
   /// Note that this interface is deprecated and MemTracker::LimitExceeded() should be
   /// used and the error status should be returned.
-  Status SetMemLimitExceeded(MemTracker* tracker = NULL,
+  void SetMemLimitExceeded(MemTracker* tracker,
       int64_t failed_allocation_size = 0, const ErrorMsg* msg = NULL);
 
   /// Returns a non-OK status if query execution should stop (e.g., the query was

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0e30604/be/src/util/debug-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/debug-util.cc b/be/src/util/debug-util.cc
index 596e7d2..d04550c 100644
--- a/be/src/util/debug-util.cc
+++ b/be/src/util/debug-util.cc
@@ -45,6 +45,9 @@ using boost::tokenizer;
 using namespace beeswax;
 using namespace parquet;
 
+DECLARE_int32(be_port);
+DECLARE_string(hostname);
+
 namespace impala {
 
 #define THRIFT_ENUM_OUTPUT_FN_IMPL(E, MAP) \
@@ -322,4 +325,8 @@ string GetStackTrace() {
   return s;
 }
 
+string GetBackendString() {
+  return Substitute("$0:$1", FLAGS_hostname, FLAGS_be_port);
+}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/c0e30604/be/src/util/debug-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/debug-util.h b/be/src/util/debug-util.h
index a02a288..fcfb9d6 100644
--- a/be/src/util/debug-util.h
+++ b/be/src/util/debug-util.h
@@ -104,6 +104,9 @@ std::string GetVersionString(bool compact = false);
 /// for recursive calls.
 std::string GetStackTrace();
 
+/// Returns the backend name in "host:port" form suitable for human consumption.
+std::string GetBackendString();
+
 // FILE_CHECKs are conditions that we expect to be true but could fail due to a malformed
 // input file. They differentiate these cases from DCHECKs, which indicate conditions that
 // are true unless there's a bug in Impala. We would ideally always return a bad Status


[6/6] incubator-impala git commit: IMPALA-3748: minimum buffer requirements in planner

Posted by ta...@apache.org.
IMPALA-3748: minimum buffer requirements in planner

Compute the minimum buffer requirement for spilling nodes and
per-host estimates for the entire plan tree.

This builds on top of the existing resource estimation code, which
computes the sets of plan nodes that can execute concurrently. This is
cleaned up so that the process of producing resource requirements is
clearer. It also removes the unused VCore estimates.

Fixes various bugs and other issues:
* computeCosts() was not called for unpartitioned fragments, so
  the per-operator memory estimate was not visible.
* Nested loop join was not treated as a blocking join.
* The TODO comment about union was misleading
* Fix the computation for mt_dop > 1 by distinguishing per-instance and
  per-host estimates.
* Always generate an estimate instead of unpredictably returning
  -1/"unavailable" in many circumstances - there was little rhyme or
  reason to when this happened.
* Remove the special "trivial plan" estimates. With the rest of the
  cleanup we generate estimates <= 10MB for those trivial plans through
  the normal code path.

I left one bug (IMPALA-4862) unfixed because it is subtle, will affect
estimates for many plans and will be easier to review once we have the
test infra in place.

Testing:
Added basic planner tests for resource requirements in both the MT and
non-MT cases.

Re-enabled the explain_level tests, which appears to be the only
coverage for many of these estimates. Removed the complex and
brittle test cases and replaced with a couple of much simpler
end-to-end tests.

Change-Id: I1e358182bcf2bc5fe5c73883eb97878735b12d37
Reviewed-on: http://gerrit.cloudera.org:8080/5847
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/9a29dfc9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/9a29dfc9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/9a29dfc9

Branch: refs/heads/master
Commit: 9a29dfc91b1ff8bbae3c94b53bf2b6ac81a271e0
Parents: c0e3060
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Jan 25 15:19:35 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Apr 18 20:36:08 2017 +0000

----------------------------------------------------------------------
 be/src/exec/analytic-eval-node.cc               |    4 +-
 be/src/exec/partitioned-aggregation-node.h      |    1 +
 be/src/exec/partitioned-hash-join-builder.h     |    1 +
 be/src/runtime/sorter.cc                        |    1 +
 be/src/scheduling/query-schedule.cc             |   15 +-
 be/src/service/query-exec-state.cc              |   14 +-
 common/thrift/Frontend.thrift                   |    8 +-
 .../apache/impala/analysis/AggregateInfo.java   |   13 +-
 .../apache/impala/analysis/TupleDescriptor.java |   10 +
 .../org/apache/impala/common/PrintUtils.java    |   12 +-
 .../apache/impala/planner/AggregationNode.java  |   38 +-
 .../apache/impala/planner/AnalyticEvalNode.java |   12 +-
 .../org/apache/impala/planner/DataSink.java     |   44 +-
 .../impala/planner/DataSourceScanNode.java      |    4 +-
 .../apache/impala/planner/DataStreamSink.java   |   14 +-
 .../impala/planner/DistributedPlanner.java      |   10 +-
 .../org/apache/impala/planner/EmptySetNode.java |   10 +-
 .../org/apache/impala/planner/ExchangeNode.java |    7 +
 .../apache/impala/planner/HBaseScanNode.java    |    4 +-
 .../apache/impala/planner/HBaseTableSink.java   |   18 +-
 .../org/apache/impala/planner/HashJoinNode.java |   29 +-
 .../org/apache/impala/planner/HdfsScanNode.java |   59 +-
 .../apache/impala/planner/HdfsTableSink.java    |   67 +-
 .../apache/impala/planner/JoinBuildSink.java    |   19 +-
 .../org/apache/impala/planner/JoinNode.java     |    6 +-
 .../org/apache/impala/planner/KuduScanNode.java |    6 +
 .../apache/impala/planner/KuduTableSink.java    |   19 +-
 .../impala/planner/NestedLoopJoinNode.java      |   15 +-
 .../apache/impala/planner/ParallelPlanner.java  |    8 +-
 .../impala/planner/PipelinedPlanNodeSet.java    |  141 +--
 .../org/apache/impala/planner/PlanFragment.java |   96 +-
 .../org/apache/impala/planner/PlanNode.java     |   60 +-
 .../org/apache/impala/planner/PlanRootSink.java |   13 +-
 .../java/org/apache/impala/planner/Planner.java |  123 +-
 .../apache/impala/planner/ResourceProfile.java  |   83 ++
 .../org/apache/impala/planner/SelectNode.java   |    8 +
 .../impala/planner/SingularRowSrcNode.java      |    8 +
 .../org/apache/impala/planner/SortNode.java     |   16 +-
 .../org/apache/impala/planner/SubplanNode.java  |    8 +
 .../org/apache/impala/planner/UnionNode.java    |    7 +
 .../org/apache/impala/planner/UnnestNode.java   |    7 +
 .../org/apache/impala/service/Frontend.java     |   17 +-
 .../org/apache/impala/planner/PlannerTest.java  |   10 +
 .../apache/impala/planner/PlannerTestBase.java  |   48 +-
 .../queries/PlannerTest/constant-folding.test   |   91 +-
 .../queries/PlannerTest/kudu-selectivity.test   |   70 +-
 .../queries/PlannerTest/mt-dop-validation.test  |  126 +-
 .../queries/PlannerTest/parquet-filtering.test  |   16 +-
 .../PlannerTest/resource-requirements.test      | 1121 ++++++++++++++++++
 .../queries/QueryTest/explain-level0.test       |  136 +--
 .../queries/QueryTest/explain-level1.test       |  224 +---
 .../queries/QueryTest/explain-level2.test       |  388 +-----
 .../queries/QueryTest/explain-level3.test       |  436 +------
 .../custom_cluster/test_admission_controller.py |    6 +-
 tests/metadata/test_explain.py                  |   12 +-
 55 files changed, 2119 insertions(+), 1620 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/be/src/exec/analytic-eval-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/analytic-eval-node.cc b/be/src/exec/analytic-eval-node.cc
index 16dce50..083ab1a 100644
--- a/be/src/exec/analytic-eval-node.cc
+++ b/be/src/exec/analytic-eval-node.cc
@@ -177,9 +177,11 @@ Status AnalyticEvalNode::Prepare(RuntimeState* state) {
     }
   }
 
+  // Must be kept in sync with AnalyticEvalNode.computeResourceProfile() in fe.
+  const int MIN_REQUIRED_BUFFERS = 2;
   RETURN_IF_ERROR(state->block_mgr()->RegisterClient(
       Substitute("AnalyticEvalNode id=$0 ptr=$1", id_, this),
-      2, false, mem_tracker(), state, &client_));
+      MIN_REQUIRED_BUFFERS, false, mem_tracker(), state, &client_));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/be/src/exec/partitioned-aggregation-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-aggregation-node.h b/be/src/exec/partitioned-aggregation-node.h
index f26a252..2155473 100644
--- a/be/src/exec/partitioned-aggregation-node.h
+++ b/be/src/exec/partitioned-aggregation-node.h
@@ -679,6 +679,7 @@ class PartitionedAggregationNode : public ExecNode {
   /// as the partitions aggregate stream needs to be serialized and rewritten.
   /// We do not spill streaming preaggregations, so we do not need to reserve any buffers.
   int MinRequiredBuffers() const {
+    // Must be kept in sync with AggregationNode.computeResourceProfile() in fe.
     if (is_streaming_preagg_) return 0;
     return 2 * PARTITION_FANOUT + 1 + (needs_serialize_ ? 1 : 0);
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/be/src/exec/partitioned-hash-join-builder.h
----------------------------------------------------------------------
diff --git a/be/src/exec/partitioned-hash-join-builder.h b/be/src/exec/partitioned-hash-join-builder.h
index 48f4c88..b133be6 100644
--- a/be/src/exec/partitioned-hash-join-builder.h
+++ b/be/src/exec/partitioned-hash-join-builder.h
@@ -247,6 +247,7 @@ class PhjBuilder : public DataSink {
   /// For NAAJ, we need 3 additional buffers for 'null_aware_partition_',
   /// 'null_aware_probe_partition_' and 'null_probe_rows_'.
   int MinRequiredBuffers() const {
+    // Must be kept in sync with HashJoinNode.computeResourceProfile() in fe.
     int num_reserved_buffers = PARTITION_FANOUT + 1;
     if (join_op_ == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) num_reserved_buffers += 3;
     return num_reserved_buffers;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/be/src/runtime/sorter.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/sorter.cc b/be/src/runtime/sorter.cc
index 01554b8..1acdf15 100644
--- a/be/src/runtime/sorter.cc
+++ b/be/src/runtime/sorter.cc
@@ -1376,6 +1376,7 @@ Status Sorter::Init() {
   in_mem_sort_timer_ = ADD_TIMER(profile_, "InMemorySortTime");
   sorted_data_size_ = ADD_COUNTER(profile_, "SortDataSize", TUnit::BYTES);
 
+  // Must be kept in sync with SortNode.computeResourceProfile() in fe.
   int min_buffers_required = MIN_BUFFERS_PER_MERGE;
   // Fixed and var-length blocks are separate, so we need MIN_BUFFERS_PER_MERGE
   // blocks for both if there is var-length data.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/be/src/scheduling/query-schedule.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-schedule.cc b/be/src/scheduling/query-schedule.cc
index 668c9c8..bb44145 100644
--- a/be/src/scheduling/query-schedule.cc
+++ b/be/src/scheduling/query-schedule.cc
@@ -190,13 +190,6 @@ int64_t QuerySchedule::GetPerHostMemoryEstimate() const {
     has_query_option = true;
   }
 
-  int64_t estimate_limit = numeric_limits<int64_t>::max();
-  bool has_estimate = false;
-  if (request_.__isset.per_host_mem_req && request_.per_host_mem_req > 0) {
-    estimate_limit = request_.per_host_mem_req;
-    has_estimate = true;
-  }
-
   int64_t per_host_mem = 0L;
   // TODO: Remove rm_initial_mem and associated logic when we're sure that clients won't
   // be affected.
@@ -204,13 +197,9 @@ int64_t QuerySchedule::GetPerHostMemoryEstimate() const {
     per_host_mem = query_options_.rm_initial_mem;
   } else if (has_query_option) {
     per_host_mem = query_option_memory_limit;
-  } else if (has_estimate) {
-    per_host_mem = estimate_limit;
   } else {
-    // If no estimate or query option, use the server-side limits anyhow.
-    bool ignored;
-    per_host_mem = ParseUtil::ParseMemSpec(FLAGS_rm_default_memory,
-        &ignored, 0);
+    DCHECK(request_.__isset.per_host_mem_estimate);
+    per_host_mem = request_.per_host_mem_estimate;
   }
   // Cap the memory estimate at the amount of physical memory available. The user's
   // provided value or the estimate from planning can each be unreasonable.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/be/src/service/query-exec-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-exec-state.cc b/be/src/service/query-exec-state.cc
index f704676..72dba6f 100644
--- a/be/src/service/query-exec-state.cc
+++ b/be/src/service/query-exec-state.cc
@@ -58,7 +58,7 @@ namespace impala {
 // Keys into the info string map of the runtime profile referring to specific
 // items used by CM for monitoring purposes.
 static const string PER_HOST_MEM_KEY = "Estimated Per-Host Mem";
-static const string PER_HOST_VCORES_KEY = "Estimated Per-Host VCores";
+static const string PER_HOST_MEMORY_RESERVATION_KEY = "Per-Host Memory Reservation";
 static const string TABLES_MISSING_STATS_KEY = "Tables Missing Stats";
 static const string TABLES_WITH_CORRUPT_STATS_KEY = "Tables With Corrupt Table Stats";
 static const string TABLES_WITH_MISSING_DISK_IDS_KEY = "Tables With Missing Disk Ids";
@@ -394,16 +394,16 @@ Status ImpalaServer::QueryExecState::ExecQueryOrDmlRequest(
             << "----------------";
     summary_profile_.AddInfoString("Plan", plan_ss.str());
   }
-  // Add info strings consumed by CM: Estimated mem/vcores and tables missing stats.
-  if (query_exec_request.__isset.per_host_mem_req) {
+  // Add info strings consumed by CM: Estimated mem and tables missing stats.
+  if (query_exec_request.__isset.per_host_mem_estimate) {
     stringstream ss;
-    ss << query_exec_request.per_host_mem_req;
+    ss << query_exec_request.per_host_mem_estimate;
     summary_profile_.AddInfoString(PER_HOST_MEM_KEY, ss.str());
   }
-  if (query_exec_request.__isset.per_host_vcores) {
+  if (query_exec_request.__isset.per_host_min_reservation) {
     stringstream ss;
-    ss << query_exec_request.per_host_vcores;
-    summary_profile_.AddInfoString(PER_HOST_VCORES_KEY, ss.str());
+    ss << query_exec_request.per_host_min_reservation;
+    summary_profile_.AddInfoString(PER_HOST_MEMORY_RESERVATION_KEY, ss.str());
   }
   if (!query_exec_request.query_ctx.__isset.parent_query_id &&
       query_exec_request.query_ctx.__isset.tables_missing_stats &&

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/common/thrift/Frontend.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index 4cef592..4337851 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -387,12 +387,10 @@ struct TQueryExecRequest {
   7: required Types.TStmtType stmt_type
 
   // Estimated per-host peak memory consumption in bytes. Used for resource management.
-  8: optional i64 per_host_mem_req
+  8: optional i64 per_host_mem_estimate
 
-  // Estimated per-host CPU requirements in YARN virtual cores.
-  // Used for resource management.
-  // TODO: Remove this and associated code in Planner.
-  9: optional i16 per_host_vcores
+  // Minimum buffer reservation required per host in bytes.
+  9: optional i64 per_host_min_reservation;
 
   // List of replica hosts.  Used by the host_idx field of TScanRangeLocation.
   10: required list<Types.TNetworkAddress> host_list

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java b/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java
index 3e12ee1..4f8b4fc 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AggregateInfo.java
@@ -20,10 +20,10 @@ package org.apache.impala.analysis;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.impala.catalog.AggregateFunction;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.InternalException;
-import org.apache.impala.planner.DataPartition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -700,6 +700,17 @@ public class AggregateInfo extends AggregateInfoBase {
     }
   }
 
+  /// Return true if any aggregate functions have a serialize function.
+  /// Only valid to call once analyzed.
+  public boolean needsSerialize() {
+    for (FunctionCallExpr aggregateExpr: aggregateExprs_) {
+      Preconditions.checkState(aggregateExpr.isAnalyzed());
+      AggregateFunction fn = (AggregateFunction)aggregateExpr.getFn();
+      if (fn.getSerializeFnSymbol() != null) return true;
+    }
+    return false;
+  }
+
   @Override
   public String debugString() {
     StringBuilder out = new StringBuilder(super.debugString());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java b/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java
index 2e501c1..6ec2d26 100644
--- a/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java
+++ b/fe/src/main/java/org/apache/impala/analysis/TupleDescriptor.java
@@ -347,4 +347,14 @@ public class TupleDescriptor {
     }
     return partitionSlots;
   }
+
+  /**
+   * Returns true if the tuple has any variable-length slots.
+   */
+  public boolean hasVarLenSlots() {
+    for (SlotDescriptor slot: slots_) {
+      if (!slot.getType().isFixedLengthType()) return true;
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/common/PrintUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/common/PrintUtils.java b/fe/src/main/java/org/apache/impala/common/PrintUtils.java
index 3062161..77d77dd 100644
--- a/fe/src/main/java/org/apache/impala/common/PrintUtils.java
+++ b/fe/src/main/java/org/apache/impala/common/PrintUtils.java
@@ -26,6 +26,7 @@ import static org.apache.impala.common.ByteUnits.TERABYTE;
 import java.text.DecimalFormat;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.impala.planner.PlanFragmentId;
 
 /**
  * Utility functions for pretty printing.
@@ -51,20 +52,19 @@ public class PrintUtils {
         ((cardinality != -1) ? String.valueOf(cardinality) : "unavailable");
   }
 
-  public static String printHosts(String prefix, long numHosts) {
+  public static String printNumHosts(String prefix, long numHosts) {
     return prefix + "hosts=" + ((numHosts != -1) ? numHosts : "unavailable");
   }
 
-  public static String printMemCost(String prefix, long perHostMemCost) {
-    return prefix + "per-host-mem=" +
-        ((perHostMemCost != -1) ? printBytes(perHostMemCost) : "unavailable");
+  public static String printNumInstances(String prefix, long numInstances) {
+    return prefix + "instances=" + ((numInstances != -1) ? numInstances : "unavailable");
   }
 
   /**
    * Prints the given square matrix into matrixStr. Separates cells by cellSpacing.
    */
-  public static void printMatrix(boolean[][] matrix, int cellSpacing,
-      StringBuilder matrixStr) {
+  public static void printMatrix(
+      boolean[][] matrix, int cellSpacing, StringBuilder matrixStr) {
     // Print labels.
     matrixStr.append(StringUtils.repeat(' ', cellSpacing));
     String formatStr = "%Xd".replace("X", String.valueOf(cellSpacing));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
----------------------------------------------------------------------
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 07c51f1..5cfce82 100644
--- a/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AggregationNode.java
@@ -48,9 +48,9 @@ import com.google.common.collect.Sets;
 public class AggregationNode extends PlanNode {
   private final static Logger LOG = LoggerFactory.getLogger(AggregationNode.class);
 
-  // Default per-host memory requirement used if no valid stats are available.
+  // Default per-instance memory requirement used if no valid stats are available.
   // TODO: Come up with a more useful heuristic.
-  private final static long DEFAULT_PER_HOST_MEM = 128L * 1024L * 1024L;
+  private final static long DEFAULT_PER_INSTANCE_MEM = 128L * 1024L * 1024L;
 
   // Conservative minimum size of hash table for low-cardinality aggregations.
   private final static long MIN_HASH_TBL_MEM = 10L * 1024L * 1024L;
@@ -278,21 +278,33 @@ public class AggregationNode extends PlanNode {
   }
 
   @Override
-  public void computeCosts(TQueryOptions queryOptions) {
-    Preconditions.checkNotNull(fragment_,
-        "PlanNode must be placed into a fragment before calling this method.");
-    perHostMemCost_ = 0;
-    long perHostCardinality = fragment_.getNumDistinctValues(aggInfo_.getGroupingExprs());
-    if (perHostCardinality == -1) {
-      perHostMemCost_ = DEFAULT_PER_HOST_MEM;
-      return;
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    Preconditions.checkNotNull(
+        fragment_, "PlanNode must be placed into a fragment before calling this method.");
+    // Must be kept in sync with PartitionedAggregationNode::MinRequiredBuffers() in be.
+    long perInstanceMinBuffers;
+    if (aggInfo_.getGroupingExprs().isEmpty() || useStreamingPreagg_) {
+      perInstanceMinBuffers = 0;
+    } else {
+      final int PARTITION_FANOUT = 16;
+      long minBuffers = 2 * PARTITION_FANOUT + 1 + (aggInfo_.needsSerialize() ? 1 : 0);
+      perInstanceMinBuffers = SPILLABLE_BUFFER_BYTES * minBuffers;
     }
 
-    // Per-host cardinality cannot be greater than the total output cardinality.
+    long perInstanceCardinality = fragment_.getPerInstanceNdv(
+        queryOptions.getMt_dop(), aggInfo_.getGroupingExprs());
+    if (perInstanceCardinality == -1) {
+      resourceProfile_ =
+          new ResourceProfile(DEFAULT_PER_INSTANCE_MEM, perInstanceMinBuffers);
+      return;
+    }
+    // Per-instance cardinality cannot be greater than the total output cardinality.
     if (cardinality_ != -1) {
-      perHostCardinality = Math.min(perHostCardinality, cardinality_);
+      perInstanceCardinality = Math.min(perInstanceCardinality, cardinality_);
     }
-    perHostMemCost_ += Math.max(perHostCardinality * avgRowSize_ *
+    long perInstanceMemEstimate = (long)Math.max(perInstanceCardinality * avgRowSize_ *
         PlannerContext.HASH_TBL_SPACE_OVERHEAD, MIN_HASH_TBL_MEM);
+    resourceProfile_ =
+        new ResourceProfile(perInstanceMemEstimate, perInstanceMinBuffers);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
----------------------------------------------------------------------
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 408680b..e0981c7 100644
--- a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
@@ -242,10 +242,14 @@ public class AnalyticEvalNode extends PlanNode {
   }
 
   @Override
-  public void computeCosts(TQueryOptions queryOptions) {
-    Preconditions.checkNotNull(fragment_,
-        "PlanNode must be placed into a fragment before calling this method.");
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    Preconditions.checkNotNull(
+        fragment_, "PlanNode must be placed into a fragment before calling this method.");
     // TODO: come up with estimate based on window
-    perHostMemCost_ = 0;
+    long perInstanceMemEstimate = 0;
+
+    // Must be kept in sync with MIN_REQUIRED_BUFFERS in AnalyticEvalNode in be.
+    long perInstanceMinBufferBytes = 2 * SPILLABLE_BUFFER_BYTES;
+    resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, perInstanceMinBufferBytes);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/DataSink.java
----------------------------------------------------------------------
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 b1977f4..7fc0c83 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSink.java
@@ -17,15 +17,9 @@
 
 package org.apache.impala.planner;
 
-import java.util.List;
-
-import org.apache.impala.analysis.Expr;
-import org.apache.impala.catalog.HBaseTable;
-import org.apache.impala.catalog.HdfsTable;
-import org.apache.impala.catalog.KuduTable;
-import org.apache.impala.catalog.Table;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TExplainLevel;
+import org.apache.impala.thrift.TQueryOptions;
 
 /**
  * A DataSink describes the destination of a plan fragment's output rows.
@@ -35,30 +29,44 @@ import org.apache.impala.thrift.TExplainLevel;
  */
 public abstract class DataSink {
 
-  // estimated per-host memory requirement for sink;
-  // set in computeCosts(); invalid: -1
-  protected long perHostMemCost_ = -1;
-
   // Fragment that this DataSink belongs to. Set by the PlanFragment enclosing this sink.
   protected PlanFragment fragment_;
 
+  // resource requirements and estimates for this plan node.
+  // set in computeResourceProfile()
+  protected ResourceProfile resourceProfile_ = null;
+
   /**
    * Return an explain string for the DataSink. Each line of the explain will be prefixed
    * by "prefix".
    */
-  public abstract String getExplainString(String prefix, String detailPrefix,
-      TExplainLevel explainLevel);
+  public final String getExplainString(String prefix, String detailPrefix,
+      TQueryOptions queryOptions, TExplainLevel explainLevel) {
+    StringBuilder output = new StringBuilder();
+    appendSinkExplainString(prefix, detailPrefix, queryOptions, explainLevel, output);
+    if (explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
+      output.append(detailPrefix);
+      output.append(resourceProfile_.getExplainString());
+      output.append("\n");
+    }
+    return output.toString();
+  }
+
+  /**
+   * Append the node-specific lines of the explain string to "output".
+   */
+  abstract protected void appendSinkExplainString(String prefix, String detailPrefix,
+      TQueryOptions queryOptions, TExplainLevel explainLevel, StringBuilder output);
 
   protected abstract TDataSink toThrift();
 
   public void setFragment(PlanFragment fragment) { fragment_ = fragment; }
   public PlanFragment getFragment() { return fragment_; }
-  public long getPerHostMemCost() { return perHostMemCost_; }
+  public ResourceProfile getResourceProfile() { return resourceProfile_; }
 
   /**
-   * Estimates the cost of executing this DataSink. Currently only sets perHostMemCost.
+   * Compute the resource profile for an instance of this DataSink.
    */
-  public void computeCosts() {
-    perHostMemCost_ = 0;
-  }
+  public abstract void computeResourceProfile(TQueryOptions queryOptions);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
----------------------------------------------------------------------
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 89f8377..ab80439 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataSourceScanNode.java
@@ -331,9 +331,9 @@ public class DataSourceScanNode extends ScanNode {
   }
 
   @Override
-  public void computeCosts(TQueryOptions queryOptions) {
+  public void computeResourceProfile(TQueryOptions queryOptions) {
     // TODO: What's a good estimate of memory consumption?
-    perHostMemCost_ = 1024L * 1024L * 1024L;
+    resourceProfile_ = new ResourceProfile(1024L * 1024L * 1024L, 0);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
----------------------------------------------------------------------
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 952215e..d1369f5 100644
--- a/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/DataStreamSink.java
@@ -21,6 +21,8 @@ import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TDataStreamSink;
 import org.apache.impala.thrift.TExplainLevel;
+import org.apache.impala.thrift.TQueryOptions;
+
 import com.google.common.base.Preconditions;
 
 /**
@@ -38,14 +40,18 @@ public class DataStreamSink extends DataSink {
   }
 
   @Override
-  public String getExplainString(String prefix, String detailPrefix,
-      TExplainLevel detailLevel) {
-    StringBuilder output = new StringBuilder();
+  public void appendSinkExplainString(String prefix, String detailPrefix,
+      TQueryOptions queryOptions, TExplainLevel detailLevel, StringBuilder output) {
     output.append(
         String.format("%sDATASTREAM SINK [FRAGMENT=%s, EXCHANGE=%s, %s]",
         prefix, exchNode_.getFragment().getId().toString(),
         exchNode_.getId().toString(), exchNode_.getDisplayLabelDetail()));
-    return output.toString();
+    output.append("\n");
+  }
+
+  @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    resourceProfile_ = new ResourceProfile(0, 0);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java b/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
index 8fe3cf9..85a8ab8 100644
--- a/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
@@ -19,7 +19,6 @@ package org.apache.impala.planner;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.impala.analysis.AggregateInfo;
 import org.apache.impala.analysis.AnalysisContext;
@@ -28,7 +27,6 @@ import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.InsertStmt;
 import org.apache.impala.analysis.JoinOperator;
 import org.apache.impala.analysis.QueryStmt;
-import org.apache.impala.analysis.TupleId;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.planner.JoinNode.DistributionMode;
@@ -38,8 +36,6 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
 
 /**
  * The distributed planner is responsible for creating an executable, distributed plan
@@ -155,8 +151,8 @@ public class DistributedPlanner {
       result = new PlanFragment(
           ctx_.getNextFragmentId(), root, DataPartition.UNPARTITIONED);
     } else {
-      throw new InternalException(
-          "Cannot create plan fragment for this node type: " + root.getExplainString());
+      throw new InternalException("Cannot create plan fragment for this node type: "
+          + root.getExplainString(ctx_.getQueryOptions()));
     }
     // move 'result' to end, it depends on all of its children
     fragments.remove(result);
@@ -478,7 +474,7 @@ public class DistributedPlanner {
           + Float.toString(lhsTree.getAvgRowSize()));
       LOG.trace("rhs card=" + Long.toString(rhsTree.getCardinality()) + " row_size="
           + Float.toString(rhsTree.getAvgRowSize()));
-      LOG.trace(rhsTree.getExplainString());
+      LOG.trace(rhsTree.getExplainString(ctx_.getQueryOptions()));
     }
 
     boolean doBroadcast = false;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java b/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
index 2b46bd9..174051d 100644
--- a/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/EmptySetNode.java
@@ -24,6 +24,8 @@ import org.apache.impala.analysis.TupleId;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
+import org.apache.impala.thrift.TQueryOptions;
+
 import com.google.common.base.Preconditions;
 
 /**
@@ -42,7 +44,6 @@ public class EmptySetNode extends PlanNode {
   public void computeStats(Analyzer analyzer) {
     avgRowSize_ = 0;
     cardinality_ = 0;
-    perHostMemCost_ = 0;
     numNodes_ = 1;
   }
 
@@ -59,6 +60,12 @@ public class EmptySetNode extends PlanNode {
   }
 
   @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add an estimate
+    resourceProfile_ = new ResourceProfile(0, 0);
+  }
+
+  @Override
   protected String getNodeExplainString(String prefix, String detailPrefix,
       TExplainLevel detailLevel) {
     return String.format("%s%s:%s\n", prefix, id_.toString(), displayName_);
@@ -68,4 +75,5 @@ public class EmptySetNode extends PlanNode {
   protected void toThrift(TPlanNode msg) {
     msg.node_type = TPlanNodeType.EMPTY_SET_NODE;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
index 32673ae..d3997b8 100644
--- a/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/ExchangeNode.java
@@ -29,6 +29,7 @@ import org.apache.impala.thrift.TExchangeNode;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
+import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TSortInfo;
 import com.google.common.base.Preconditions;
 
@@ -184,6 +185,12 @@ public class ExchangeNode extends PlanNode {
   }
 
   @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add an estimate
+    resourceProfile_ =  new ResourceProfile(0, 0);
+  }
+
+  @Override
   protected void toThrift(TPlanNode msg) {
     msg.node_type = TPlanNodeType.EXCHANGE_NODE;
     msg.exchange_node = new TExchangeNode();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
----------------------------------------------------------------------
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 aea3750..223362f 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseScanNode.java
@@ -494,9 +494,9 @@ public class HBaseScanNode extends ScanNode {
   }
 
   @Override
-  public void computeCosts(TQueryOptions queryOptions) {
+  public void computeResourceProfile(TQueryOptions queryOptions) {
     // TODO: What's a good estimate of memory consumption?
-    perHostMemCost_ = 1024L * 1024L * 1024L;
+    resourceProfile_ =  new ResourceProfile(1024L * 1024L * 1024L, 0);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java
----------------------------------------------------------------------
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 1d7994b..947665e 100644
--- a/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HBaseTableSink.java
@@ -20,10 +20,10 @@ package org.apache.impala.planner;
 
 import org.apache.impala.analysis.DescriptorTable;
 import org.apache.impala.catalog.Table;
-import org.apache.impala.common.PrintUtils;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExplainLevel;
+import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TTableSink;
 import org.apache.impala.thrift.TTableSinkType;
 
@@ -37,16 +37,14 @@ public class HBaseTableSink extends TableSink {
   }
 
   @Override
-  public String getExplainString(String prefix, String detailPrefix,
-      TExplainLevel explainLevel) {
-    StringBuilder output = new StringBuilder();
+  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(PrintUtils.printHosts(detailPrefix, fragment_.getNumNodes()));
-      output.append(PrintUtils.printMemCost(" ", perHostMemCost_));
-      output.append("\n");
-    }
-    return output.toString();
+  }
+
+  @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    resourceProfile_ = new ResourceProfile(0, 0);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
----------------------------------------------------------------------
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 008cd5c..f819513 100644
--- a/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HashJoinNode.java
@@ -19,9 +19,6 @@ package org.apache.impala.planner;
 
 import java.util.List;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.Expr;
@@ -57,6 +54,9 @@ public class HashJoinNode extends JoinNode {
   }
 
   @Override
+  public boolean isBlockingJoinNode() { return true; }
+
+  @Override
   public List<BinaryPredicate> getEqJoinConjuncts() { return eqJoinConjuncts_; }
 
   @Override
@@ -177,15 +177,24 @@ public class HashJoinNode extends JoinNode {
   }
 
   @Override
-  public void computeCosts(TQueryOptions queryOptions) {
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // Must be kept in sync with PartitionedHashJoinBuilder::MinRequiredBuffers() in be.
+    final int PARTITION_FANOUT = 16;
+    long minBuffers = PARTITION_FANOUT + 1
+        + (joinOp_ == JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN ? 3 : 0);
+    long perInstanceMinBufferBytes = SPILLABLE_BUFFER_BYTES * minBuffers;
+
+    long perInstanceMemEstimate;
     if (getChild(1).getCardinality() == -1 || getChild(1).getAvgRowSize() == -1
         || numNodes_ == 0) {
-      perHostMemCost_ = DEFAULT_PER_HOST_MEM;
-      return;
+      perInstanceMemEstimate = DEFAULT_PER_INSTANCE_MEM;
+    } else {
+      perInstanceMemEstimate = (long) Math.ceil(getChild(1).cardinality_
+          * getChild(1).avgRowSize_ * PlannerContext.HASH_TBL_SPACE_OVERHEAD);
+      if (distrMode_ == DistributionMode.PARTITIONED) {
+        perInstanceMemEstimate /= fragment_.getNumInstances(queryOptions.getMt_dop());
+      }
     }
-    perHostMemCost_ =
-        (long) Math.ceil(getChild(1).cardinality_ * getChild(1).avgRowSize_
-          * PlannerContext.HASH_TBL_SPACE_OVERHEAD);
-    if (distrMode_ == DistributionMode.PARTITIONED) perHostMemCost_ /= numNodes_;
+    resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, perInstanceMinBufferBytes);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
----------------------------------------------------------------------
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 cb6627f..9ff1a6b 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -32,7 +32,6 @@ import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.DescriptorTable;
 import org.apache.impala.analysis.Expr;
-import org.apache.impala.analysis.FunctionCallExpr;
 import org.apache.impala.analysis.SlotDescriptor;
 import org.apache.impala.analysis.SlotId;
 import org.apache.impala.analysis.SlotRef;
@@ -51,6 +50,7 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.NotImplementedException;
 import org.apache.impala.common.PrintUtils;
 import org.apache.impala.common.RuntimeEnv;
+import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TExpr;
 import org.apache.impala.thrift.THdfsFileBlock;
@@ -72,7 +72,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.base.Predicates;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
@@ -82,7 +81,7 @@ import com.google.common.collect.Sets;
  *
  * It's expected that the creator of this object has already done any necessary
  * partition pruning before creating this object. In other words, the 'conjuncts'
- * passed to the constructors are conjucts not fully evaluated by partition pruning
+ * passed to the constructors are conjuncts not fully evaluated by partition pruning
  * and 'partitions' are the remaining partitions after pruning.
  *
  * For scans of tables with Parquet files the class creates an additional list of
@@ -95,14 +94,13 @@ import com.google.common.collect.Sets;
 public class HdfsScanNode extends ScanNode {
   private final static Logger LOG = LoggerFactory.getLogger(HdfsScanNode.class);
 
-  // Read size of the backend I/O manager. Used in computeCosts().
-  private final static long IO_MGR_BUFFER_SIZE = 8L * 1024L * 1024L;
-
   // Maximum number of I/O buffers per thread executing this scan.
+  // TODO: it's unclear how this was chosen - this seems like a very high number
   private final static long MAX_IO_BUFFERS_PER_THREAD = 10;
 
-  // Number of scanner threads per core executing this scan.
-  private final static int THREADS_PER_CORE = 3;
+  // Maximum number of thread tokens per core that may be used to spin up extra scanner
+  // threads. Corresponds to the default value of --num_threads_per_core in the backend.
+  private final static int MAX_THREAD_TOKENS_PER_CORE = 3;
 
   // Factor capturing the worst-case deviation from a uniform distribution of scan ranges
   // among nodes. The factor of 1.2 means that a particular node may have 20% more
@@ -808,10 +806,10 @@ public class HdfsScanNode extends ScanNode {
   }
 
   @Override
-  public void computeCosts(TQueryOptions queryOptions) {
+  public void computeResourceProfile(TQueryOptions queryOptions) {
     Preconditions.checkNotNull(scanRanges_, "Cost estimation requires scan ranges.");
     if (scanRanges_.isEmpty()) {
-      perHostMemCost_ = 0;
+      resourceProfile_ = new ResourceProfile(0, 0);
       return;
     }
     Preconditions.checkState(0 < numNodes_ && numNodes_ <= scanRanges_.size());
@@ -834,34 +832,37 @@ public class HdfsScanNode extends ScanNode {
           (double) scanRanges_.size() / (double) numNodes_) * SCAN_RANGE_SKEW_FACTOR);
     }
 
-    // TODO: The total memory consumption for a particular query depends on the number
-    // of *available* cores, i.e., it depends the resource consumption of other
-    // concurrent queries. Figure out how to account for that.
-    int maxScannerThreads = Math.min(perHostScanRanges,
-        RuntimeEnv.INSTANCE.getNumCores() * THREADS_PER_CORE);
-    // Account for the max scanner threads query option.
-    if (queryOptions.isSetNum_scanner_threads() &&
-        queryOptions.getNum_scanner_threads() > 0) {
-      maxScannerThreads =
-          Math.min(maxScannerThreads, queryOptions.getNum_scanner_threads());
+    int maxScannerThreads;
+    if (queryOptions.getMt_dop() >= 1) {
+      maxScannerThreads = 1;
+    } else {
+      maxScannerThreads = Math.min(perHostScanRanges, RuntimeEnv.INSTANCE.getNumCores());
+      // Account for the max scanner threads query option.
+      if (queryOptions.isSetNum_scanner_threads() &&
+          queryOptions.getNum_scanner_threads() > 0) {
+        maxScannerThreads =
+            Math.min(maxScannerThreads, queryOptions.getNum_scanner_threads());
+      }
     }
 
     long avgScanRangeBytes = (long) Math.ceil(totalBytes_ / (double) scanRanges_.size());
     // The +1 accounts for an extra I/O buffer to read past the scan range due to a
     // trailing record spanning Hdfs blocks.
+    long readSize = BackendConfig.INSTANCE.getReadSize();
     long perThreadIoBuffers =
-        Math.min((long) Math.ceil(avgScanRangeBytes / (double) IO_MGR_BUFFER_SIZE),
+        Math.min((long) Math.ceil(avgScanRangeBytes / (double) readSize),
             MAX_IO_BUFFERS_PER_THREAD) + 1;
-    perHostMemCost_ = maxScannerThreads * perThreadIoBuffers * IO_MGR_BUFFER_SIZE;
+    long perInstanceMemEstimate = maxScannerThreads * perThreadIoBuffers * readSize;
 
     // Sanity check: the tighter estimation should not exceed the per-host maximum.
     long perHostUpperBound = getPerHostMemUpperBound();
-    if (perHostMemCost_ > perHostUpperBound) {
-      LOG.warn(String.format("Per-host mem cost %s exceeded per-host upper bound %s.",
-          PrintUtils.printBytes(perHostMemCost_),
+    if (perInstanceMemEstimate > perHostUpperBound) {
+      LOG.warn(String.format("Per-instance mem cost %s exceeded per-host upper bound %s.",
+          PrintUtils.printBytes(perInstanceMemEstimate),
           PrintUtils.printBytes(perHostUpperBound)));
-      perHostMemCost_ = perHostUpperBound;
+      perInstanceMemEstimate = perHostUpperBound;
     }
+    resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0);
   }
 
   /**
@@ -873,9 +874,9 @@ public class HdfsScanNode extends ScanNode {
    */
   public static long getPerHostMemUpperBound() {
     // THREADS_PER_CORE each using a default of
-    // MAX_IO_BUFFERS_PER_THREAD * IO_MGR_BUFFER_SIZE bytes.
-    return (long) RuntimeEnv.INSTANCE.getNumCores() * (long) THREADS_PER_CORE *
-        MAX_IO_BUFFERS_PER_THREAD * IO_MGR_BUFFER_SIZE;
+    // MAX_IO_BUFFERS_PER_THREAD * read_size bytes.
+    return (long) RuntimeEnv.INSTANCE.getNumCores() * (long) MAX_THREAD_TOKENS_PER_CORE *
+        MAX_IO_BUFFERS_PER_THREAD * BackendConfig.INSTANCE.getReadSize();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
----------------------------------------------------------------------
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 996f981..8dc9f62 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsTableSink.java
@@ -24,11 +24,11 @@ import org.apache.impala.analysis.Expr;
 import org.apache.impala.catalog.HdfsFileFormat;
 import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.Table;
-import org.apache.impala.common.PrintUtils;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.THdfsTableSink;
+import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TTableSink;
 import org.apache.impala.thrift.TTableSinkType;
 import com.google.common.base.Preconditions;
@@ -39,7 +39,8 @@ import com.google.common.collect.Lists;
  *
  */
 public class HdfsTableSink extends TableSink {
-  // Default number of partitions used for computeCosts() in the absence of column stats.
+  // Default number of partitions used for computeResourceProfile() in the absence of
+  // column stats.
   protected final long DEFAULT_NUM_PARTITIONS = 10;
 
   // Exprs for computing the output partition(s).
@@ -67,31 +68,37 @@ public class HdfsTableSink extends TableSink {
   }
 
   @Override
-  public void computeCosts() {
+  public void computeResourceProfile(TQueryOptions queryOptions) {
     HdfsTable table = (HdfsTable) targetTable_;
     // TODO: Estimate the memory requirements more accurately by partition type.
     HdfsFileFormat format = table.getMajorityFormat();
     PlanNode inputNode = fragment_.getPlanRoot();
-    int numNodes = fragment_.getNumNodes();
-    // Compute the per-host number of partitions, taking the number of nodes
+    int numInstances = fragment_.getNumInstances(queryOptions.getMt_dop());
+    // Compute the per-instance number of partitions, taking the number of nodes
     // and the data partition of the fragment executing this sink into account.
-    long numPartitions = fragment_.getNumDistinctValues(partitionKeyExprs_);
-    if (numPartitions == -1) numPartitions = DEFAULT_NUM_PARTITIONS;
+    long numPartitionsPerInstance =
+        fragment_.getPerInstanceNdv(queryOptions.getMt_dop(), partitionKeyExprs_);
+    if (numPartitionsPerInstance == -1) {
+      numPartitionsPerInstance = DEFAULT_NUM_PARTITIONS;
+    }
     long perPartitionMemReq = getPerPartitionMemReq(format);
 
+    long perInstanceMemEstimate;
     // The estimate is based purely on the per-partition mem req if the input cardinality_
     // or the avg row size is unknown.
     if (inputNode.getCardinality() == -1 || inputNode.getAvgRowSize() == -1) {
-      perHostMemCost_ = numPartitions * perPartitionMemReq;
-      return;
+      perInstanceMemEstimate = numPartitionsPerInstance * perPartitionMemReq;
+    } else {
+      // The per-partition estimate may be higher than the memory required to buffer
+      // the entire input data.
+      long perInstanceInputCardinality =
+          Math.max(1L, inputNode.getCardinality() / numInstances);
+      long perInstanceInputBytes =
+          (long) Math.ceil(perInstanceInputCardinality * inputNode.getAvgRowSize());
+      perInstanceMemEstimate =
+          Math.min(perInstanceInputBytes, numPartitionsPerInstance * perPartitionMemReq);
     }
-
-    // The per-partition estimate may be higher than the memory required to buffer
-    // the entire input data.
-    long perHostInputCardinality = Math.max(1L, inputNode.getCardinality() / numNodes);
-    long perHostInputBytes =
-        (long) Math.ceil(perHostInputCardinality * inputNode.getAvgRowSize());
-    perHostMemCost_ = Math.min(perHostInputBytes, numPartitions * perPartitionMemReq);
+    resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0);
   }
 
   /**
@@ -100,10 +107,19 @@ public class HdfsTableSink extends TableSink {
    */
   private long getPerPartitionMemReq(HdfsFileFormat format) {
     switch (format) {
-      // Writing to a Parquet table requires up to 1GB of buffer per partition.
-      // TODO: The per-partition memory requirement is configurable in the QueryOptions.
-      case PARQUET: return 1024L * 1024L * 1024L;
-      case TEXT: return 100L * 1024L;
+      case PARQUET:
+        // Writing to a Parquet table requires up to 1GB of buffer per partition.
+        // TODO: The per-partition memory requirement is configurable in the QueryOptions.
+        return 1024L * 1024L * 1024L;
+      case TEXT:
+      case LZO_TEXT:
+        // Very approximate estimate of amount of data buffered.
+        return 100L * 1024L;
+      case RC_FILE:
+      case SEQUENCE_FILE:
+      case AVRO:
+        // Very approximate estimate of amount of data buffered.
+        return 100L * 1024L;
       default:
         Preconditions.checkState(false, "Unsupported TableSink format " +
             format.toString());
@@ -112,9 +128,8 @@ public class HdfsTableSink extends TableSink {
   }
 
   @Override
-  public String getExplainString(String prefix, String detailPrefix,
-      TExplainLevel explainLevel) {
-    StringBuilder output = new StringBuilder();
+  public void appendSinkExplainString(String prefix, String detailPrefix,
+      TQueryOptions queryOptions, TExplainLevel explainLevel, StringBuilder output) {
     String overwriteStr = ", OVERWRITE=" + (overwrite_ ? "true" : "false");
     String partitionKeyStr = "";
     if (!partitionKeyExprs_.isEmpty()) {
@@ -139,13 +154,7 @@ public class HdfsTableSink extends TableSink {
             + (totalNumPartitions == 0 ? 1 : totalNumPartitions));
       }
       output.append("\n");
-      if (explainLevel.ordinal() >= TExplainLevel.EXTENDED.ordinal()) {
-        output.append(PrintUtils.printHosts(detailPrefix, fragment_.getNumNodes()));
-        output.append(PrintUtils.printMemCost(" ", perHostMemCost_));
-        output.append("\n");
-      }
     }
-    return output.toString();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
----------------------------------------------------------------------
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 7afced6..69cc133 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinBuildSink.java
@@ -19,21 +19,13 @@ package org.apache.impala.planner;
 
 import java.util.List;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.Expr;
-import org.apache.impala.common.ImpalaException;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TJoinBuildSink;
-import org.apache.impala.thrift.TPlanNode;
-import org.apache.impala.thrift.TPlanNodeType;
 import org.apache.impala.thrift.TQueryOptions;
-import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
@@ -77,9 +69,8 @@ public class JoinBuildSink extends DataSink {
   }
 
   @Override
-  public String getExplainString(String prefix, String detailPrefix,
-      TExplainLevel detailLevel) {
-    StringBuilder output = new StringBuilder();
+  public void appendSinkExplainString(String prefix, String detailPrefix,
+      TQueryOptions queryOptions, TExplainLevel detailLevel, StringBuilder output) {
     output.append(String.format("%s%s\n", prefix, "JOIN BUILD"));
     if (detailLevel.ordinal() > TExplainLevel.MINIMAL.ordinal()) {
       output.append(
@@ -91,11 +82,11 @@ public class JoinBuildSink extends DataSink {
             .append(Expr.toSql(buildExprs_) + "\n");
       }
     }
-    return output.toString();
   }
 
   @Override
-  public void computeCosts() {
-    // TODO: implement?
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // The memory consumption is counted against the join PlanNode.
+    resourceProfile_ = new ResourceProfile(0, 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/JoinNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/JoinNode.java b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
index 8e963ff..b40ef55 100644
--- a/fe/src/main/java/org/apache/impala/planner/JoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/JoinNode.java
@@ -41,9 +41,9 @@ import com.google.common.base.Preconditions;
 public abstract class JoinNode extends PlanNode {
   private final static Logger LOG = LoggerFactory.getLogger(JoinNode.class);
 
-  // Default per-host memory requirement used if no valid stats are available.
+  // Default per-instance memory requirement used if no valid stats are available.
   // TODO: Come up with a more useful heuristic (e.g., based on scanned partitions).
-  protected final static long DEFAULT_PER_HOST_MEM = 2L * 1024L * 1024L * 1024L;
+  protected final static long DEFAULT_PER_INSTANCE_MEM = 2L * 1024L * 1024L * 1024L;
 
   // Slop in percent allowed when comparing stats for the purpose of determining whether
   // an equi-join condition is a foreign/primary key join.
@@ -153,6 +153,8 @@ public abstract class JoinNode extends PlanNode {
   public void setDistributionMode(DistributionMode distrMode) { distrMode_ = distrMode; }
   public JoinTableId getJoinTableId() { return joinTableId_; }
   public void setJoinTableId(JoinTableId id) { joinTableId_ = id; }
+  /// True if this consumes all of its right input before outputting any rows.
+  abstract public boolean isBlockingJoinNode();
 
   @Override
   public void init(Analyzer analyzer) throws ImpalaException {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
----------------------------------------------------------------------
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 64e5fde..bc3cdc0 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -43,6 +43,7 @@ import org.apache.impala.thrift.TKuduScanNode;
 import org.apache.impala.thrift.TNetworkAddress;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
+import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TScanRange;
 import org.apache.impala.thrift.TScanRangeLocation;
 import org.apache.impala.thrift.TScanRangeLocationList;
@@ -275,6 +276,11 @@ public class KuduScanNode extends ScanNode {
   }
 
   @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    resourceProfile_ = new ResourceProfile(0, 0);
+  }
+
+  @Override
   protected String getNodeExplainString(String prefix, String detailPrefix,
       TExplainLevel detailLevel) {
     StringBuilder result = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java
----------------------------------------------------------------------
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 35b9022..b7dcdd8 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduTableSink.java
@@ -23,11 +23,11 @@ import java.util.List;
 
 import org.apache.impala.analysis.DescriptorTable;
 import org.apache.impala.catalog.Table;
-import org.apache.impala.common.PrintUtils;
 import org.apache.impala.thrift.TDataSink;
 import org.apache.impala.thrift.TDataSinkType;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TKuduTableSink;
+import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.thrift.TTableSink;
 import org.apache.impala.thrift.TTableSinkType;
 
@@ -51,17 +51,16 @@ public class KuduTableSink extends TableSink {
   }
 
   @Override
-  public String getExplainString(String prefix, String detailPrefix,
-      TExplainLevel explainLevel) {
-    StringBuilder output = new StringBuilder();
+  public void appendSinkExplainString(String prefix, String detailPrefix,
+      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(PrintUtils.printHosts(detailPrefix, fragment_.getNumNodes()));
-      output.append(PrintUtils.printMemCost(" ", perHostMemCost_));
-      output.append("\n");
-    }
-    return output.toString();
+  }
+
+  @Override
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    // TODO: add a memory estimate
+    resourceProfile_ = new ResourceProfile(0, 0);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
----------------------------------------------------------------------
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 0213c8b..e69f97b 100644
--- a/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/NestedLoopJoinNode.java
@@ -58,6 +58,9 @@ public class NestedLoopJoinNode extends JoinNode {
   }
 
   @Override
+  public boolean isBlockingJoinNode() { return true; }
+
+  @Override
   public void init(Analyzer analyzer) throws ImpalaException {
     super.init(analyzer);
     Preconditions.checkState(eqJoinConjuncts_.isEmpty());
@@ -74,14 +77,16 @@ public class NestedLoopJoinNode extends JoinNode {
   }
 
   @Override
-  public void computeCosts(TQueryOptions queryOptions) {
+  public void computeResourceProfile(TQueryOptions queryOptions) {
+    long perInstanceMemEstimate;
     if (getChild(1).getCardinality() == -1 || getChild(1).getAvgRowSize() == -1
         || numNodes_ == 0) {
-      perHostMemCost_ = DEFAULT_PER_HOST_MEM;
-      return;
+      perInstanceMemEstimate = DEFAULT_PER_INSTANCE_MEM;
+    } else {
+      perInstanceMemEstimate =
+          (long) Math.ceil(getChild(1).cardinality_ * getChild(1).avgRowSize_);
     }
-    perHostMemCost_ =
-        (long) Math.ceil(getChild(1).cardinality_ * getChild(1).avgRowSize_);
+    resourceProfile_ = new ResourceProfile(perInstanceMemEstimate, 0);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java b/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
index 8f2a1a4..f5cc5d8 100644
--- a/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/ParallelPlanner.java
@@ -17,18 +17,13 @@
 
 package org.apache.impala.planner;
 
-import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.impala.common.IdGenerator;
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 
 /**
  * The parallel planner is responsible for breaking up a single distributed plan
@@ -183,6 +178,9 @@ public class ParallelPlanner {
       buildFragment.getChildren().add(inputFragments.get(i));
     }
 
+    // compute the resource profile for the newly-added build sink.
+    buildSink.computeResourceProfile(ctx_.getQueryOptions());
+
     // assign plan and cohort id
     buildFragment.setPlanId(planIdGenerator_.getNextId());
     PlanId parentPlanId = join.getFragment().getPlanId();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/PipelinedPlanNodeSet.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/PipelinedPlanNodeSet.java b/fe/src/main/java/org/apache/impala/planner/PipelinedPlanNodeSet.java
index 249987a..c2ae0fd 100644
--- a/fe/src/main/java/org/apache/impala/planner/PipelinedPlanNodeSet.java
+++ b/fe/src/main/java/org/apache/impala/planner/PipelinedPlanNodeSet.java
@@ -38,7 +38,7 @@ import com.google.common.collect.Sets;
  * set contains all build-side nodes. The second set contains the leftmost
  * scan. Both sets contain all join nodes because they execute and consume
  * resources during the build and probe phases. Similarly, all nodes below a 'blocking'
- * node (e.g, an AggregationNode) are placed into a differnet plan node set than the
+ * node (e.g, an AggregationNode) are placed into a different plan node set than the
  * nodes above it, but the blocking node itself belongs to both sets.
  */
 public class PipelinedPlanNodeSet {
@@ -46,39 +46,32 @@ public class PipelinedPlanNodeSet {
 
   // Minimum per-host resource requirements to ensure that no plan node set can have
   // estimates of zero, even if the contained PlanNodes have estimates of zero.
-  public static final long MIN_PER_HOST_MEM = 10 * 1024 * 1024;
-  public static final int MIN_PER_HOST_VCORES = 1;
+  public static final long MIN_PER_HOST_MEM_ESTIMATE_BYTES = 10 * 1024 * 1024;
 
   // List of plan nodes that execute and consume resources concurrently.
-  private final ArrayList<PlanNode> planNodes = Lists.newArrayList();
+  private final ArrayList<PlanNode> planNodes_ = Lists.newArrayList();
 
   // DataSinks that execute and consume resources concurrently.
   // Primarily used for estimating the cost of insert queries.
-  private final List<DataSink> dataSinks = Lists.newArrayList();
+  private final List<DataSink> dataSinks_ = Lists.newArrayList();
 
-  // Estimated per-host memory and CPU requirements.
-  // Valid after computeResourceEstimates().
-  private long perHostMem = MIN_PER_HOST_MEM;
-  private int perHostVcores = MIN_PER_HOST_VCORES;
-
-  public void add(PlanNode node) {
+  private void addNode(PlanNode node) {
     Preconditions.checkNotNull(node.getFragment());
-    planNodes.add(node);
+    planNodes_.add(node);
   }
 
-  public void addSink(DataSink sink) {
+  private void addSink(DataSink sink) {
     Preconditions.checkNotNull(sink);
-    dataSinks.add(sink);
+    dataSinks_.add(sink);
   }
 
   /**
-   * Computes the estimated per-host memory and CPU requirements of this plan node set.
-   * Optionally excludes unpartitioned fragments from the estimation.
-   * Returns true if at least one plan node was included in the estimation.
-   * Otherwise returns false indicating the estimates are invalid.
+   * Computes the per-host resource profile of this plan node set.
+   *
+   * If there are no nodes included in the estimate, the returned estimate will not be
+   * valid.
    */
-  public boolean computeResourceEstimates(boolean excludeUnpartitionedFragments,
-      TQueryOptions queryOptions) {
+  public ResourceProfile computePerHostResources(TQueryOptions queryOptions) {
     Set<PlanFragment> uniqueFragments = Sets.newHashSet();
 
     // Distinguish the per-host memory estimates for scan nodes and non-scan nodes to
@@ -86,74 +79,66 @@ public class PipelinedPlanNodeSet {
     // scans. The memory required by all concurrent scans of the same type (Hdfs/Hbase)
     // cannot exceed the per-host upper memory bound for that scan type. Intuitively,
     // the amount of I/O buffers is limited by the disk bandwidth.
-    long perHostHbaseScanMem = 0L;
-    long perHostHdfsScanMem = 0L;
-    long perHostNonScanMem = 0L;
+    long hbaseScanMemEstimate = 0L;
+    long hdfsScanMemEstimate = 0L;
+    long nonScanMemEstimate = 0L;
+    long minReservationBytes = 0L;
+    int numNodesIncluded = 0;
 
-    for (int i = 0; i < planNodes.size(); ++i) {
-      PlanNode node = planNodes.get(i);
+    for (PlanNode node : planNodes_) {
       PlanFragment fragment = node.getFragment();
-      if (!fragment.isPartitioned() && excludeUnpartitionedFragments) continue;
-      node.computeCosts(queryOptions);
+      // Multiple instances of a partitioned fragment may execute per host
+      int instancesPerHost = fragment.getNumInstancesPerHost(queryOptions.getMt_dop());
+
+      ResourceProfile nodeProfile = node.getResourceProfile();
+      Preconditions.checkState(nodeProfile.getMemEstimateBytes() >= 0);
+      long memEstimate = instancesPerHost * nodeProfile.getMemEstimateBytes();
+      ++numNodesIncluded;
       uniqueFragments.add(fragment);
-      if (node.getPerHostMemCost() < 0) {
-        LOG.warn(String.format("Invalid per-host memory requirement %s of node %s.\n" +
-            "PlanNode stats are: numNodes_=%s ", node.getPerHostMemCost(),
-            node.getClass().getSimpleName(), node.getNumNodes()));
-      }
       if (node instanceof HBaseScanNode) {
-        perHostHbaseScanMem += node.getPerHostMemCost();
+        hbaseScanMemEstimate += memEstimate;
       } else if (node instanceof HdfsScanNode) {
-        perHostHdfsScanMem += node.getPerHostMemCost();
+        hdfsScanMemEstimate += memEstimate;
       } else {
-        perHostNonScanMem += node.getPerHostMemCost();
+        nonScanMemEstimate += memEstimate;
       }
+      Preconditions.checkState(nodeProfile.getMinReservationBytes() >= 0);
+      minReservationBytes += instancesPerHost * nodeProfile.getMinReservationBytes();
+    }
+
+    if (queryOptions.getMt_dop() == 0) {
+      // The thread tokens for the non-MT path impose a limit on the memory that can
+      // be consumed by concurrent scans.
+      hbaseScanMemEstimate =
+          Math.min(hbaseScanMemEstimate, HBaseScanNode.getPerHostMemUpperBound());
+      hdfsScanMemEstimate =
+          Math.min(hdfsScanMemEstimate, HdfsScanNode.getPerHostMemUpperBound());
     }
 
-    // The memory required by concurrent scans cannot exceed the upper memory bound
-    // for that scan type.
-    // TODO: In the future, we may want to restrict scanner concurrency based on a
-    // memory limit. This estimation will need to accoung for that as well.
-    perHostHbaseScanMem =
-        Math.min(perHostHbaseScanMem, HBaseScanNode.getPerHostMemUpperBound());
-    perHostHdfsScanMem =
-        Math.min(perHostHdfsScanMem, HdfsScanNode.getPerHostMemUpperBound());
-
-    long perHostDataSinkMem = 0L;
-    for (int i = 0; i < dataSinks.size(); ++i) {
-      DataSink sink = dataSinks.get(i);
+    long dataSinkMemEstimate = 0L;
+    for (DataSink sink: dataSinks_) {
       PlanFragment fragment = sink.getFragment();
-      if (!fragment.isPartitioned() && excludeUnpartitionedFragments) continue;
       // Sanity check that this plan-node set has at least one PlanNode of fragment.
       Preconditions.checkState(uniqueFragments.contains(fragment));
-      sink.computeCosts();
-      if (sink.getPerHostMemCost() < 0) {
-        LOG.warn(String.format("Invalid per-host memory requirement %s of sink %s.\n",
-            sink.getPerHostMemCost(), sink.getClass().getSimpleName()));
-      }
-      perHostDataSinkMem += sink.getPerHostMemCost();
+      int instancesPerHost = fragment.getNumInstancesPerHost(queryOptions.getMt_dop());
+
+      ResourceProfile sinkProfile = sink.getResourceProfile();
+      Preconditions.checkState(sinkProfile.getMemEstimateBytes() >= 0);
+      dataSinkMemEstimate += instancesPerHost * sinkProfile.getMemEstimateBytes();
+      Preconditions.checkState(sinkProfile.getMinReservationBytes() >= 0);
+      minReservationBytes += instancesPerHost * sinkProfile.getMinReservationBytes();
     }
 
     // Combine the memory estimates of all sinks, scans nodes and non-scan nodes.
-    long perHostMem = perHostHdfsScanMem + perHostHbaseScanMem + perHostNonScanMem +
-        perHostDataSinkMem;
-
-    // The backend needs at least one thread per fragment.
-    int perHostVcores = uniqueFragments.size();
-
-    // This plan node set might only have unpartitioned fragments.
-    // Only set estimates if they are valid.
-    if (perHostMem >= 0 && perHostVcores >= 0) {
-      this.perHostMem = perHostMem;
-      this.perHostVcores = perHostVcores;
-      return true;
-    }
-    return false;
+    long perHostMemEstimate =
+        Math.max(MIN_PER_HOST_MEM_ESTIMATE_BYTES, hdfsScanMemEstimate
+                + hbaseScanMemEstimate + nonScanMemEstimate + dataSinkMemEstimate);
+    // This plan node set might only have unpartitioned fragments and be invalid.
+    return numNodesIncluded > 0 ?
+        new ResourceProfile(perHostMemEstimate, minReservationBytes) :
+          ResourceProfile.invalid();
   }
 
-  public long getPerHostMem() { return perHostMem; }
-  public int getPerHostVcores() { return perHostVcores; }
-
   /**
    * Computes and returns the pipelined plan node sets of the given plan.
    */
@@ -175,19 +160,19 @@ public class PipelinedPlanNodeSet {
    */
   private static void computePlanNodeSets(PlanNode node, PipelinedPlanNodeSet lhsSet,
       PipelinedPlanNodeSet rhsSet, ArrayList<PipelinedPlanNodeSet> planNodeSets) {
-    lhsSet.add(node);
+    lhsSet.addNode(node);
     if (node == node.getFragment().getPlanRoot() && node.getFragment().hasSink()) {
       lhsSet.addSink(node.getFragment().getSink());
     }
 
-    if (node instanceof HashJoinNode) {
+    if (node instanceof JoinNode && ((JoinNode)node).isBlockingJoinNode()) {
       // Create a new set for the right-hand sides of joins if necessary.
       if (rhsSet == null) {
         rhsSet = new PipelinedPlanNodeSet();
         planNodeSets.add(rhsSet);
       }
       // The join node itself is added to the lhsSet (above) and the rhsSet.
-      rhsSet.add(node);
+      rhsSet.addNode(node);
       computePlanNodeSets(node.getChild(1), rhsSet, null, planNodeSets);
       computePlanNodeSets(node.getChild(0), lhsSet, rhsSet, planNodeSets);
       return;
@@ -197,8 +182,10 @@ public class PipelinedPlanNodeSet {
       // We add blocking nodes to two plan node sets because they require resources while
       // consuming their input (execution of the preceding set) and while they
       // emit their output (execution of the following set).
+      // TODO: IMPALA-4862: this logic does not accurately reflect the behaviour of
+      // concurrent join builds in the backend
       lhsSet = new PipelinedPlanNodeSet();
-      lhsSet.add(node);
+      lhsSet.addNode(node);
       planNodeSets.add(lhsSet);
       // Join builds under this blocking node belong in a new rhsSet.
       rhsSet = null;
@@ -207,7 +194,9 @@ public class PipelinedPlanNodeSet {
     // Assume that non-join, non-blocking nodes with multiple children
     // (e.g., ExchangeNodes) consume their inputs in an arbitrary order,
     // i.e., all child subtrees execute concurrently.
-    // TODO: This is not true for UnionNodes anymore. Fix the estimates accordingly.
+    // TODO: IMPALA-4862: can overestimate resource consumption of UnionNodes - the
+    // execution of union branches is serialised within a fragment (but not across
+    // fragment boundaries).
     for (PlanNode child: node.getChildren()) {
       computePlanNodeSets(child, lhsSet, rhsSet, planNodeSets);
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
----------------------------------------------------------------------
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 e4888f9..3e89137 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -26,11 +26,13 @@ import org.apache.impala.analysis.TupleId;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.common.NotImplementedException;
+import org.apache.impala.common.PrintUtils;
 import org.apache.impala.common.TreeNode;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPartitionType;
 import org.apache.impala.thrift.TPlanFragment;
 import org.apache.impala.thrift.TPlanFragmentTree;
+import org.apache.impala.thrift.TQueryOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -74,6 +76,8 @@ import com.google.common.collect.Sets;
  *   fix that
  */
 public class PlanFragment extends TreeNode<PlanFragment> {
+  private final static Logger LOG = LoggerFactory.getLogger(PlanFragment.class);
+
   private final PlanFragmentId fragmentId_;
   private PlanId planId_;
   private CohortId cohortId_;
@@ -145,6 +149,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
 
   /**
    * Finalize plan tree and create stream sink, if needed.
+   * Computes resource profiles for all nodes and sinks in this fragment.
    * If this fragment is hash partitioned, ensures that the corresponding partition
    * exprs of all hash-partitioning senders are cast to identical types.
    * Otherwise, the hashes generated for identical partition values may differ
@@ -159,6 +164,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
       streamSink.setFragment(this);
       sink_ = streamSink;
     }
+    computeResourceProfile(analyzer);
 
     if (!dataPartition_.isHashPartitioned()) return;
 
@@ -196,6 +202,19 @@ public class PlanFragment extends TreeNode<PlanFragment> {
   }
 
   /**
+   * Compute the resource profile of the fragment. Must be called after all the
+   * plan nodes and sinks are added to the fragment.
+   */
+  private void computeResourceProfile(Analyzer analyzer) {
+    sink_.computeResourceProfile(analyzer.getQueryOptions());
+    List<PlanNode> nodes = Lists.newArrayList();
+    collectPlanNodes(nodes);
+    for (PlanNode node: nodes) {
+      node.computeResourceProfile(analyzer.getQueryOptions());
+    }
+  }
+
+  /**
    * Return the number of nodes on which the plan fragment will execute.
    * invalid: -1
    */
@@ -203,18 +222,40 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     return dataPartition_ == DataPartition.UNPARTITIONED ? 1 : planRoot_.getNumNodes();
   }
 
- /**
-   * Estimates the per-node number of distinct values of exprs based on the data
-   * partition of this fragment and its number of nodes. Returns -1 for an invalid
-   * estimate, e.g., because getNumDistinctValues() failed on one of the exprs.
+  /**
+   * Return the number of instances of this fragment per host that it executes on.
+   * invalid: -1
    */
-  public long getNumDistinctValues(List<Expr> exprs) {
+  public int getNumInstancesPerHost(int mt_dop) {
+    Preconditions.checkState(mt_dop >= 0);
+    if (dataPartition_ == DataPartition.UNPARTITIONED) return 1;
+    return mt_dop == 0 ? 1 : mt_dop;
+  }
+
+  /**
+   * Return the total number of instances of this fragment across all hosts.
+   * invalid: -1
+   */
+  public int getNumInstances(int mt_dop) {
+    if (dataPartition_ == DataPartition.UNPARTITIONED) return 1;
+    int numNodes = planRoot_.getNumNodes();
+    if (numNodes == -1) return -1;
+    return getNumInstancesPerHost(mt_dop) * numNodes;
+  }
+
+  /**
+    * Estimates the number of distinct values of exprs per fragment instance based on the
+    * data partition of this fragment, the number of nodes, and the degree of parallelism.
+    * Returns -1 for an invalid estimate, e.g., because getNumDistinctValues() failed on
+    * one of the exprs.
+    */
+  public long getPerInstanceNdv(int mt_dop, List<Expr> exprs) {
     Preconditions.checkNotNull(dataPartition_);
     long result = 1;
-    int numNodes = getNumNodes();
-    Preconditions.checkState(numNodes >= 0);
+    int numInstances = getNumInstances(mt_dop);
+    Preconditions.checkState(numInstances >= 0);
     // The number of nodes is zero for empty tables.
-    if (numNodes == 0) return 0;
+    if (numInstances == 0) return 0;
     for (Expr expr: exprs) {
       long numDistinct = expr.getNumDistinctValues();
       if (numDistinct == -1) {
@@ -222,7 +263,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
         break;
       }
       if (dataPartition_.getPartitionExprs().contains(expr)) {
-        numDistinct = (long)Math.max((double) numDistinct / (double) numNodes, 1L);
+        numDistinct = (long)Math.max((double) numDistinct / (double) numInstances, 1L);
       }
       result = PlanNode.multiplyCardinalities(result, numDistinct);
     }
@@ -254,8 +295,8 @@ public class PlanFragment extends TreeNode<PlanFragment> {
     }
   }
 
-  public String getExplainString(TExplainLevel detailLevel) {
-    return getExplainString("", "", detailLevel);
+  public String getExplainString(TQueryOptions queryOptions, TExplainLevel detailLevel) {
+    return getExplainString("", "", queryOptions, detailLevel);
   }
 
   /**
@@ -263,7 +304,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
    * output will be prefixed by prefix.
    */
   protected final String getExplainString(String rootPrefix, String prefix,
-      TExplainLevel detailLevel) {
+      TQueryOptions queryOptions, TExplainLevel detailLevel) {
     StringBuilder str = new StringBuilder();
     Preconditions.checkState(dataPartition_ != null);
     String detailPrefix = prefix + "|  ";  // sink detail
@@ -272,17 +313,25 @@ public class PlanFragment extends TreeNode<PlanFragment> {
       prefix = "  ";
       rootPrefix = "  ";
       detailPrefix = prefix + "|  ";
-      str.append(String.format("%s:PLAN FRAGMENT [%s]\n", fragmentId_.toString(),
-          dataPartition_.getExplainString()));
+      str.append(getFragmentHeaderString(queryOptions.getMt_dop()));
+      str.append("\n");
       if (sink_ != null && sink_ instanceof DataStreamSink) {
-        str.append(sink_.getExplainString(rootPrefix, prefix, detailLevel) + "\n");
+        str.append(
+            sink_.getExplainString(rootPrefix, detailPrefix, queryOptions, detailLevel));
       }
+    } else if (detailLevel == TExplainLevel.EXTENDED) {
+      // Print a fragment prefix displaying the # nodes and # instances
+      str.append(rootPrefix);
+      str.append(getFragmentHeaderString(queryOptions.getMt_dop()));
+      str.append("\n");
+      rootPrefix = prefix;
     }
 
     String planRootPrefix = rootPrefix;
     // Always print sinks other than DataStreamSinks.
     if (sink_ != null && !(sink_ instanceof DataStreamSink)) {
-      str.append(sink_.getExplainString(rootPrefix, detailPrefix, detailLevel));
+      str.append(
+          sink_.getExplainString(rootPrefix, detailPrefix, queryOptions, detailLevel));
       if (detailLevel.ordinal() >= TExplainLevel.STANDARD.ordinal()) {
         str.append(prefix + "|\n");
       }
@@ -290,11 +339,24 @@ public class PlanFragment extends TreeNode<PlanFragment> {
       planRootPrefix = prefix;
     }
     if (planRoot_ != null) {
-      str.append(planRoot_.getExplainString(planRootPrefix, prefix, detailLevel));
+      str.append(
+          planRoot_.getExplainString(planRootPrefix, prefix, queryOptions, detailLevel));
     }
     return str.toString();
   }
 
+  /**
+   * Get a header string for a fragment in an explain plan.
+   */
+  public String getFragmentHeaderString(int mt_dop) {
+    StringBuilder builder = new StringBuilder();
+    builder.append(String.format("%s:PLAN FRAGMENT [%s]", fragmentId_.toString(),
+        dataPartition_.getExplainString()));
+    builder.append(PrintUtils.printNumHosts(" ", getNumNodes()));
+    builder.append(PrintUtils.printNumInstances(" ", getNumInstances(mt_dop)));
+    return builder.toString();
+  }
+
   /** Returns true if this fragment is partitioned. */
   public boolean isPartitioned() {
     return (dataPartition_.getType() != TPartitionType.UNPARTITIONED);


[4/6] incubator-impala git commit: IMPALA-3748: minimum buffer requirements in planner

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
new file mode 100644
index 0000000..7ae30fa
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -0,0 +1,1121 @@
+# Parquet scan
+select * from tpch_parquet.lineitem
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=80.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=160.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# Text scan
+select * from tpch.lineitem;
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=88.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=176.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# HBase scan
+select * from functional_hbase.alltypes
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=1.00GB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_hbase.alltypes
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=88B cardinality=14298
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=100 instances=100
+00:SCAN HBASE [functional_hbase.alltypes]
+   table stats: unavailable
+   column stats: unavailable
+   mem-estimate=1.00GB mem-reservation=0B
+   tuple-ids=0 row-size=88B cardinality=14298
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=2.00GB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_hbase.alltypes
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=88B cardinality=14298
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=100 instances=200
+00:SCAN HBASE [functional_hbase.alltypes]
+   table stats: unavailable
+   column stats: unavailable
+   mem-estimate=1.00GB mem-reservation=0B
+   tuple-ids=0 row-size=88B cardinality=14298
+====
+# Data source scan
+select * from functional.alltypes_datasource
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=1.00GB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional.alltypes_datasource
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=116B cardinality=5000
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+00:SCAN DATA SOURCE [functional.alltypes_datasource]
+   mem-estimate=1.00GB mem-reservation=0B
+   tuple-ids=0 row-size=116B cardinality=5000
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=2.00GB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional.alltypes_datasource
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=116B cardinality=5000
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+00:SCAN DATA SOURCE [functional.alltypes_datasource]
+   mem-estimate=1.00GB mem-reservation=0B
+   tuple-ids=0 row-size=116B cardinality=5000
+====
+# Union
+select * from tpch.lineitem
+union all
+select * from tpch.lineitem
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=176.00MB
+
+F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+03:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=2 row-size=263B cardinality=12002430
+|
+F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+00:UNION
+|  pass-through-operands: all
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=2 row-size=263B cardinality=12002430
+|
+|--02:SCAN HDFS [tpch.lineitem, RANDOM]
+|     partitions=1/1 files=1 size=718.94MB
+|     table stats: 6001215 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=1 row-size=263B cardinality=6001215
+|
+01:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=352.00MB
+
+F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+03:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=2 row-size=263B cardinality=12002430
+|
+F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+00:UNION
+|  pass-through-operands: all
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=2 row-size=263B cardinality=12002430
+|
+|--02:SCAN HDFS [tpch.lineitem, RANDOM]
+|     partitions=1/1 files=1 size=718.94MB
+|     table stats: 6001215 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=1 row-size=263B cardinality=6001215
+|
+01:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# Grouping aggregation
+select l_orderkey, count(*)
+from tpch_parquet.lineitem
+group by l_orderkey
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=264.00MB
+Per-Host Resource Estimates: Memory=116.24MB
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=16B cardinality=1563438
+|
+F01:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
+03:AGGREGATE [FINALIZE]
+|  output: count:merge(*)
+|  group by: l_orderkey
+|  mem-estimate=10.00MB mem-reservation=264.00MB
+|  tuple-ids=1 row-size=16B cardinality=1563438
+|
+02:EXCHANGE [HASH(l_orderkey)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=16B cardinality=1563438
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+01:AGGREGATE [STREAMING]
+|  output: count(*)
+|  group by: l_orderkey
+|  mem-estimate=26.24MB mem-reservation=0B
+|  tuple-ids=1 row-size=16B cardinality=1563438
+|
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=8B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=528.00MB
+Per-Host Resource Estimates: Memory=232.48MB
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=16B cardinality=1563438
+|
+F01:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
+03:AGGREGATE [FINALIZE]
+|  output: count:merge(*)
+|  group by: l_orderkey
+|  mem-estimate=10.00MB mem-reservation=264.00MB
+|  tuple-ids=1 row-size=16B cardinality=1563438
+|
+02:EXCHANGE [HASH(l_orderkey)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=16B cardinality=1563438
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+01:AGGREGATE [STREAMING]
+|  output: count(*)
+|  group by: l_orderkey
+|  mem-estimate=26.24MB mem-reservation=0B
+|  tuple-ids=1 row-size=16B cardinality=1563438
+|
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=8B cardinality=6001215
+====
+# Non-grouping aggregation with zero-slot parquet scan
+select count(*) from tpch_parquet.lineitem
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=20.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+03:AGGREGATE [FINALIZE]
+|  output: count:merge(*)
+|  mem-estimate=10.00MB mem-reservation=0B
+|  tuple-ids=1 row-size=8B cardinality=1
+|
+02:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=8B cardinality=1
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+01:AGGREGATE
+|  output: count(*)
+|  mem-estimate=10.00MB mem-reservation=0B
+|  tuple-ids=1 row-size=8B cardinality=1
+|
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=0B mem-reservation=0B
+   tuple-ids=0 row-size=0B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=180.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+03:AGGREGATE [FINALIZE]
+|  output: count:merge(*)
+|  mem-estimate=10.00MB mem-reservation=0B
+|  tuple-ids=1 row-size=8B cardinality=1
+|
+02:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=8B cardinality=1
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+01:AGGREGATE
+|  output: count(*)
+|  mem-estimate=10.00MB mem-reservation=0B
+|  tuple-ids=1 row-size=8B cardinality=1
+|
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=0B cardinality=6001215
+====
+# Sort
+select *
+from tpch_parquet.lineitem
+order by l_comment
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=48.00MB
+Per-Host Resource Estimates: Memory=240.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+02:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: l_comment ASC
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+01:SORT
+|  order by: l_comment ASC
+|  mem-estimate=160.00MB mem-reservation=48.00MB
+|  tuple-ids=1 row-size=263B cardinality=6001215
+|
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=96.00MB
+Per-Host Resource Estimates: Memory=480.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+02:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: l_comment ASC
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+01:SORT
+|  order by: l_comment ASC
+|  mem-estimate=160.00MB mem-reservation=48.00MB
+|  tuple-ids=1 row-size=263B cardinality=6001215
+|
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# TOP-N
+select *
+from tpch_parquet.lineitem
+order by l_comment
+limit 100
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=80.03MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+02:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: l_comment ASC
+|  limit: 100
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=263B cardinality=100
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+01:TOP-N [LIMIT=100]
+|  order by: l_comment ASC
+|  mem-estimate=25.66KB mem-reservation=0B
+|  tuple-ids=1 row-size=263B cardinality=100
+|
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=160.05MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+02:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: l_comment ASC
+|  limit: 100
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1 row-size=263B cardinality=100
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+01:TOP-N [LIMIT=100]
+|  order by: l_comment ASC
+|  mem-estimate=25.66KB mem-reservation=0B
+|  tuple-ids=1 row-size=263B cardinality=100
+|
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.61MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=80.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# Hash Join
+select *
+from tpch.lineitem inner join tpch.orders on l_orderkey = o_orderkey
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=136.00MB
+Per-Host Resource Estimates: Memory=388.41MB
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0,1 row-size=454B cardinality=5757710
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+02:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: l_orderkey = o_orderkey
+|  runtime filters: RF000 <- o_orderkey
+|  mem-estimate=300.41MB mem-reservation=136.00MB
+|  tuple-ids=0,1 row-size=454B cardinality=5757710
+|
+|--03:EXCHANGE [BROADCAST]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=1 row-size=191B cardinality=1500000
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  01:SCAN HDFS [tpch.orders, RANDOM]
+|     partitions=1/1 files=1 size=162.56MB
+|     table stats: 1500000 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=1 row-size=191B cardinality=1500000
+|
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   runtime filters: RF000 -> l_orderkey
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=272.00MB
+Per-Host Resource Estimates: Memory=952.83MB
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0,1 row-size=454B cardinality=5757710
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+02:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=00
+|  hash predicates: l_orderkey = o_orderkey
+|  runtime filters: RF000 <- o_orderkey
+|  mem-estimate=300.41MB mem-reservation=136.00MB
+|  tuple-ids=0,1 row-size=454B cardinality=5757710
+|
+|--F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: o_orderkey
+|  |  mem-estimate=0B mem-reservation=0B
+|  |
+|  03:EXCHANGE [BROADCAST]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=1 row-size=191B cardinality=1500000
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  01:SCAN HDFS [tpch.orders, RANDOM]
+|     partitions=1/1 files=1 size=162.56MB
+|     table stats: 1500000 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=1 row-size=191B cardinality=1500000
+|
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   runtime filters: RF000 -> l_orderkey
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# Nested loop join
+select *
+from tpch.lineitem, tpch.orders
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=361.10MB
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0,1 row-size=454B cardinality=9001822500000
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
+|  mem-estimate=273.10MB mem-reservation=0B
+|  tuple-ids=0,1 row-size=454B cardinality=9001822500000
+|
+|--03:EXCHANGE [BROADCAST]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=1 row-size=191B cardinality=1500000
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  01:SCAN HDFS [tpch.orders, RANDOM]
+|     partitions=1/1 files=1 size=162.56MB
+|     table stats: 1500000 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=1 row-size=191B cardinality=1500000
+|
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=898.21MB
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0,1 row-size=454B cardinality=9001822500000
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
+|  join table id: 00
+|  mem-estimate=273.10MB mem-reservation=0B
+|  tuple-ids=0,1 row-size=454B cardinality=9001822500000
+|
+|--F03:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  mem-estimate=0B mem-reservation=0B
+|  |
+|  03:EXCHANGE [BROADCAST]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=1 row-size=191B cardinality=1500000
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  01:SCAN HDFS [tpch.orders, RANDOM]
+|     partitions=1/1 files=1 size=162.56MB
+|     table stats: 1500000 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=1 row-size=191B cardinality=1500000
+|
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# Empty set node
+select * from functional.alltypes where 1 = 2
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=10.00MB
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:EMPTYSET
+   mem-estimate=0B mem-reservation=0B
+   tuple-ids=0 row-size=0B cardinality=0
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=10.00MB
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:EMPTYSET
+   mem-estimate=0B mem-reservation=0B
+   tuple-ids=0 row-size=0B cardinality=0
+====
+# Analytic function
+select max(tinyint_col) over(partition by int_col)
+from functional.alltypes
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=40.00MB
+Per-Host Resource Estimates: Memory=24.00MB
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=3,2 row-size=6B cardinality=7300
+|
+F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=3
+02:ANALYTIC
+|  functions: max(tinyint_col)
+|  partition by: int_col
+|  mem-estimate=0B mem-reservation=16.00MB
+|  tuple-ids=3,2 row-size=6B cardinality=7300
+|
+01:SORT
+|  order by: int_col ASC NULLS FIRST
+|  mem-estimate=8.00MB mem-reservation=24.00MB
+|  tuple-ids=3 row-size=5B cardinality=7300
+|
+03:EXCHANGE [HASH(int_col)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=5B cardinality=7300
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+00:SCAN HDFS [functional.alltypes, RANDOM]
+   partitions=24/24 files=24 size=478.45KB
+   table stats: 7300 rows total
+   column stats: all
+   mem-estimate=16.00MB mem-reservation=0B
+   tuple-ids=0 row-size=5B cardinality=7300
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=80.00MB
+Per-Host Resource Estimates: Memory=48.00MB
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+04:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=3,2 row-size=6B cardinality=7300
+|
+F01:PLAN FRAGMENT [HASH(int_col)] hosts=3 instances=6
+02:ANALYTIC
+|  functions: max(tinyint_col)
+|  partition by: int_col
+|  mem-estimate=0B mem-reservation=16.00MB
+|  tuple-ids=3,2 row-size=6B cardinality=7300
+|
+01:SORT
+|  order by: int_col ASC NULLS FIRST
+|  mem-estimate=8.00MB mem-reservation=24.00MB
+|  tuple-ids=3 row-size=5B cardinality=7300
+|
+03:EXCHANGE [HASH(int_col)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=5B cardinality=7300
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+00:SCAN HDFS [functional.alltypes, RANDOM]
+   partitions=24/24 files=24 size=478.45KB
+   table stats: 7300 rows total
+   column stats: all
+   mem-estimate=16.00MB mem-reservation=0B
+   tuple-ids=0 row-size=5B cardinality=7300
+====
+# TPC-H Q18: Pipeline of joins with some non-trivial subtrees on right side of joins
+ select
+  c_name,
+  c_custkey,
+  o_orderkey,
+  o_orderdate,
+  o_totalprice,
+  sum(l_quantity)
+from
+  tpch.customer,
+  tpch.orders,
+  tpch.lineitem
+where
+  o_orderkey in (
+    select
+      l_orderkey
+    from
+      tpch.lineitem
+    group by
+      l_orderkey
+    having
+      sum(l_quantity) > 300
+    )
+  and c_custkey = o_custkey
+  and o_orderkey = l_orderkey
+group by
+  c_name,
+  c_custkey,
+  o_orderkey,
+  o_orderdate,
+  o_totalprice
+order by
+  o_totalprice desc,
+  o_orderdate
+limit 100
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=672.00MB
+Per-Host Resource Estimates: Memory=242.95MB
+
+F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+17:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: o_totalprice DESC, o_orderdate ASC
+|  limit: 100
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=7 row-size=100B cardinality=100
+|
+F06:PLAN FRAGMENT [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)] hosts=3 instances=3
+09:TOP-N [LIMIT=100]
+|  order by: o_totalprice DESC, o_orderdate ASC
+|  mem-estimate=9.77KB mem-reservation=0B
+|  tuple-ids=7 row-size=100B cardinality=100
+|
+16:AGGREGATE [FINALIZE]
+|  output: sum:merge(l_quantity)
+|  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
+|  mem-estimate=60.40MB mem-reservation=264.00MB
+|  tuple-ids=6 row-size=100B cardinality=575772
+|
+15:EXCHANGE [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=6 row-size=100B cardinality=575772
+|
+F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=3
+08:AGGREGATE [STREAMING]
+|  output: sum(l_quantity)
+|  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
+|  mem-estimate=60.40MB mem-reservation=0B
+|  tuple-ids=6 row-size=100B cardinality=575772
+|
+07:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  hash predicates: o_orderkey = l_orderkey
+|  runtime filters: RF000 <- l_orderkey
+|  mem-estimate=1.31MB mem-reservation=136.00MB
+|  tuple-ids=2,1,0 row-size=108B cardinality=575772
+|
+|--14:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(l_quantity)
+|  |  group by: l_orderkey
+|  |  having: sum(l_quantity) > 300
+|  |  mem-estimate=10.00MB mem-reservation=264.00MB
+|  |  tuple-ids=4 row-size=24B cardinality=156344
+|  |
+|  13:EXCHANGE [HASH(l_orderkey)]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=4 row-size=24B cardinality=1563438
+|  |
+|  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+|  04:AGGREGATE [STREAMING]
+|  |  output: sum(l_quantity)
+|  |  group by: l_orderkey
+|  |  mem-estimate=39.36MB mem-reservation=0B
+|  |  tuple-ids=4 row-size=24B cardinality=1563438
+|  |
+|  03:SCAN HDFS [tpch.lineitem, RANDOM]
+|     partitions=1/1 files=1 size=718.94MB
+|     table stats: 6001215 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=3 row-size=16B cardinality=6001215
+|
+06:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: o_custkey = c_custkey
+|  runtime filters: RF001 <- c_custkey
+|  mem-estimate=6.61MB mem-reservation=136.00MB
+|  tuple-ids=2,1,0 row-size=108B cardinality=5757710
+|
+|--12:EXCHANGE [BROADCAST]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=0 row-size=42B cardinality=150000
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+|  00:SCAN HDFS [tpch.customer, RANDOM]
+|     partitions=1/1 files=1 size=23.08MB
+|     table stats: 150000 rows total
+|     column stats: all
+|     mem-estimate=32.00MB mem-reservation=0B
+|     tuple-ids=0 row-size=42B cardinality=150000
+|
+05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash predicates: l_orderkey = o_orderkey
+|  runtime filters: RF002 <- o_orderkey
+|  mem-estimate=26.23MB mem-reservation=136.00MB
+|  tuple-ids=2,1 row-size=66B cardinality=5757710
+|
+|--11:EXCHANGE [HASH(o_orderkey)]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=1 row-size=50B cardinality=1500000
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+|  01:SCAN HDFS [tpch.orders, RANDOM]
+|     partitions=1/1 files=1 size=162.56MB
+|     runtime filters: RF000 -> o_orderkey, RF001 -> o_custkey
+|     table stats: 1500000 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=1 row-size=50B cardinality=1500000
+|
+10:EXCHANGE [HASH(l_orderkey)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=2 row-size=16B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+02:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF002 -> l_orderkey
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=2 row-size=16B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=1.83GB
+Per-Host Resource Estimates: Memory=973.08MB
+
+F07:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+17:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: o_totalprice DESC, o_orderdate ASC
+|  limit: 100
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=7 row-size=100B cardinality=100
+|
+F06:PLAN FRAGMENT [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)] hosts=3 instances=6
+09:TOP-N [LIMIT=100]
+|  order by: o_totalprice DESC, o_orderdate ASC
+|  mem-estimate=9.77KB mem-reservation=0B
+|  tuple-ids=7 row-size=100B cardinality=100
+|
+16:AGGREGATE [FINALIZE]
+|  output: sum:merge(l_quantity)
+|  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
+|  mem-estimate=60.40MB mem-reservation=264.00MB
+|  tuple-ids=6 row-size=100B cardinality=575772
+|
+15:EXCHANGE [HASH(c_name,c_custkey,o_orderkey,o_orderdate,o_totalprice)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=6 row-size=100B cardinality=575772
+|
+F02:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
+08:AGGREGATE [STREAMING]
+|  output: sum(l_quantity)
+|  group by: c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice
+|  mem-estimate=60.40MB mem-reservation=0B
+|  tuple-ids=6 row-size=100B cardinality=575772
+|
+07:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
+|  hash-table-id=00
+|  hash predicates: o_orderkey = l_orderkey
+|  runtime filters: RF000 <- l_orderkey
+|  mem-estimate=671.79KB mem-reservation=136.00MB
+|  tuple-ids=2,1,0 row-size=108B cardinality=575772
+|
+|--F08:PLAN FRAGMENT [HASH(l_orderkey)] hosts=3 instances=6
+|  JOIN BUILD
+|  |  join-table-id=00 plan-id=01 cohort-id=01
+|  |  build expressions: l_orderkey
+|  |  mem-estimate=0B mem-reservation=0B
+|  |
+|  14:AGGREGATE [FINALIZE]
+|  |  output: sum:merge(l_quantity)
+|  |  group by: l_orderkey
+|  |  having: sum(l_quantity) > 300
+|  |  mem-estimate=10.00MB mem-reservation=264.00MB
+|  |  tuple-ids=4 row-size=24B cardinality=156344
+|  |
+|  13:EXCHANGE [HASH(l_orderkey)]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=4 row-size=24B cardinality=1563438
+|  |
+|  F04:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+|  04:AGGREGATE [STREAMING]
+|  |  output: sum(l_quantity)
+|  |  group by: l_orderkey
+|  |  mem-estimate=39.36MB mem-reservation=0B
+|  |  tuple-ids=4 row-size=24B cardinality=1563438
+|  |
+|  03:SCAN HDFS [tpch.lineitem, RANDOM]
+|     partitions=1/1 files=1 size=718.94MB
+|     table stats: 6001215 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=3 row-size=16B cardinality=6001215
+|
+06:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash-table-id=01
+|  hash predicates: o_custkey = c_custkey
+|  runtime filters: RF001 <- c_custkey
+|  mem-estimate=6.61MB mem-reservation=136.00MB
+|  tuple-ids=2,1,0 row-size=108B cardinality=5757710
+|
+|--F09:PLAN FRAGMENT [HASH(l_orderkey)] hosts=1 instances=2
+|  JOIN BUILD
+|  |  join-table-id=01 plan-id=02 cohort-id=01
+|  |  build expressions: c_custkey
+|  |  mem-estimate=0B mem-reservation=0B
+|  |
+|  12:EXCHANGE [BROADCAST]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=0 row-size=42B cardinality=150000
+|  |
+|  F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+|  00:SCAN HDFS [tpch.customer, RANDOM]
+|     partitions=1/1 files=1 size=23.08MB
+|     table stats: 150000 rows total
+|     column stats: all
+|     mem-estimate=32.00MB mem-reservation=0B
+|     tuple-ids=0 row-size=42B cardinality=150000
+|
+05:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash-table-id=02
+|  hash predicates: l_orderkey = o_orderkey
+|  runtime filters: RF002 <- o_orderkey
+|  mem-estimate=13.11MB mem-reservation=136.00MB
+|  tuple-ids=2,1 row-size=66B cardinality=5757710
+|
+|--F10:PLAN FRAGMENT [HASH(l_orderkey)] hosts=2 instances=4
+|  JOIN BUILD
+|  |  join-table-id=02 plan-id=03 cohort-id=01
+|  |  build expressions: o_orderkey
+|  |  mem-estimate=0B mem-reservation=0B
+|  |
+|  11:EXCHANGE [HASH(o_orderkey)]
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=1 row-size=50B cardinality=1500000
+|  |
+|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+|  01:SCAN HDFS [tpch.orders, RANDOM]
+|     partitions=1/1 files=1 size=162.56MB
+|     runtime filters: RF000 -> o_orderkey, RF001 -> o_custkey
+|     table stats: 1500000 rows total
+|     column stats: all
+|     mem-estimate=88.00MB mem-reservation=0B
+|     tuple-ids=1 row-size=50B cardinality=1500000
+|
+10:EXCHANGE [HASH(l_orderkey)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=2 row-size=16B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+02:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF002 -> l_orderkey
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=2 row-size=16B cardinality=6001215
+====
+# Unpartitioned HDFS insert
+insert into table functional.alltypesnopart
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col,
+float_col, double_col, date_string_col, string_col, timestamp_col
+from functional.alltypes
+where year=2009 and month=05
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=16.03MB
+
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false]
+|  partitions=1
+|  mem-estimate=26.94KB mem-reservation=0B
+|
+00:SCAN HDFS [functional.alltypes, RANDOM]
+   partitions=1/24 files=1 size=20.36KB
+   table stats: 7300 rows total
+   column stats: all
+   mem-estimate=16.00MB mem-reservation=0B
+   tuple-ids=0 row-size=89B cardinality=310
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=32.03MB
+
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+WRITE TO HDFS [functional.alltypesnopart, OVERWRITE=false]
+|  partitions=1
+|  mem-estimate=13.47KB mem-reservation=0B
+|
+00:SCAN HDFS [functional.alltypes, RANDOM]
+   partitions=1/24 files=1 size=20.36KB
+   table stats: 7300 rows total
+   column stats: all
+   mem-estimate=16.00MB mem-reservation=0B
+   tuple-ids=0 row-size=89B cardinality=310
+====
+# Dynamically partitioned HDFS insert
+create table dummy_insert
+partitioned by (l_partkey) as
+select l_comment, l_partkey from tpch.lineitem
+---- DISTRIBUTEDPLAN
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=184.33MB
+
+F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=3
+WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)]
+|  partitions=200516
+|  mem-estimate=96.33MB mem-reservation=0B
+|
+01:EXCHANGE [HASH(l_partkey)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=50B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=50B cardinality=6001215
+---- PARALLELPLANS
+Per-Host Resource Reservation: Memory=0B
+Per-Host Resource Estimates: Memory=272.33MB
+
+F01:PLAN FRAGMENT [HASH(l_partkey)] hosts=3 instances=6
+WRITE TO HDFS [default.dummy_insert, OVERWRITE=false, PARTITION-KEYS=(l_partkey)]
+|  partitions=200516
+|  mem-estimate=48.16MB mem-reservation=0B
+|
+01:EXCHANGE [HASH(l_partkey)]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=50B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   table stats: 6001215 rows total
+   column stats: all
+   mem-estimate=88.00MB mem-reservation=0B
+   tuple-ids=0 row-size=50B cardinality=6001215
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test
index 0cb0610..06d76c8 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test
@@ -1,117 +1,19 @@
 ====
 ---- QUERY
-# Tests explaining a query (TPCDS-Q19)
+# Explain a simple hash join query.
 explain
-select
-  i_brand_id,
-  i_brand,
-  i_manufact_id,
-  i_manufact,
-  sum(ss_ext_sales_price) ext_price
-from
-  tpcds.store_sales
-  join tpcds.item on (store_sales.ss_item_sk = item.i_item_sk)
-  join tpcds.customer on (store_sales.ss_customer_sk = customer.c_customer_sk)
-  join tpcds.customer_address on (customer.c_current_addr_sk = customer_address.ca_address_sk)
-  join tpcds.store on (store_sales.ss_store_sk = store.s_store_sk)
-where
-  ss_date between '1999-11-01' and '1999-11-30'
-  and i_manager_id = 7
-  and substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5)
-group by
-  i_brand,
-  i_brand_id,
-  i_manufact_id,
-  i_manufact
-order by
-  ext_price desc,
-  i_brand,
-  i_brand_id,
-  i_manufact_id,
-  i_manufact
-limit 100
----- RESULTS
-'Estimated Per-Host Requirements: Memory=72.66MB VCores=5'
+select *
+from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
+---- RESULTS: VERIFY_IS_EQUAL
+'Per-Host Resource Reservation: Memory=136.00MB'
+'Per-Host Resource Estimates: Memory=388.41MB'
 ''
-'19:TOP-N [LIMIT=100]'
-'18:EXCHANGE [UNPARTITIONED]'
-'10:TOP-N [LIMIT=100]'
-'17:AGGREGATE [MERGE FINALIZE]'
-'16:EXCHANGE [HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]'
-'09:AGGREGATE'
-'08:HASH JOIN [INNER JOIN, BROADCAST]'
-'|--15:EXCHANGE [BROADCAST]'
-'|  04:SCAN HDFS [tpcds.store]'
-'07:HASH JOIN [INNER JOIN, BROADCAST]'
-'|--14:EXCHANGE [BROADCAST]'
-'|  01:SCAN HDFS [tpcds.item]'
-'06:HASH JOIN [INNER JOIN, BROADCAST]'
-'|--13:EXCHANGE [BROADCAST]'
-'|  00:SCAN HDFS [tpcds.store_sales]'
-'05:HASH JOIN [INNER JOIN, PARTITIONED]'
-'|--12:EXCHANGE [HASH(customer.c_current_addr_sk)]'
-'|  02:SCAN HDFS [tpcds.customer]'
-'11:EXCHANGE [HASH(customer_address.ca_address_sk)]'
-'03:SCAN HDFS [tpcds.customer_address]'
-====
----- QUERY
-# Tests explaining an insert query
-explain insert overwrite functional.alltypessmall (id, string_col)
-partition (year, month)
-select a.id, a.string_col, a.year, a.month from functional.alltypes a
-  left semi join functional.alltypesagg b on (a.id = b.id)
-where a.year > 2009 and a.month = 4
-union distinct
-select id, string_col, year, month from functional.alltypes
----- RESULTS
-'Estimated Per-Host Requirements: Memory=30.00MB VCores=3'
-''
-'WRITE TO HDFS [functional.alltypessmall, OVERWRITE=true, PARTITION-KEYS=(year,month)]'
-'14:EXCHANGE [HASH(year,month)]'
-'13:AGGREGATE [MERGE FINALIZE]'
-'12:EXCHANGE [HASH(id,string_col,year,month)]'
-'|--11:AGGREGATE'
-'|  10:MERGE'
-'|  04:SCAN HDFS [functional.alltypes]'
-'05:AGGREGATE'
-'09:MERGE'
-'03:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]'
-'|--07:EXCHANGE [HASH(b.id)]'
-'|  02:SCAN HDFS [functional.alltypesagg b]'
-'06:EXCHANGE [HASH(a.id)]'
-'01:SCAN HDFS [functional.alltypes a]'
-====
----- QUERY
-# Tests explaining an insert query to/from an HBase table
-explain insert into functional_hbase.alltypes
-select a.* from functional_hbase.alltypessmall a
-  cross join functional.alltypessmall b
-where a.year > 2009 and a.month = 4
-union all
-select * from functional_hbase.alltypessmall
----- RESULTS
-'Estimated Per-Host Requirements: Memory=1.02GB VCores=3'
-''
-'WRITE TO HBASE table=functional_hbase.alltypes'
-'06:EXCHANGE [UNPARTITIONED]'
-'|--08:MERGE'
-'|  04:SCAN HBASE [functional_hbase.alltypessmall]'
-'07:MERGE'
-'03:CROSS JOIN [BROADCAST]'
-'|--05:EXCHANGE [BROADCAST]'
-'|  02:SCAN HDFS [functional.alltypessmall b]'
-'01:SCAN HBASE [functional_hbase.alltypessmall a]'
-====
----- QUERY
-# Tests explaining an CTAS statement.
-explain create table t as
-select * from functional.alltypes
-where month = 2
----- RESULTS
-'Estimated Per-Host Requirements: Memory=16.02MB VCores=1'
-''
-'WRITE TO HDFS [functional.t, OVERWRITE=false]'
-'00:SCAN HDFS [functional.alltypes]'
+'PLAN-ROOT SINK'
+'04:EXCHANGE [UNPARTITIONED]'
+'02:HASH JOIN [INNER JOIN, BROADCAST]'
+'|--03:EXCHANGE [BROADCAST]'
+'|  01:SCAN HDFS [tpch.orders]'
+'00:SCAN HDFS [tpch.lineitem]'
 ====
 ---- QUERY
 # Tests the warning about missing table stats in the explain header.
@@ -120,19 +22,7 @@ from functional_avro.alltypes t1
   inner join functional_parquet.alltypessmall t2 on (t1.id = t2.id)
   left outer join functional_avro.alltypes t3 on (t2.id = t3.id)
 where t1.month = 1 and t2.year = 2009 and t3.bool_col = false
----- RESULTS
-'Estimated Per-Host Requirements: Memory=4.03GB VCores=3'
+---- RESULTS: VERIFY_IS_SUBSET
 'WARNING: The following tables are missing relevant table and/or column statistics.'
 'functional_avro.alltypes, functional_parquet.alltypessmall'
-''
-'09:AGGREGATE [MERGE FINALIZE]'
-'08:EXCHANGE [UNPARTITIONED]'
-'05:AGGREGATE'
-'04:HASH JOIN [LEFT OUTER JOIN, BROADCAST]'
-'|--07:EXCHANGE [BROADCAST]'
-'|  02:SCAN HDFS [functional_avro.alltypes t3]'
-'03:HASH JOIN [INNER JOIN, BROADCAST]'
-'|--06:EXCHANGE [BROADCAST]'
-'|  01:SCAN HDFS [functional_parquet.alltypessmall t2]'
-'00:SCAN HDFS [functional_avro.alltypes t1]'
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test
index a1a172d..a082603 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test
@@ -1,187 +1,29 @@
 ====
 ---- QUERY
-# Tests explaining a query (TPCDS-Q19)
+# Explain a simple hash join query.
 explain
-select
-  i_brand_id,
-  i_brand,
-  i_manufact_id,
-  i_manufact,
-  sum(ss_ext_sales_price) ext_price
-from
-  tpcds.store_sales
-  join tpcds.item on (store_sales.ss_item_sk = item.i_item_sk)
-  join tpcds.customer on (store_sales.ss_customer_sk = customer.c_customer_sk)
-  join tpcds.customer_address on (customer.c_current_addr_sk = customer_address.ca_address_sk)
-  join tpcds.store on (store_sales.ss_store_sk = store.s_store_sk)
-where
-  ss_date between '1999-11-01' and '1999-11-30'
-  and i_manager_id = 7
-  and substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5)
-group by
-  i_brand,
-  i_brand_id,
-  i_manufact_id,
-  i_manufact
-order by
-  ext_price desc,
-  i_brand,
-  i_brand_id,
-  i_manufact_id,
-  i_manufact
-limit 100
----- RESULTS
-'Estimated Per-Host Requirements: Memory=72.66MB VCores=5'
+select *
+from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
+---- RESULTS: VERIFY_IS_EQUAL
+'Per-Host Resource Reservation: Memory=136.00MB'
+'Per-Host Resource Estimates: Memory=388.41MB'
 ''
-'19:TOP-N [LIMIT=100]'
-'|  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC'
+'PLAN-ROOT SINK'
 '|'
-'18:EXCHANGE [UNPARTITIONED]'
+'04:EXCHANGE [UNPARTITIONED]'
 '|'
-'10:TOP-N [LIMIT=100]'
-'|  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC'
+'02:HASH JOIN [INNER JOIN, BROADCAST]'
+'|  hash predicates: l_orderkey = o_orderkey'
+'|  runtime filters: RF000 <- o_orderkey'
 '|'
-'17:AGGREGATE [MERGE FINALIZE]'
-'|  output: sum(sum(ss_ext_sales_price))'
-'|  group by: i_brand, i_brand_id, i_manufact_id, i_manufact'
-'|'
-'16:EXCHANGE [HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]'
-'|'
-'09:AGGREGATE'
-'|  output: sum(ss_ext_sales_price)'
-'|  group by: i_brand, i_brand_id, i_manufact_id, i_manufact'
-'|'
-'08:HASH JOIN [INNER JOIN, BROADCAST]'
-'|  hash predicates: store_sales.ss_store_sk = store.s_store_sk'
-'|  other predicates: substr(ca_zip, 1, 5) != substr(s_zip, 1, 5)'
-'|'
-'|--15:EXCHANGE [BROADCAST]'
-'|  |'
-'|  04:SCAN HDFS [tpcds.store]'
-'|     partitions=1/1 size=3.08KB'
-'|'
-'07:HASH JOIN [INNER JOIN, BROADCAST]'
-'|  hash predicates: store_sales.ss_item_sk = item.i_item_sk'
-'|'
-'|--14:EXCHANGE [BROADCAST]'
+'|--03:EXCHANGE [BROADCAST]'
 '|  |'
-'|  01:SCAN HDFS [tpcds.item]'
-'|     partitions=1/1 size=4.82MB'
-'|     predicates: i_manager_id = 7'
-'|'
-'06:HASH JOIN [INNER JOIN, BROADCAST]'
-'|  hash predicates: customer.c_customer_sk = store_sales.ss_customer_sk'
-'|'
-'|--13:EXCHANGE [BROADCAST]'
-'|  |'
-'|  00:SCAN HDFS [tpcds.store_sales]'
-'|     partitions=2/120 size=663.52KB'
-'|'
-'05:HASH JOIN [INNER JOIN, PARTITIONED]'
-'|  hash predicates: customer_address.ca_address_sk = customer.c_current_addr_sk'
-'|'
-'|--12:EXCHANGE [HASH(customer.c_current_addr_sk)]'
-'|  |'
-'|  02:SCAN HDFS [tpcds.customer]'
-'|     partitions=1/1 size=12.60MB'
-'|'
-'11:EXCHANGE [HASH(customer_address.ca_address_sk)]'
+'|  01:SCAN HDFS [tpch.orders]'
+row_regex:.*partitions=1/1 files=1 size=.*
 '|'
-'03:SCAN HDFS [tpcds.customer_address]'
-'   partitions=1/1 size=5.25MB'
-====
----- QUERY
-# Tests explaining an insert query
-explain insert overwrite functional.alltypessmall (id, string_col)
-partition (year, month)
-select a.id, a.string_col, a.year, a.month from functional.alltypes a
-  left semi join functional.alltypesagg b on (a.id = b.id)
-where a.year > 2009 and a.month = 4
-union distinct
-select id, string_col, year, month from functional.alltypes
----- RESULTS
-'Estimated Per-Host Requirements: Memory=30.00MB VCores=3'
-''
-'WRITE TO HDFS [functional.alltypessmall, OVERWRITE=true, PARTITION-KEYS=(year,month)]'
-'|  partitions=96'
-'|'
-'14:EXCHANGE [HASH(year,month)]'
-'|'
-'13:AGGREGATE [MERGE FINALIZE]'
-'|  group by: id, string_col, year, month'
-'|'
-'12:EXCHANGE [HASH(id,string_col,year,month)]'
-'|'
-'|--11:AGGREGATE'
-'|  |  group by: id, string_col, year, month'
-'|  |'
-'|  10:MERGE'
-'|  |'
-'|  04:SCAN HDFS [functional.alltypes]'
-'|     partitions=24/24 size=478.45KB'
-'|'
-'05:AGGREGATE'
-'|  group by: id, string_col, year, month'
-'|'
-'09:MERGE'
-'|'
-'03:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]'
-'|  hash predicates: a.id = b.id'
-'|'
-'|--07:EXCHANGE [HASH(b.id)]'
-'|  |'
-'|  02:SCAN HDFS [functional.alltypesagg b]'
-'|     partitions=10/10 size=743.67KB'
-'|'
-'06:EXCHANGE [HASH(a.id)]'
-'|'
-'01:SCAN HDFS [functional.alltypes a]'
-'   partitions=1/24 size=19.71KB'
-====
----- QUERY
-# Tests explaining an insert query to/from an HBase table
-explain insert into functional_hbase.alltypes
-select a.* from functional_hbase.alltypessmall a
-  cross join functional.alltypessmall b
-where a.year > 2009 and a.month = 4
-union all
-select * from functional_hbase.alltypessmall
----- RESULTS
-'Estimated Per-Host Requirements: Memory=1.02GB VCores=3'
-''
-'WRITE TO HBASE table=functional_hbase.alltypes'
-'|'
-'06:EXCHANGE [UNPARTITIONED]'
-'|'
-'|--08:MERGE'
-'|  |'
-'|  04:SCAN HBASE [functional_hbase.alltypessmall]'
-'|'
-'07:MERGE'
-'|'
-'03:CROSS JOIN [BROADCAST]'
-'|'
-'|--05:EXCHANGE [BROADCAST]'
-'|  |'
-'|  02:SCAN HDFS [functional.alltypessmall b]'
-'|     partitions=4/4 size=6.32KB'
-'|'
-'01:SCAN HBASE [functional_hbase.alltypessmall a]'
-'   predicates: a.year > 2009, a.month = 4'
-====
----- QUERY
-# Tests explaining an CTAS statement.
-explain create table t as
-select * from functional.alltypes
-where month = 2
----- RESULTS
-'Estimated Per-Host Requirements: Memory=16.02MB VCores=1'
-''
-'WRITE TO HDFS [functional.t, OVERWRITE=false]'
-'|  partitions=1'
-'|'
-'00:SCAN HDFS [functional.alltypes]'
-'   partitions=2/24 size=36.51KB'
+'00:SCAN HDFS [tpch.lineitem]'
+row_regex:.*partitions=1/1 files=1 size=.*
+'   runtime filters: RF000 -> l_orderkey'
 ====
 ---- QUERY
 # Tests the warning about missing table stats in the explain header.
@@ -190,37 +32,7 @@ from functional_avro.alltypes t1
   inner join functional_parquet.alltypessmall t2 on (t1.id = t2.id)
   left outer join functional_avro.alltypes t3 on (t2.id = t3.id)
 where t1.month = 1 and t2.year = 2009 and t3.bool_col = false
----- RESULTS
-'Estimated Per-Host Requirements: Memory=4.03GB VCores=3'
+---- RESULTS: VERIFY_IS_SUBSET
 'WARNING: The following tables are missing relevant table and/or column statistics.'
 'functional_avro.alltypes, functional_parquet.alltypessmall'
-''
-'09:AGGREGATE [MERGE FINALIZE]'
-'|  output: sum(count(t1.int_col)), sum(sum(t2.float_col)), sum(count(t2.float_col)), sum(sum(t3.bigint_col))'
-'|'
-'08:EXCHANGE [UNPARTITIONED]'
-'|'
-'05:AGGREGATE'
-'|  output: count(t1.int_col), sum(t2.float_col), count(t2.float_col), sum(t3.bigint_col)'
-'|'
-'04:HASH JOIN [LEFT OUTER JOIN, BROADCAST]'
-'|  hash predicates: t2.id = t3.id'
-'|  other predicates: t3.bool_col = FALSE'
-'|'
-'|--07:EXCHANGE [BROADCAST]'
-'|  |'
-'|  02:SCAN HDFS [functional_avro.alltypes t3]'
-'|     partitions=24/24 size=470.35KB'
-'|     predicates: t3.bool_col = FALSE'
-'|'
-'03:HASH JOIN [INNER JOIN, BROADCAST]'
-'|  hash predicates: t1.id = t2.id'
-'|'
-'|--06:EXCHANGE [BROADCAST]'
-'|  |'
-'|  01:SCAN HDFS [functional_parquet.alltypessmall t2]'
-'|     partitions=4/4 size=9.63KB'
-'|'
-'00:SCAN HDFS [functional_avro.alltypes t1]'
-'   partitions=2/24 size=39.87KB'
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
index 2206ae8..32171fb 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
@@ -1,298 +1,47 @@
 ====
 ---- QUERY
-# Tests explaining a query (TPCDS-Q19)
+# Explain a simple hash join query.
 explain
-select
-  i_brand_id,
-  i_brand,
-  i_manufact_id,
-  i_manufact,
-  sum(ss_ext_sales_price) ext_price
-from
-  tpcds.store_sales
-  join tpcds.item on (store_sales.ss_item_sk = item.i_item_sk)
-  join tpcds.customer on (store_sales.ss_customer_sk = customer.c_customer_sk)
-  join tpcds.customer_address on (customer.c_current_addr_sk = customer_address.ca_address_sk)
-  join tpcds.store on (store_sales.ss_store_sk = store.s_store_sk)
-where
-  ss_date between '1999-11-01' and '1999-11-30'
-  and i_manager_id = 7
-  and substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5)
-group by
-  i_brand,
-  i_brand_id,
-  i_manufact_id,
-  i_manufact
-order by
-  ext_price desc,
-  i_brand,
-  i_brand_id,
-  i_manufact_id,
-  i_manufact
-limit 100
----- RESULTS
-'Estimated Per-Host Requirements: Memory=72.66MB VCores=5'
+select *
+from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
+---- RESULTS: VERIFY_IS_EQUAL
+'Per-Host Resource Reservation: Memory=136.00MB'
+'Per-Host Resource Estimates: Memory=388.41MB'
 ''
-'19:TOP-N [LIMIT=100]'
-'|  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC'
-'|  hosts=3 per-host-mem=unavailable'
-'|  tuple-ids=5 row-size=48B cardinality=100'
-'|'
-'18:EXCHANGE [UNPARTITIONED]'
-'|  hosts=3 per-host-mem=unavailable'
-'|  tuple-ids=5 row-size=48B cardinality=100'
-'|'
-'10:TOP-N [LIMIT=100]'
-'|  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC'
-'|  hosts=3 per-host-mem=4.69KB'
-'|  tuple-ids=5 row-size=48B cardinality=100'
-'|'
-'17:AGGREGATE [MERGE FINALIZE]'
-'|  output: sum(sum(ss_ext_sales_price))'
-'|  group by: i_brand, i_brand_id, i_manufact_id, i_manufact'
-'|  hosts=3 per-host-mem=10.00MB'
-'|  tuple-ids=5 row-size=48B cardinality=87208'
-'|'
-'16:EXCHANGE [HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]'
-'|  hosts=3 per-host-mem=0B'
-'|  tuple-ids=5 row-size=48B cardinality=87208'
-'|'
-'09:AGGREGATE'
-'|  output: sum(ss_ext_sales_price)'
-'|  group by: i_brand, i_brand_id, i_manufact_id, i_manufact'
-'|  hosts=3 per-host-mem=10.00MB'
-'|  tuple-ids=5 row-size=48B cardinality=87208'
-'|'
-'08:HASH JOIN [INNER JOIN, BROADCAST]'
-'|  hash predicates: store_sales.ss_store_sk = store.s_store_sk'
-'|  other predicates: substr(ca_zip, 1, 5) != substr(s_zip, 1, 5)'
-'|  hosts=3 per-host-mem=383B'
-'|  tuple-ids=3,2,0,1,4 row-size=192B cardinality=87208'
-'|'
-'|--15:EXCHANGE [BROADCAST]'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=4 row-size=29B cardinality=12'
-'|  |'
-'|  04:SCAN HDFS [tpcds.store, RANDOM]'
-'|     partitions=1/1 size=3.08KB'
-'|     table stats: 12 rows total'
-'|     column stats: all'
-'|     hosts=3 per-host-mem=16.00MB'
-'|     tuple-ids=4 row-size=29B cardinality=12'
-'|'
-'07:HASH JOIN [INNER JOIN, BROADCAST]'
-'|  hash predicates: store_sales.ss_item_sk = item.i_item_sk'
-'|  hosts=3 per-host-mem=14.60KB'
-'|  tuple-ids=3,2,0,1 row-size=163B cardinality=87208'
-'|'
-'|--14:EXCHANGE [BROADCAST]'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=1 row-size=79B cardinality=171'
-'|  |'
-'|  01:SCAN HDFS [tpcds.item, RANDOM]'
-'|     partitions=1/1 size=4.82MB'
-'|     predicates: i_manager_id = 7'
-'|     table stats: 18000 rows total'
-'|     column stats: all'
-'|     hosts=3 per-host-mem=16.00MB'
-'|     tuple-ids=1 row-size=79B cardinality=171'
-'|'
-'06:HASH JOIN [INNER JOIN, BROADCAST]'
-'|  hash predicates: customer.c_customer_sk = store_sales.ss_customer_sk'
-'|  hosts=3 per-host-mem=234.75KB'
-'|  tuple-ids=3,2,0 row-size=84B cardinality=87208'
-'|'
-'|--13:EXCHANGE [BROADCAST]'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=0 row-size=43B cardinality=5082'
-'|  |'
-'|  00:SCAN HDFS [tpcds.store_sales, RANDOM]'
-'|     partitions=2/120 size=663.52KB'
-'|     table stats: 183592 rows total'
-'|     column stats: all'
-'|     hosts=3 per-host-mem=16.00MB'
-'|     tuple-ids=0 row-size=43B cardinality=5082'
-'|'
-'05:HASH JOIN [INNER JOIN, PARTITIONED]'
-'|  hash predicates: customer_address.ca_address_sk = customer.c_current_addr_sk'
-'|  hosts=3 per-host-mem=429.69KB'
-'|  tuple-ids=3,2 row-size=41B cardinality=87208'
-'|'
-'|--12:EXCHANGE [HASH(customer.c_current_addr_sk)]'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=2 row-size=12B cardinality=100000'
-'|  |'
-'|  02:SCAN HDFS [tpcds.customer, RANDOM]'
-'|     partitions=1/1 size=12.60MB'
-'|     table stats: 100000 rows total'
-'|     column stats: all'
-'|     hosts=3 per-host-mem=24.00MB'
-'|     tuple-ids=2 row-size=12B cardinality=100000'
-'|'
-'11:EXCHANGE [HASH(customer_address.ca_address_sk)]'
-'|  hosts=3 per-host-mem=0B'
-'|  tuple-ids=3 row-size=29B cardinality=50000'
-'|'
-'03:SCAN HDFS [tpcds.customer_address, RANDOM]'
-'   partitions=1/1 size=5.25MB'
-'   table stats: 50000 rows total'
+'F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
+'PLAN-ROOT SINK'
+'|  mem-estimate=0B mem-reservation=0B'
+'|'
+'04:EXCHANGE [UNPARTITIONED]'
+'|  mem-estimate=0B mem-reservation=0B'
+'|  tuple-ids=0,1 row-size=454B cardinality=5757710'
+'|'
+'F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3'
+'02:HASH JOIN [INNER JOIN, BROADCAST]'
+'|  hash predicates: l_orderkey = o_orderkey'
+'|  runtime filters: RF000 <- o_orderkey'
+'|  mem-estimate=300.41MB mem-reservation=136.00MB'
+'|  tuple-ids=0,1 row-size=454B cardinality=5757710'
+'|'
+'|--03:EXCHANGE [BROADCAST]'
+'|  |  mem-estimate=0B mem-reservation=0B'
+'|  |  tuple-ids=1 row-size=191B cardinality=1500000'
+'|  |'
+'|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2'
+'|  01:SCAN HDFS [tpch.orders, RANDOM]'
+row_regex:.*partitions=1/1 files=1 size=.*
+'|     table stats: 1500000 rows total'
+'|     column stats: all'
+'|     mem-estimate=88.00MB mem-reservation=0B'
+'|     tuple-ids=1 row-size=191B cardinality=1500000'
+'|'
+'00:SCAN HDFS [tpch.lineitem, RANDOM]'
+row_regex:.*partitions=1/1 files=1 size=.*
+'   runtime filters: RF000 -> l_orderkey'
+'   table stats: 6001215 rows total'
 '   column stats: all'
-'   hosts=3 per-host-mem=16.00MB'
-'   tuple-ids=3 row-size=29B cardinality=50000'
-====
----- QUERY
-# Tests explaining an insert query
-explain insert overwrite functional.alltypessmall (id, string_col)
-partition (year, month)
-select a.id, a.string_col, a.year, a.month from functional.alltypes a
-  left semi join functional.alltypesagg b on (a.id = b.id)
-where a.year > 2009 and a.month = 4
-union distinct
-select id, string_col, year, month from functional.alltypes
----- RESULTS
-'Estimated Per-Host Requirements: Memory=30.00MB VCores=3'
-''
-'WRITE TO HDFS [functional.alltypessmall, OVERWRITE=true, PARTITION-KEYS=(year,month)]'
-'|  partitions=96'
-'|  hosts=3 per-host-mem=69.26KB'
-'|'
-'14:EXCHANGE [HASH(year,month)]'
-'|  hosts=3 per-host-mem=0B'
-'|  tuple-ids=3 row-size=28B cardinality=7600'
-'|'
-'13:AGGREGATE [MERGE FINALIZE]'
-'|  group by: id, string_col, year, month'
-'|  hosts=3 per-host-mem=10.00MB'
-'|  tuple-ids=3 row-size=28B cardinality=7600'
-'|'
-'12:EXCHANGE [HASH(id,string_col,year,month)]'
-'|  hosts=3 per-host-mem=0B'
-'|  tuple-ids=3 row-size=28B cardinality=7600'
-'|'
-'|--11:AGGREGATE'
-'|  |  group by: id, string_col, year, month'
-'|  |  hosts=3 per-host-mem=10.00MB'
-'|  |  tuple-ids=3 row-size=28B cardinality=7300'
-'|  |'
-'|  10:MERGE'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=3 row-size=28B cardinality=7300'
-'|  |'
-'|  04:SCAN HDFS [functional.alltypes, RANDOM]'
-'|     partitions=24/24 size=478.45KB'
-'|     table stats: 7300 rows total'
-'|     column stats: all'
-'|     hosts=3 per-host-mem=16.00MB'
-'|     tuple-ids=2 row-size=29B cardinality=7300'
-'|'
-'05:AGGREGATE'
-'|  group by: id, string_col, year, month'
-'|  hosts=3 per-host-mem=10.00MB'
-'|  tuple-ids=3 row-size=28B cardinality=300'
-'|'
-'09:MERGE'
-'|  hosts=3 per-host-mem=0B'
-'|  tuple-ids=3 row-size=28B cardinality=300'
-'|'
-'03:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]'
-'|  hash predicates: a.id = b.id'
-'|  hosts=3 per-host-mem=14.32KB'
-'|  tuple-ids=0,1 row-size=33B cardinality=300'
-'|'
-'|--07:EXCHANGE [HASH(b.id)]'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=1 row-size=4B cardinality=10000'
-'|  |'
-'|  02:SCAN HDFS [functional.alltypesagg b, RANDOM]'
-'|     partitions=10/10 size=743.67KB'
-'|     table stats: 10000 rows total'
-'|     column stats: all'
-'|     hosts=3 per-host-mem=16.00MB'
-'|     tuple-ids=1 row-size=4B cardinality=10000'
-'|'
-'06:EXCHANGE [HASH(a.id)]'
-'|  hosts=3 per-host-mem=0B'
-'|  tuple-ids=0 row-size=29B cardinality=300'
-'|'
-'01:SCAN HDFS [functional.alltypes a, RANDOM]'
-'   partitions=1/24 size=19.71KB'
-'   table stats: 7300 rows total'
-'   column stats: all'
-'   hosts=3 per-host-mem=16.00MB'
-'   tuple-ids=0 row-size=29B cardinality=300'
-====
----- QUERY
-# Tests explaining an insert query to/from an HBase table
-explain insert into functional_hbase.alltypes
-select a.* from functional_hbase.alltypessmall a
-  cross join functional.alltypessmall b
-where a.year > 2009 and a.month = 4
-union all
-select * from functional_hbase.alltypessmall
----- RESULTS
-'Estimated Per-Host Requirements: Memory=1.02GB VCores=3'
-''
-'WRITE TO HBASE table=functional_hbase.alltypes'
-'|  hosts=1 per-host-mem=unavailable'
-'|'
-'06:EXCHANGE [UNPARTITIONED]'
-'|  hosts=100 per-host-mem=unavailable'
-'|  tuple-ids=3 row-size=88B cardinality=596'
-'|'
-'|--08:MERGE'
-'|  |  hosts=100 per-host-mem=0B'
-'|  |  tuple-ids=3 row-size=88B cardinality=196'
-'|  |'
-'|  04:SCAN HBASE [functional_hbase.alltypessmall]'
-'|     table stats: 100 rows total'
-'|     column stats: all'
-'|     hosts=100 per-host-mem=1.00GB'
-'|     tuple-ids=2 row-size=97B cardinality=196'
-'|'
-'07:MERGE'
-'|  hosts=100 per-host-mem=0B'
-'|  tuple-ids=3 row-size=88B cardinality=400'
-'|'
-'03:CROSS JOIN [BROADCAST]'
-'|  hosts=100 per-host-mem=0B'
-'|  tuple-ids=0,1 row-size=97B cardinality=400'
-'|'
-'|--05:EXCHANGE [BROADCAST]'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=1 row-size=0B cardinality=100'
-'|  |'
-'|  02:SCAN HDFS [functional.alltypessmall b, RANDOM]'
-'|     partitions=4/4 size=6.32KB'
-'|     table stats: 100 rows total'
-'|     column stats: all'
-'|     hosts=3 per-host-mem=16.00MB'
-'|     tuple-ids=1 row-size=0B cardinality=100'
-'|'
-'01:SCAN HBASE [functional_hbase.alltypessmall a]'
-'   predicates: a.year > 2009, a.month = 4'
-'   table stats: 100 rows total'
-'   column stats: all'
-'   hosts=100 per-host-mem=1.00GB'
-'   tuple-ids=0 row-size=97B cardinality=4'
-====
----- QUERY
-# Tests explaining an CTAS statement.
-explain create table t as
-select * from functional.alltypes
-where month = 2
----- RESULTS
-'Estimated Per-Host Requirements: Memory=16.02MB VCores=1'
-''
-'WRITE TO HDFS [functional.t, OVERWRITE=false]'
-'|  partitions=1'
-'|  hosts=3 per-host-mem=17.62KB'
-'|'
-'00:SCAN HDFS [functional.alltypes, RANDOM]'
-'   partitions=2/24 size=36.51KB'
-'   table stats: 7300 rows total'
-'   column stats: all'
-'   hosts=3 per-host-mem=16.00MB'
-'   tuple-ids=1 row-size=97B cardinality=560'
+'   mem-estimate=88.00MB mem-reservation=0B'
+'   tuple-ids=0 row-size=263B cardinality=6001215'
 ====
 ---- QUERY
 # Tests the warning about missing table stats in the explain header.
@@ -301,63 +50,8 @@ from functional_avro.alltypes t1
   inner join functional_parquet.alltypessmall t2 on (t1.id = t2.id)
   left outer join functional_avro.alltypes t3 on (t2.id = t3.id)
 where t1.month = 1 and t2.year = 2009 and t3.bool_col = false
----- RESULTS
-'Estimated Per-Host Requirements: Memory=4.03GB VCores=3'
+---- RESULTS: VERIFY_IS_SUBSET
+'Per-Host Resource Estimates: Memory=4.03GB'
 'WARNING: The following tables are missing relevant table and/or column statistics.'
 'functional_avro.alltypes, functional_parquet.alltypessmall'
-''
-'09:AGGREGATE [MERGE FINALIZE]'
-'|  output: sum(count(t1.int_col)), sum(sum(t2.float_col)), sum(count(t2.float_col)), sum(sum(t3.bigint_col))'
-'|  hosts=3 per-host-mem=unavailable'
-'|  tuple-ids=3 row-size=32B cardinality=1'
-'|'
-'08:EXCHANGE [UNPARTITIONED]'
-'|  hosts=3 per-host-mem=unavailable'
-'|  tuple-ids=3 row-size=32B cardinality=1'
-'|'
-'05:AGGREGATE'
-'|  output: count(t1.int_col), sum(t2.float_col), count(t2.float_col), sum(t3.bigint_col)'
-'|  hosts=3 per-host-mem=10.00MB'
-'|  tuple-ids=3 row-size=32B cardinality=1'
-'|'
-'04:HASH JOIN [LEFT OUTER JOIN, BROADCAST]'
-'|  hash predicates: t2.id = t3.id'
-'|  other predicates: t3.bool_col = FALSE'
-'|  hosts=3 per-host-mem=2.00GB'
-'|  tuple-ids=0,1,2N row-size=37B cardinality=unavailable'
-'|'
-'|--07:EXCHANGE [BROADCAST]'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=2 row-size=13B cardinality=unavailable'
-'|  |'
-'|  02:SCAN HDFS [functional_avro.alltypes t3, RANDOM]'
-'|     partitions=24/24 size=470.35KB'
-'|     predicates: t3.bool_col = FALSE'
-'|     table stats: unavailable'
-'|     column stats: unavailable'
-'|     hosts=3 per-host-mem=16.00MB'
-'|     tuple-ids=2 row-size=13B cardinality=unavailable'
-'|'
-'03:HASH JOIN [INNER JOIN, BROADCAST]'
-'|  hash predicates: t1.id = t2.id'
-'|  hosts=3 per-host-mem=2.00GB'
-'|  tuple-ids=0,1 row-size=24B cardinality=unavailable'
-'|'
-'|--06:EXCHANGE [BROADCAST]'
-'|  |  hosts=3 per-host-mem=0B'
-'|  |  tuple-ids=1 row-size=12B cardinality=unavailable'
-'|  |'
-'|  01:SCAN HDFS [functional_parquet.alltypessmall t2, RANDOM]'
-'|     partitions=4/4 size=9.63KB'
-'|     table stats: unavailable'
-'|     columns missing stats: id, float_col'
-'|     hosts=3 per-host-mem=16.00MB'
-'|     tuple-ids=1 row-size=12B cardinality=unavailable'
-'|'
-'00:SCAN HDFS [functional_avro.alltypes t1, RANDOM]'
-'   partitions=2/24 size=39.87KB'
-'   table stats: unavailable'
-'   columns missing stats: id, int_col'
-'   hosts=3 per-host-mem=16.00MB'
-'   tuple-ids=0 row-size=12B cardinality=unavailable'
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
index 7da9b04..f1ff4a8 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
@@ -1,332 +1,51 @@
 ====
 ---- QUERY
-# Tests explaining a query (TPCDS-Q19)
+# Explain a simple hash join query.
 explain
-select
-  i_brand_id,
-  i_brand,
-  i_manufact_id,
-  i_manufact,
-  sum(ss_ext_sales_price) ext_price
-from
-  tpcds.store_sales
-  join tpcds.item on (store_sales.ss_item_sk = item.i_item_sk)
-  join tpcds.customer on (store_sales.ss_customer_sk = customer.c_customer_sk)
-  join tpcds.customer_address on (customer.c_current_addr_sk = customer_address.ca_address_sk)
-  join tpcds.store on (store_sales.ss_store_sk = store.s_store_sk)
-where
-  ss_date between '1999-11-01' and '1999-11-30'
-  and i_manager_id = 7
-  and substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5)
-group by
-  i_brand,
-  i_brand_id,
-  i_manufact_id,
-  i_manufact
-order by
-  ext_price desc,
-  i_brand,
-  i_brand_id,
-  i_manufact_id,
-  i_manufact
-limit 100
----- RESULTS
-'Estimated Per-Host Requirements: Memory=72.66MB VCores=5'
-''
-'F07:PLAN FRAGMENT [UNPARTITIONED]'
-'  19:TOP-N [LIMIT=100]'
-'  |  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC'
-'  |  hosts=3 per-host-mem=unavailable'
-'  |  tuple-ids=5 row-size=48B cardinality=100'
-'  |'
-'  18:EXCHANGE [UNPARTITIONED]'
-'     hosts=3 per-host-mem=unavailable'
-'     tuple-ids=5 row-size=48B cardinality=100'
-''
-'F06:PLAN FRAGMENT [HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]'
-'  DATASTREAM SINK [FRAGMENT=F07, EXCHANGE=18, UNPARTITIONED]'
-'  10:TOP-N [LIMIT=100]'
-'  |  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC'
-'  |  hosts=3 per-host-mem=4.69KB'
-'  |  tuple-ids=5 row-size=48B cardinality=100'
-'  |'
-'  17:AGGREGATE [MERGE FINALIZE]'
-'  |  output: sum(sum(ss_ext_sales_price))'
-'  |  group by: i_brand, i_brand_id, i_manufact_id, i_manufact'
-'  |  hosts=3 per-host-mem=10.00MB'
-'  |  tuple-ids=5 row-size=48B cardinality=87208'
-'  |'
-'  16:EXCHANGE [HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]'
-'     hosts=3 per-host-mem=0B'
-'     tuple-ids=5 row-size=48B cardinality=87208'
-''
-'F02:PLAN FRAGMENT [HASH(customer_address.ca_address_sk)]'
-'  DATASTREAM SINK [FRAGMENT=F06, EXCHANGE=16, HASH(i_brand,i_brand_id,i_manufact_id,i_manufact)]'
-'  09:AGGREGATE'
-'  |  output: sum(ss_ext_sales_price)'
-'  |  group by: i_brand, i_brand_id, i_manufact_id, i_manufact'
-'  |  hosts=3 per-host-mem=10.00MB'
-'  |  tuple-ids=5 row-size=48B cardinality=87208'
-'  |'
-'  08:HASH JOIN [INNER JOIN, BROADCAST]'
-'  |  hash predicates: store_sales.ss_store_sk = store.s_store_sk'
-'  |  other predicates: substr(ca_zip, 1, 5) != substr(s_zip, 1, 5)'
-'  |  hosts=3 per-host-mem=383B'
-'  |  tuple-ids=3,2,0,1,4 row-size=192B cardinality=87208'
-'  |'
-'  |--15:EXCHANGE [BROADCAST]'
-'  |     hosts=3 per-host-mem=0B'
-'  |     tuple-ids=4 row-size=29B cardinality=12'
-'  |'
-'  07:HASH JOIN [INNER JOIN, BROADCAST]'
-'  |  hash predicates: store_sales.ss_item_sk = item.i_item_sk'
-'  |  hosts=3 per-host-mem=14.60KB'
-'  |  tuple-ids=3,2,0,1 row-size=163B cardinality=87208'
-'  |'
-'  |--14:EXCHANGE [BROADCAST]'
-'  |     hosts=3 per-host-mem=0B'
-'  |     tuple-ids=1 row-size=79B cardinality=171'
-'  |'
-'  06:HASH JOIN [INNER JOIN, BROADCAST]'
-'  |  hash predicates: customer.c_customer_sk = store_sales.ss_customer_sk'
-'  |  hosts=3 per-host-mem=234.75KB'
-'  |  tuple-ids=3,2,0 row-size=84B cardinality=87208'
-'  |'
-'  |--13:EXCHANGE [BROADCAST]'
-'  |     hosts=3 per-host-mem=0B'
-'  |     tuple-ids=0 row-size=43B cardinality=5082'
-'  |'
-'  05:HASH JOIN [INNER JOIN, PARTITIONED]'
-'  |  hash predicates: customer_address.ca_address_sk = customer.c_current_addr_sk'
-'  |  hosts=3 per-host-mem=429.69KB'
-'  |  tuple-ids=3,2 row-size=41B cardinality=87208'
-'  |'
-'  |--12:EXCHANGE [HASH(customer.c_current_addr_sk)]'
-'  |     hosts=3 per-host-mem=0B'
-'  |     tuple-ids=2 row-size=12B cardinality=100000'
-'  |'
-'  11:EXCHANGE [HASH(customer_address.ca_address_sk)]'
-'     hosts=3 per-host-mem=0B'
-'     tuple-ids=3 row-size=29B cardinality=50000'
-''
-'F05:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=15, BROADCAST]'
-'  04:SCAN HDFS [tpcds.store, RANDOM]'
-'     partitions=1/1 size=3.08KB'
-'     table stats: 12 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=4 row-size=29B cardinality=12'
-''
-'F04:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=14, BROADCAST]'
-'  01:SCAN HDFS [tpcds.item, RANDOM]'
-'     partitions=1/1 size=4.82MB'
-'     predicates: i_manager_id = 7'
-'     table stats: 18000 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=1 row-size=79B cardinality=171'
-''
-'F03:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=13, BROADCAST]'
-'  00:SCAN HDFS [tpcds.store_sales, RANDOM]'
-'     partitions=2/120 size=663.52KB'
-'     table stats: 183592 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=0 row-size=43B cardinality=5082'
-''
-'F01:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=12, HASH(customer.c_current_addr_sk)]'
-'  02:SCAN HDFS [tpcds.customer, RANDOM]'
-'     partitions=1/1 size=12.60MB'
-'     table stats: 100000 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=24.00MB'
-'     tuple-ids=2 row-size=12B cardinality=100000'
-''
-'F00:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=11, HASH(customer_address.ca_address_sk)]'
-'  03:SCAN HDFS [tpcds.customer_address, RANDOM]'
-'     partitions=1/1 size=5.25MB'
-'     table stats: 50000 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=3 row-size=29B cardinality=50000'
-====
----- QUERY
-# Tests explaining an insert query
-explain insert overwrite functional.alltypessmall (id, string_col)
-partition (year, month)
-select a.id, a.string_col, a.year, a.month from functional.alltypes a
-  left semi join functional.alltypesagg b on (a.id = b.id)
-where a.year > 2009 and a.month = 4
-union distinct
-select id, string_col, year, month from functional.alltypes
----- RESULTS
-'Estimated Per-Host Requirements: Memory=30.00MB VCores=3'
-''
-'F05:PLAN FRAGMENT [HASH(year,month)]'
-'  WRITE TO HDFS [functional.alltypessmall, OVERWRITE=true, PARTITION-KEYS=(year,month)]'
-'  |  partitions=96'
-'  |  hosts=3 per-host-mem=69.26KB'
-'  |'
-'  14:EXCHANGE [HASH(year,month)]'
-'     hosts=3 per-host-mem=0B'
-'     tuple-ids=3 row-size=28B cardinality=7600'
-''
-'F04:PLAN FRAGMENT [HASH(id,string_col,year,month)]'
-'  DATASTREAM SINK [FRAGMENT=F05, EXCHANGE=14, HASH(year,month)]'
-'  13:AGGREGATE [MERGE FINALIZE]'
-'  |  group by: id, string_col, year, month'
-'  |  hosts=3 per-host-mem=10.00MB'
-'  |  tuple-ids=3 row-size=28B cardinality=7600'
-'  |'
-'  12:EXCHANGE [HASH(id,string_col,year,month)]'
-'     hosts=3 per-host-mem=0B'
-'     tuple-ids=3 row-size=28B cardinality=7600'
-''
-'F03:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F04, EXCHANGE=12, HASH(id,string_col,year,month)]'
-'  11:AGGREGATE'
-'  |  group by: id, string_col, year, month'
-'  |  hosts=3 per-host-mem=10.00MB'
-'  |  tuple-ids=3 row-size=28B cardinality=7300'
-'  |'
-'  10:MERGE'
-'  |  hosts=3 per-host-mem=0B'
-'  |  tuple-ids=3 row-size=28B cardinality=7300'
-'  |'
-'  04:SCAN HDFS [functional.alltypes, RANDOM]'
-'     partitions=24/24 size=478.45KB'
-'     table stats: 7300 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=2 row-size=29B cardinality=7300'
-''
-'F02:PLAN FRAGMENT [HASH(a.id)]'
-'  DATASTREAM SINK [FRAGMENT=F04, EXCHANGE=12, HASH(id,string_col,year,month)]'
-'  05:AGGREGATE'
-'  |  group by: id, string_col, year, month'
-'  |  hosts=3 per-host-mem=10.00MB'
-'  |  tuple-ids=3 row-size=28B cardinality=300'
-'  |'
-'  09:MERGE'
-'  |  hosts=3 per-host-mem=0B'
-'  |  tuple-ids=3 row-size=28B cardinality=300'
-'  |'
-'  03:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]'
-'  |  hash predicates: a.id = b.id'
-'  |  hosts=3 per-host-mem=14.32KB'
-'  |  tuple-ids=0,1 row-size=33B cardinality=300'
-'  |'
-'  |--07:EXCHANGE [HASH(b.id)]'
-'  |     hosts=3 per-host-mem=0B'
-'  |     tuple-ids=1 row-size=4B cardinality=10000'
-'  |'
-'  06:EXCHANGE [HASH(a.id)]'
-'     hosts=3 per-host-mem=0B'
-'     tuple-ids=0 row-size=29B cardinality=300'
-''
-'F01:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=07, HASH(b.id)]'
-'  02:SCAN HDFS [functional.alltypesagg b, RANDOM]'
-'     partitions=10/10 size=743.67KB'
-'     table stats: 10000 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=1 row-size=4B cardinality=10000'
-''
-'F00:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=06, HASH(a.id)]'
-'  01:SCAN HDFS [functional.alltypes a, RANDOM]'
-'     partitions=1/24 size=19.71KB'
-'     table stats: 7300 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=0 row-size=29B cardinality=300'
-====
----- QUERY
-# Tests explaining an insert query to/from an HBase table
-explain insert into functional_hbase.alltypes
-select a.* from functional_hbase.alltypessmall a
-  cross join functional.alltypessmall b
-where a.year > 2009 and a.month = 4
-union all
-select * from functional_hbase.alltypessmall
----- RESULTS
-'Estimated Per-Host Requirements: Memory=1.02GB VCores=3'
-''
-'F03:PLAN FRAGMENT [UNPARTITIONED]'
-'  WRITE TO HBASE table=functional_hbase.alltypes'
-'  |  hosts=1 per-host-mem=unavailable'
-'  |'
-'  06:EXCHANGE [UNPARTITIONED]'
-'     hosts=100 per-host-mem=unavailable'
-'     tuple-ids=3 row-size=88B cardinality=596'
-''
-'F02:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F03, EXCHANGE=06, UNPARTITIONED]'
-'  08:MERGE'
-'  |  hosts=100 per-host-mem=0B'
-'  |  tuple-ids=3 row-size=88B cardinality=196'
-'  |'
-'  04:SCAN HBASE [functional_hbase.alltypessmall]'
-'     table stats: 100 rows total'
-'     column stats: all'
-'     hosts=100 per-host-mem=1.00GB'
-'     tuple-ids=2 row-size=97B cardinality=196'
-''
-'F00:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F03, EXCHANGE=06, UNPARTITIONED]'
-'  07:MERGE'
-'  |  hosts=100 per-host-mem=0B'
-'  |  tuple-ids=3 row-size=88B cardinality=400'
-'  |'
-'  03:CROSS JOIN [BROADCAST]'
-'  |  hosts=100 per-host-mem=0B'
-'  |  tuple-ids=0,1 row-size=97B cardinality=400'
-'  |'
-'  |--05:EXCHANGE [BROADCAST]'
-'  |     hosts=3 per-host-mem=0B'
-'  |     tuple-ids=1 row-size=0B cardinality=100'
-'  |'
-'  01:SCAN HBASE [functional_hbase.alltypessmall a]'
-'     predicates: a.year > 2009, a.month = 4'
-'     table stats: 100 rows total'
-'     column stats: all'
-'     hosts=100 per-host-mem=1.00GB'
-'     tuple-ids=0 row-size=97B cardinality=4'
-''
-'F01:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F00, EXCHANGE=05, BROADCAST]'
-'  02:SCAN HDFS [functional.alltypessmall b, RANDOM]'
-'     partitions=4/4 size=6.32KB'
-'     table stats: 100 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=1 row-size=0B cardinality=100'
-====
----- QUERY
-# Tests explaining an CTAS statement.
-explain create table t as
-select * from functional.alltypes
-where month = 2
----- RESULTS
-'Estimated Per-Host Requirements: Memory=16.02MB VCores=1'
-''
-'F00:PLAN FRAGMENT [RANDOM]'
-'  WRITE TO HDFS [functional.t, OVERWRITE=false]'
-'  |  partitions=1'
-'  |  hosts=3 per-host-mem=17.62KB'
-'  |'
-'  00:SCAN HDFS [functional.alltypes, RANDOM]'
-'     partitions=2/24 size=36.51KB'
-'     table stats: 7300 rows total'
-'     column stats: all'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=1 row-size=97B cardinality=560'
+select *
+from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
+---- RESULTS: VERIFY_IS_EQUAL
+'Per-Host Resource Reservation: Memory=136.00MB'
+'Per-Host Resource Estimates: Memory=388.41MB'
+''
+'F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
+'  PLAN-ROOT SINK'
+'  |  mem-estimate=0B mem-reservation=0B'
+'  |'
+'  04:EXCHANGE [UNPARTITIONED]'
+'     mem-estimate=0B mem-reservation=0B'
+'     tuple-ids=0,1 row-size=454B cardinality=5757710'
+''
+'F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3'
+'  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=04, UNPARTITIONED]'
+'  |  mem-estimate=0B mem-reservation=0B'
+'  02:HASH JOIN [INNER JOIN, BROADCAST]'
+'  |  hash predicates: l_orderkey = o_orderkey'
+'  |  runtime filters: RF000 <- o_orderkey'
+'  |  mem-estimate=300.41MB mem-reservation=136.00MB'
+'  |  tuple-ids=0,1 row-size=454B cardinality=5757710'
+'  |'
+'  |--03:EXCHANGE [BROADCAST]'
+'  |     mem-estimate=0B mem-reservation=0B'
+'  |     tuple-ids=1 row-size=191B cardinality=1500000'
+'  |'
+'  00:SCAN HDFS [tpch.lineitem, RANDOM]'
+row_regex:.*partitions=1/1 files=1 size=.*
+'     runtime filters: RF000 -> l_orderkey'
+'     table stats: 6001215 rows total'
+'     column stats: all'
+'     mem-estimate=88.00MB mem-reservation=0B'
+'     tuple-ids=0 row-size=263B cardinality=6001215'
+''
+'F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2'
+'  DATASTREAM SINK [FRAGMENT=F00, EXCHANGE=03, BROADCAST]'
+'  |  mem-estimate=0B mem-reservation=0B'
+'  01:SCAN HDFS [tpch.orders, RANDOM]'
+row_regex:.*partitions=1/1 files=1 size=.*
+'     table stats: 1500000 rows total'
+'     column stats: all'
+'     mem-estimate=88.00MB mem-reservation=0B'
+'     tuple-ids=1 row-size=191B cardinality=1500000'
 ====
 ---- QUERY
 # Tests the warning about missing table stats in the explain header.
@@ -335,70 +54,7 @@ from functional_avro.alltypes t1
   inner join functional_parquet.alltypessmall t2 on (t1.id = t2.id)
   left outer join functional_avro.alltypes t3 on (t2.id = t3.id)
 where t1.month = 1 and t2.year = 2009 and t3.bool_col = false
----- RESULTS
-'Estimated Per-Host Requirements: Memory=4.03GB VCores=3'
+---- RESULTS: VERIFY_IS_SUBSET
 'WARNING: The following tables are missing relevant table and/or column statistics.'
 'functional_avro.alltypes, functional_parquet.alltypessmall'
-''
-'F03:PLAN FRAGMENT [UNPARTITIONED]'
-'  09:AGGREGATE [MERGE FINALIZE]'
-'  |  output: sum(count(t1.int_col)), sum(sum(t2.float_col)), sum(count(t2.float_col)), sum(sum(t3.bigint_col))'
-'  |  hosts=3 per-host-mem=unavailable'
-'  |  tuple-ids=3 row-size=32B cardinality=1'
-'  |'
-'  08:EXCHANGE [UNPARTITIONED]'
-'     hosts=3 per-host-mem=unavailable'
-'     tuple-ids=3 row-size=32B cardinality=1'
-''
-'F00:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F03, EXCHANGE=08, UNPARTITIONED]'
-'  05:AGGREGATE'
-'  |  output: count(t1.int_col), sum(t2.float_col), count(t2.float_col), sum(t3.bigint_col)'
-'  |  hosts=3 per-host-mem=10.00MB'
-'  |  tuple-ids=3 row-size=32B cardinality=1'
-'  |'
-'  04:HASH JOIN [LEFT OUTER JOIN, BROADCAST]'
-'  |  hash predicates: t2.id = t3.id'
-'  |  other predicates: t3.bool_col = FALSE'
-'  |  hosts=3 per-host-mem=2.00GB'
-'  |  tuple-ids=0,1,2N row-size=37B cardinality=unavailable'
-'  |'
-'  |--07:EXCHANGE [BROADCAST]'
-'  |     hosts=3 per-host-mem=0B'
-'  |     tuple-ids=2 row-size=13B cardinality=unavailable'
-'  |'
-'  03:HASH JOIN [INNER JOIN, BROADCAST]'
-'  |  hash predicates: t1.id = t2.id'
-'  |  hosts=3 per-host-mem=2.00GB'
-'  |  tuple-ids=0,1 row-size=24B cardinality=unavailable'
-'  |'
-'  |--06:EXCHANGE [BROADCAST]'
-'  |     hosts=3 per-host-mem=0B'
-'  |     tuple-ids=1 row-size=12B cardinality=unavailable'
-'  |'
-'  00:SCAN HDFS [functional_avro.alltypes t1, RANDOM]'
-'     partitions=2/24 size=39.87KB'
-'     table stats: unavailable'
-'     columns missing stats: id, int_col'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=0 row-size=12B cardinality=unavailable'
-''
-'F02:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F00, EXCHANGE=07, BROADCAST]'
-'  02:SCAN HDFS [functional_avro.alltypes t3, RANDOM]'
-'     partitions=24/24 size=470.35KB'
-'     predicates: t3.bool_col = FALSE'
-'     table stats: unavailable'
-'     column stats: unavailable'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=2 row-size=13B cardinality=unavailable'
-''
-'F01:PLAN FRAGMENT [RANDOM]'
-'  DATASTREAM SINK [FRAGMENT=F00, EXCHANGE=06, BROADCAST]'
-'  01:SCAN HDFS [functional_parquet.alltypessmall t2, RANDOM]'
-'     partitions=4/4 size=9.63KB'
-'     table stats: unavailable'
-'     columns missing stats: id, float_col'
-'     hosts=3 per-host-mem=16.00MB'
-'     tuple-ids=1 row-size=12B cardinality=unavailable'
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/tests/custom_cluster/test_admission_controller.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_admission_controller.py b/tests/custom_cluster/test_admission_controller.py
index bf677d9..5201c75 100644
--- a/tests/custom_cluster/test_admission_controller.py
+++ b/tests/custom_cluster/test_admission_controller.py
@@ -296,7 +296,8 @@ class TestAdmissionController(TestAdmissionControllerBase, HS2TestSuite):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
-      impalad_args=impalad_admission_ctrl_flags(1, 1, 1234, 1024 * 1024 * 1024),
+      impalad_args=impalad_admission_ctrl_flags(1, 1, 10 * 1024 * 1024,
+          1024 * 1024 * 1024),
       statestored_args=_STATESTORED_ARGS)
   def test_trivial_coord_query_limits(self):
     """Tests that trivial coordinator only queries have negligible resource requirements.
@@ -311,7 +312,8 @@ class TestAdmissionController(TestAdmissionControllerBase, HS2TestSuite):
         "select * from functional.alltypestiny"]
     for query in non_trivial_queries:
       ex = self.execute_query_expect_failure(self.client, query)
-      assert "memory needed 4.00 GB is greater than pool max mem resources" in str(ex)
+      assert re.search("Rejected query from pool default-pool : request memory needed "
+          ".* is greater than pool max mem resources 10.00 MB", str(ex))
 
 class TestAdmissionControllerStress(TestAdmissionControllerBase):
   """Submits a number of queries (parameterized) with some delay between submissions

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9a29dfc9/tests/metadata/test_explain.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_explain.py b/tests/metadata/test_explain.py
index fd5b09a..3975e38 100644
--- a/tests/metadata/test_explain.py
+++ b/tests/metadata/test_explain.py
@@ -46,25 +46,29 @@ class TestExplain(ImpalaTestSuite):
         v.get_value('exec_option')['disable_codegen'] == False and\
         v.get_value('exec_option')['num_nodes'] != 1)
 
-  @pytest.mark.xfail(run=False, reason="Expected per-host mem requirements inconsistent")
+  @pytest.mark.skip_if(pytest.config.option.testing_remote_cluster,
+                     reason='Resource profile depends on number of nodes')
   def test_explain_level0(self, vector):
     vector.get_value('exec_option')['num_scanner_threads'] = self.NUM_SCANNER_THREADS
     vector.get_value('exec_option')['explain_level'] = 0
     self.run_test_case('QueryTest/explain-level0', vector)
 
-  @pytest.mark.xfail(run=False, reason="Expected per-host mem requirements inconsistent")
+  @pytest.mark.skip_if(pytest.config.option.testing_remote_cluster,
+                     reason='Resource profile depends on number of nodes')
   def test_explain_level1(self, vector):
     vector.get_value('exec_option')['num_scanner_threads'] = self.NUM_SCANNER_THREADS
     vector.get_value('exec_option')['explain_level'] = 1
     self.run_test_case('QueryTest/explain-level1', vector)
 
-  @pytest.mark.xfail(run=False, reason="The test for missing table stats fails for avro")
+  @pytest.mark.skip_if(pytest.config.option.testing_remote_cluster,
+                     reason='Resource profile depends on number of nodes')
   def test_explain_level2(self, vector):
     vector.get_value('exec_option')['num_scanner_threads'] = self.NUM_SCANNER_THREADS
     vector.get_value('exec_option')['explain_level'] = 2
     self.run_test_case('QueryTest/explain-level2', vector)
 
-  @pytest.mark.xfail(run=False, reason="The test for missing table stats fails for avro")
+  @pytest.mark.skip_if(pytest.config.option.testing_remote_cluster,
+                     reason='Resource profile depends on number of nodes')
   def test_explain_level3(self, vector):
     vector.get_value('exec_option')['num_scanner_threads'] = self.NUM_SCANNER_THREADS
     vector.get_value('exec_option')['explain_level'] = 3