You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by px...@apache.org on 2017/02/23 04:10:53 UTC

[3/3] hive git commit: HIVE-15955: make explain formatted to include opId and etc (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

HIVE-15955: make explain formatted to include opId and etc (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/759766ee
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/759766ee
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/759766ee

Branch: refs/heads/master
Commit: 759766eeb5d8047d4acdbd2faca755985bb5a39f
Parents: 89310fe
Author: Pengcheng Xiong <px...@apache.org>
Authored: Wed Feb 22 20:10:19 2017 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Wed Feb 22 20:10:19 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/ExplainTask.java | 10 ++-
 .../AnnotateReduceSinkOutputOperator.java       | 73 ++++++++++++++++++++
 .../hadoop/hive/ql/optimizer/Optimizer.java     |  4 ++
 .../hadoop/hive/ql/plan/ReduceSinkDesc.java     | 14 ++++
 .../clientpositive/explain_formatted_oid.q      | 18 +++++
 .../clientpositive/explain_formatted_oid.q.out  | 38 ++++++++++
 ql/src/test/results/clientpositive/input4.q.out |  2 +-
 ql/src/test/results/clientpositive/join0.q.out  |  2 +-
 .../results/clientpositive/parallel_join0.q.out |  2 +-
 .../test/results/clientpositive/plan_json.q.out |  2 +-
 .../clientpositive/vector_outer_join3.q.out     |  6 +-
 .../clientpositive/vector_outer_join4.q.out     |  6 +-
 .../clientpositive/vector_outer_join6.q.out     |  4 +-
 13 files changed, 168 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
index 74cec3e..086ccb2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
@@ -115,6 +115,7 @@ import org.slf4j.LoggerFactory;
 public class ExplainTask extends Task<ExplainWork> implements Serializable {
   private static final long serialVersionUID = 1L;
   public static final String EXPL_COLUMN_NAME = "Explain";
+  public static final String OUTPUT_OPERATORS = "OutputOperators:";
   private final Set<Operator<?>> visitedOps = new HashSet<Operator<?>>();
   private boolean isLogical = false;
   protected final Logger LOG;
@@ -790,10 +791,17 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
         String appender = isLogical ? " (" + operator.getOperatorId() + ")" : "";
         JSONObject jsonOut = outputPlan(operator.getConf(), out, extended,
             jsonOutput, jsonOutput ? 0 : indent, appender);
-        if (this.work != null && this.work.isUserLevelExplain()) {
+        if (this.work != null && (this.work.isUserLevelExplain() || this.work.isFormatted())) {
           if (jsonOut != null && jsonOut.length() > 0) {
             ((JSONObject) jsonOut.get(JSONObject.getNames(jsonOut)[0])).put("OperatorId:",
                 operator.getOperatorId());
+            if (!this.work.isUserLevelExplain() && this.work.isFormatted()
+                && operator instanceof ReduceSinkOperator) {
+              ((JSONObject) jsonOut.get(JSONObject.getNames(jsonOut)[0])).put(
+                  OUTPUT_OPERATORS,
+                  Arrays.toString(((ReduceSinkOperator) operator).getConf().getOutputOperators()
+                      .toArray()));
+            }
           }
         }
         if (jsonOutput) {

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AnnotateReduceSinkOutputOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AnnotateReduceSinkOutputOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AnnotateReduceSinkOutputOperator.java
new file mode 100644
index 0000000..0b61f4b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AnnotateReduceSinkOutputOperator.java
@@ -0,0 +1,73 @@
+/**
+ * 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.hadoop.hive.ql.optimizer;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
+import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
+import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+
+/**
+ * Implementation of AnnotateReduceSinkOutputOperator optimization step.
+ */
+public class AnnotateReduceSinkOutputOperator extends Transform {
+  @Override
+  public ParseContext transform(ParseContext pctx) throws SemanticException {
+
+    // 1. We apply the transformation
+    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    opRules.put(new RuleRegExp("R1",
+      "(" + ReduceSinkOperator.getOperatorName() + "%)"), new ReduceSinkOutputOperatorAnnotator());
+    GraphWalker ogw = new DefaultGraphWalker(new DefaultRuleDispatcher(null, opRules, null));
+    ArrayList<Node> topNodes = new ArrayList<Node>();
+    topNodes.addAll(pctx.getTopOps().values());
+    ogw.startWalking(topNodes, null);
+    return pctx;
+  }
+
+  private static class ReduceSinkOutputOperatorAnnotator implements NodeProcessor {
+    @Override
+    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
+        Object... nodeOutputs) throws SemanticException {
+      ReduceSinkOperator rs = (ReduceSinkOperator) nd;
+      List<Operator<? extends OperatorDesc>> children = rs.getChildOperators();
+      List<String> outputOperators = new ArrayList<>();
+      for (Operator<? extends OperatorDesc> operator : children) {
+        outputOperators.add(operator.getOperatorId());
+      }
+      rs.getConf().setOutputOperators(outputOperators);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
index c6287e4..a3a19f4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
@@ -232,6 +232,10 @@ public class Optimizer {
       transformations.add(new SimpleFetchAggregation());
     }
 
+    if (pctx.getContext().getExplainConfig() != null
+        && pctx.getContext().getExplainConfig().isFormatted()) {
+      transformations.add(new AnnotateReduceSinkOutputOperator());
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
index b8c2d42..d77a223 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
@@ -80,6 +80,12 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
   private String outputName;
 
   /**
+   * Holds the name of the output operators
+   * that this reduce sink is outputing to.
+   */
+  private List<String> outputOperators;
+
+  /**
    * The partition columns (CLUSTER BY or DISTRIBUTE BY in Hive language).
    * Partition columns decide the reducer that the current row goes to.
    * Partition columns are not passed to reducer.
@@ -587,4 +593,12 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
     }
     return new ReduceSinkOperatorExplainVectorization(this, vectorDesc);
   }
+
+  public List<String> getOutputOperators() {
+    return outputOperators;
+  }
+
+  public void setOutputOperators(List<String> outputOperators) {
+    this.outputOperators = outputOperators;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/queries/clientpositive/explain_formatted_oid.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explain_formatted_oid.q b/ql/src/test/queries/clientpositive/explain_formatted_oid.q
new file mode 100644
index 0000000..932f119
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/explain_formatted_oid.q
@@ -0,0 +1,18 @@
+set hive.auto.convert.join=false;
+
+create table srcTable (key string, value string);
+
+explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN srcTable z ON (x.value = z.value);
+
+explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+union
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value);
+

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/explain_formatted_oid.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_formatted_oid.q.out b/ql/src/test/results/clientpositive/explain_formatted_oid.q.out
new file mode 100644
index 0000000..46c2090
--- /dev/null
+++ b/ql/src/test/results/clientpositive/explain_formatted_oid.q.out
@@ -0,0 +1,38 @@
+PREHOOK: query: create table srcTable (key string, value string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcTable
+POSTHOOK: query: create table srcTable (key string, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcTable
+PREHOOK: query: explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN srcTable z ON (x.value = z.value)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN srcTable z ON (x.value = z.value)
+POSTHOOK: type: QUERY
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"x","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_0","children":{"Filter Operator":{"predicate:":"(key is not null and value is not null) (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_17","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_2","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NON
 E","value expressions:":"_col1 (type: string)","OperatorId:":"RS_9","OutputOperators:":"[JOIN_11]"}}}}}}}},{"TableScan":{"alias:":"y","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_3","children":{"Filter Operator":{"predicate:":"key is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_18","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_5","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col1 (type: string)","OperatorId:":"RS_10","OutputOperators:":"[JOIN_11]"}}}}}}}}],"Reduce 
 Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"outputColumnNames:":["_col0","_col1","_col3"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_11","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_20"}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_21","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col0 (type: string), _col3 (type: string)","OperatorId
 :":"RS_12","OutputOperators:":"[JOIN_14]"}}}},{"TableScan":{"alias:":"z","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_6","children":{"Filter Operator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_19","children":{"Select Operator":{"expressions:":"value (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_8","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_13","OutputOperators:":"[JOIN_14]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type:
  string)"},"outputColumnNames:":["_col0","_col3","_col4"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_14","children":{"Select Operator":{"expressions:":"_col0 (type: string), _col4 (type: string), _col3 (type: string)","outputColumnNames:":["_col0","_col1","_col2"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"},"OperatorId:":"FS_16"}}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_22"}}}}}}
+PREHOOK: query: explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+union
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+union
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+POSTHOOK: type: QUERY
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1, Stage-5"},"Stage-3":{"DEPENDENT STAGES":"Stage-2, Stage-8"},"Stage-5":{"ROOT STAGE":"TRUE"},"Stage-7":{"ROOT STAGE":"TRUE"},"Stage-8":{"DEPENDENT STAGES":"Stage-7, Stage-10"},"Stage-10":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"x","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_0","children":{"Filter Operator":{"predicate:":"(key is not null and value is not null) (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_60","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_2","children":{"Reduce Outp
 ut Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col1 (type: string)","OperatorId:":"RS_19","OutputOperators:":"[JOIN_21]"}}}}}}}},{"TableScan":{"alias:":"y","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_3","children":{"Filter Operator":{"predicate:":"key is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_61","children":{"Select Operator":{"expressions:":"key (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_5","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Nu
 m rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_20","OutputOperators:":"[JOIN_21]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_21","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_68"}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_69","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data 
 size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col0 (type: string)","OperatorId:":"RS_22","OutputOperators:":"[JOIN_24]"}}}},{"TableScan":{"OperatorId:":"TS_73","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_23","OutputOperators:":"[JOIN_24]"}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col1 (type: string)"},"outputColumnNames:":["_col0","_col4"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_24","children":{"Select Operator":{"expressions:":"_col0 (type: string), _col4 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":
 "SEL_25","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_70"}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_71","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_52","children":{"Group By Operator":{"keys:":"_col0 (type: string), _col1 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_55","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 
 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_56","OutputOperators:":"[GBY_57]"}}}}}}}},{"TableScan":{"OperatorId:":"TS_77","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_52","children":{"Group By Operator":{"keys:":"_col0 (type: string), _col1 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_55","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_56","OutputOperators:":"[GBY_57]"}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"keys:":"KEY._col0 (type: string), KEY._col1 (type: string)","mode:":"mergepartial","out
 putColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_57","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"},"OperatorId:":"FS_59"}}}}}},"Stage-5":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"srctable","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_6","children":{"Filter Operator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_62","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames
 :":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_8","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_12","children":{"Group By Operator":{"keys:":"_col1 (type: string), _col0 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_15","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_16","OutputOperators:":"[GBY_17]"}}}}}}}}}}}},{"TableScan":{"alias:":"srctable","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_9","children":{"Filter Ope
 rator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_63","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_11","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_12","children":{"Group By Operator":{"keys:":"_col1 (type: string), _col0 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_15","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data siz
 e: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_16","OutputOperators:":"[GBY_17]"}}}}}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"keys:":"KEY._col0 (type: string), KEY._col1 (type: string)","mode:":"mergepartial","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_17","children":{"Select Operator":{"expressions:":"_col0 (type: string)","outputColumnNames:":["_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_18","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_72"}}}}}}}},"Stage-7":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"x","Statistics:":"Num rows:
  1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_26","children":{"Filter Operator":{"predicate:":"(key is not null and value is not null) (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_64","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_28","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col1 (type: string)","OperatorId:":"RS_45","OutputOperators:":"[JOIN_47]"}}}}}}}},{"TableScan":{"alias:":"y","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_29","children":
 {"Filter Operator":{"predicate:":"key is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_65","children":{"Select Operator":{"expressions:":"key (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_31","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_46","OutputOperators:":"[JOIN_47]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_47","children":{"File Output Operator"
 :{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_74"}}}}}},"Stage-8":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_75","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col0 (type: string)","OperatorId:":"RS_48","OutputOperators:":"[JOIN_50]"}}}},{"TableScan":{"OperatorId:":"TS_79","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_49","OutputOpera
 tors:":"[JOIN_50]"}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col1 (type: string)"},"outputColumnNames:":["_col0","_col4"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_50","children":{"Select Operator":{"expressions:":"_col0 (type: string), _col4 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_51","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_76"}}}}}}}},"Stage-10":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"srctable","Statistics:":"Num rows: 1 Data size: 0 Basic stats:
  PARTIAL Column stats: NONE","OperatorId:":"TS_32","children":{"Filter Operator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_66","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_34","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_38","children":{"Group By Operator":{"keys:":"_col1 (type: string), _col0 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_41","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0
  (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_42","OutputOperators:":"[GBY_43]"}}}}}}}}}}}},{"TableScan":{"alias:":"srctable","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_35","children":{"Filter Operator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_67","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_37","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_38","children":{"Group By Operator":{"keys:":"_col1 (type: string), _col0 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_
 col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_41","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_42","OutputOperators:":"[GBY_43]"}}}}}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"keys:":"KEY._col0 (type: string), KEY._col1 (type: string)","mode:":"mergepartial","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_43","children":{"Select Operator":{"expressions:":"_col0 (type: string)","outputColumnNames:":["_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_44","children":{"File Output Operator":{"compressed:":"false","table:
 ":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_78"}}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_80"}}}}}}

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/input4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/input4.q.out b/ql/src/test/results/clientpositive/input4.q.out
index 83912f6..6984318 100644
--- a/ql/src/test/results/clientpositive/input4.q.out
+++ b/ql/src/test/results/clientpositive/input4.q.out
@@ -44,7 +44,7 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN FORMATTED
 SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"TableScan":{"alias:":"input4alias","Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"value (type: string), key (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"ListSink":{}}}}}}}}}}
+{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"TableScan":{"alias:":"input4alias","Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"value (type: string), key (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_1","children":{"ListSink":{"OperatorId:":"LIST_SINK_3"}}}}}}}}}}
 PREHOOK: query: SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 PREHOOK: type: QUERY
 PREHOOK: Input: default@input4

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join0.q.out b/ql/src/test/results/clientpositive/join0.q.out
index c02319e..b1e85a3 100644
--- a/ql/src/test/results/clientpositive/join0.q.out
+++ b/ql/src/test/results/clientpositive/join0.q.out
@@ -112,7 +112,7 @@ SELECT src1.key as k1, src1.value as v1,
   (SELECT * FROM src WHERE src.key < 10) src2
   SORT BY k1, v1, k2, v2
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COM
 PLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","
 output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"}}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"
 org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_0","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"FIL_13","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_2","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)","OperatorId:":"RS_6","OutputOper
 ators:":"[JOIN_8]"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_3","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"FIL_14","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)","OperatorId:":"RS_7","OutputOperators:":"[JOIN_8]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_
 col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"JOIN_8","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_15"}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_16","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"RS_10","OutputOperators:":"[SEL_11]"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.
 reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_11","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"},"OperatorId:":"FS_12"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_17"}}}}}}
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/parallel_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parallel_join0.q.out b/ql/src/test/results/clientpositive/parallel_join0.q.out
index c02319e..b1e85a3 100644
--- a/ql/src/test/results/clientpositive/parallel_join0.q.out
+++ b/ql/src/test/results/clientpositive/parallel_join0.q.out
@@ -112,7 +112,7 @@ SELECT src1.key as k1, src1.value as v1,
   (SELECT * FROM src WHERE src.key < 10) src2
   SORT BY k1, v1, k2, v2
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COM
 PLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","
 output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"}}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"
 org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_0","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"FIL_13","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_2","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)","OperatorId:":"RS_6","OutputOper
 ators:":"[JOIN_8]"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_3","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"FIL_14","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)","OperatorId:":"RS_7","OutputOperators:":"[JOIN_8]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_
 col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"JOIN_8","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"},"OperatorId:":"FS_15"}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_16","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"RS_10","OutputOperators:":"[SEL_11]"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.
 reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_11","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"},"OperatorId:":"FS_12"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_17"}}}}}}
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/plan_json.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/plan_json.q.out b/ql/src/test/results/clientpositive/plan_json.q.out
index 75d5b73..ba6d0be 100644
--- a/ql/src/test/results/clientpositive/plan_json.q.out
+++ b/ql/src/test/results/clientpositive/plan_json.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: EXPLAIN FORMATTED SELECT count(1) FROM src
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN FORMATTED SELECT count(1) FROM src
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"1","Processor Tree:":{"ListSink":{}}}}}}
+{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_8"}}}}}}