You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/09/16 21:18:42 UTC

[24/50] [abbrv] hive git commit: HIVE-11792: User explain in tez does not preserve ordering (Prasanth Jayachandran reviewed by Pengcheng Xiong)

HIVE-11792: User explain in tez does not preserve ordering (Prasanth Jayachandran reviewed by Pengcheng Xiong)


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

Branch: refs/heads/spark
Commit: da0be3db7741e59813adcc197dc545b57dcc3c0a
Parents: 8bcd07d
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Mon Sep 14 02:57:19 2015 -0500
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Mon Sep 14 02:57:19 2015 -0500

----------------------------------------------------------------------
 .../hadoop/hive/common/jsonexplain/tez/Op.java  |    8 +-
 .../hive/common/jsonexplain/tez/Stage.java      |   14 +-
 .../common/jsonexplain/tez/TezJsonParser.java   |   17 +-
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |   24 +-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |    4 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |    9 +-
 .../authorization_explain.q.java1.7.out         |    2 +-
 .../authorization_explain.q.java1.8.out         |    2 +-
 .../clientpositive/explain_dependency.q.out     |   18 +-
 .../clientpositive/explain_dependency2.q.out    |   16 +-
 .../results/clientpositive/input4.q.java1.7.out |    2 +-
 .../results/clientpositive/input4.q.java1.8.out |    2 +-
 .../results/clientpositive/join0.q.java1.7.out  |    2 +-
 .../results/clientpositive/join0.q.java1.8.out  |    4 +-
 .../results/clientpositive/parallel_join0.q.out |    2 +-
 .../clientpositive/plan_json.q.java1.7.out      |    2 +-
 .../clientpositive/plan_json.q.java1.8.out      |    2 +-
 .../clientpositive/tez/constprog_dpp.q.out      |    4 +-
 .../clientpositive/tez/explainuser_1.q.out      |  496 +--
 .../clientpositive/tez/explainuser_2.q.out      | 3280 +++++++++---------
 .../clientpositive/tez/explainuser_3.q.out      |   10 +-
 21 files changed, 1953 insertions(+), 1967 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java
index 9ecba7c..d0c1037 100644
--- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java
+++ b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Op.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.common.jsonexplain.tez;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -62,7 +62,7 @@ public final class Op {
       JSONObject mapjoinObj = opObject.getJSONObject("Map Join Operator");
       // get the map for posToVertex
       JSONObject verticeObj = mapjoinObj.getJSONObject("input vertices:");
-      Map<String, String> posToVertex = new HashMap<>();
+      Map<String, String> posToVertex = new LinkedHashMap<>();
       for (String pos : JSONObject.getNames(verticeObj)) {
         String vertexName = verticeObj.getString(pos);
         posToVertex.put(pos, vertexName);
@@ -83,7 +83,7 @@ public final class Op {
       // update the keys to use vertex name
       JSONObject keys = mapjoinObj.getJSONObject("keys:");
       if (keys.length() != 0) {
-        JSONObject newKeys = new JSONObject();
+        JSONObject newKeys = new JSONObject(new LinkedHashMap<>());
         for (String key : JSONObject.getNames(keys)) {
           String vertexName = posToVertex.get(key);
           if (vertexName != null) {
@@ -116,7 +116,7 @@ public final class Op {
   }
 
   /**
-   * @param out
+   * @param printer
    * @param indentFlag
    * @param branchOfJoinOp
    *          This parameter is used to show if it is a branch of a Join

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java
index c5a78b5..455d59f 100644
--- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java
+++ b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/Stage.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.common.jsonexplain.tez;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -28,8 +27,6 @@ import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.hadoop.fs.Path;
-import org.codehaus.jackson.JsonParseException;
-import org.codehaus.jackson.map.JsonMappingException;
 import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
@@ -176,16 +173,11 @@ public final class Stage {
    * @param opName
    * @param opObj
    * @return
-   * @throws JSONException
-   * @throws JsonParseException
-   * @throws JsonMappingException
-   * @throws IOException
    * @throws Exception
    *           This method address the create table operator, fetch operator,
    *           etc
    */
-  Op extractOp(String opName, JSONObject opObj) throws JSONException, JsonParseException,
-      JsonMappingException, IOException, Exception {
+  Op extractOp(String opName, JSONObject opObj) throws Exception {
     List<Attr> attrs = new ArrayList<>();
     Vertex v = null;
     if (opObj.length() > 0) {
@@ -198,7 +190,7 @@ public final class Stage {
           JSONObject attrObj = (JSONObject) o;
           if (attrObj.length() > 0) {
             if (name.equals("Processor Tree:")) {
-              JSONObject object = new JSONObject();
+              JSONObject object = new JSONObject(new LinkedHashMap<>());
               object.put(name, attrObj);
               v = new Vertex(null, object, parser);
               v.extractOpTree();
@@ -232,7 +224,7 @@ public final class Stage {
     return false;
   }
 
-  public void print(Printer printer, List<Boolean> indentFlag) throws JSONException, Exception {
+  public void print(Printer printer, List<Boolean> indentFlag) throws Exception {
     // print stagename
     if (parser.printSet.contains(this)) {
       printer.println(TezJsonParser.prefixString(indentFlag) + " Please refer to the previous "

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java
index c6ee4f6..20ce27b 100644
--- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java
+++ b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/tez/TezJsonParser.java
@@ -18,11 +18,10 @@
 
 package org.apache.hadoop.hive.common.jsonexplain.tez;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -31,27 +30,23 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.common.jsonexplain.JsonParser;
-import org.codehaus.jackson.JsonParseException;
-import org.codehaus.jackson.map.JsonMappingException;
-import org.json.JSONException;
 import org.json.JSONObject;
 
 public final class TezJsonParser implements JsonParser {
-  public final Map<String, Stage> stages = new HashMap<String, Stage>();;
+  public final Map<String, Stage> stages = new LinkedHashMap<>();
   protected final Log LOG;
   // the object that has been printed.
-  public final Set<Object> printSet = new HashSet<>();
+  public final Set<Object> printSet = new LinkedHashSet<>();
   // the vertex that should be inlined. <Operator, list of Vertex that is
   // inlined>
-  public final Map<Op, List<Connection>> inlineMap = new HashMap<>();
+  public final Map<Op, List<Connection>> inlineMap = new LinkedHashMap<>();
 
   public TezJsonParser() {
     super();
     LOG = LogFactory.getLog(this.getClass().getName());
   }
 
-  public void extractStagesAndPlans(JSONObject inputObject) throws JSONException,
-      JsonParseException, JsonMappingException, Exception, IOException {
+  public void extractStagesAndPlans(JSONObject inputObject) throws Exception {
     // extract stages
     JSONObject dependency = inputObject.getJSONObject("STAGE DEPENDENCIES");
     if (dependency != null && dependency.length() > 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/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 21de3cf..a74a8ad 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
@@ -97,7 +97,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       throws Exception {
     assert(work.getDependency());
 
-    JSONObject outJSONObject = new JSONObject();
+    JSONObject outJSONObject = new JSONObject(new LinkedHashMap<>());
     List<Map<String, String>> inputTableInfo = new ArrayList<Map<String, String>>();
     List<Map<String, String>> inputPartitionInfo = new ArrayList<Map<String, String>>();
     for (ReadEntity input: work.getInputs()) {
@@ -133,7 +133,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
   public JSONObject getJSONLogicalPlan(PrintStream out, ExplainWork work) throws Exception {
     isLogical = true;
 
-    JSONObject outJSONObject = new JSONObject();
+    JSONObject outJSONObject = new JSONObject(new LinkedHashMap<>());
     boolean jsonOutput = work.isFormatted();
     if (jsonOutput) {
       out = null;
@@ -181,7 +181,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
 
     // If the user asked for a formatted output, dump the json output
     // in the output stream
-    JSONObject outJSONObject = new JSONObject();
+    JSONObject outJSONObject = new JSONObject(new LinkedHashMap<>());
 
     if (jsonOutput) {
       out = null;
@@ -339,7 +339,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
     BaseSemanticAnalyzer analyzer = work.getAnalyzer();
     HiveOperation operation = SessionState.get().getHiveOperation();
 
-    JSONObject object = new JSONObject();
+    JSONObject object = new JSONObject(new LinkedHashMap<>());
     Object jsonInput = toJson("INPUTS", toString(analyzer.getInputs()), out, work);
     if (work.isFormatted()) {
       object.put("INPUTS", jsonInput);
@@ -402,7 +402,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
 
     TreeMap<Object, Object> tree = new TreeMap<Object, Object>();
     tree.putAll(mp);
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     if (out != null && hasHeader && !mp.isEmpty()) {
       out.println();
     }
@@ -446,7 +446,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
           }
           if (jsonOutput) {
             for (TezWork.Dependency dep: (List<TezWork.Dependency>)ent.getValue()) {
-              JSONObject jsonDep = new JSONObject();
+              JSONObject jsonDep = new JSONObject(new LinkedHashMap<>());
               jsonDep.put("parent", dep.getName());
               jsonDep.put("type", dep.getType());
               json.accumulate(ent.getKey().toString(), jsonDep);
@@ -475,7 +475,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
           }
           if (jsonOutput) {
             for (SparkWork.Dependency dep: (List<SparkWork.Dependency>) ent.getValue()) {
-              JSONObject jsonDep = new JSONObject();
+              JSONObject jsonDep = new JSONObject(new LinkedHashMap<>());
               jsonDep.put("parent", dep.getName());
               jsonDep.put("type", dep.getShuffleType());
               jsonDep.put("partitions", dep.getNumPartitions());
@@ -613,7 +613,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       }
     }
 
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     // If this is an operator then we need to call the plan generation on the
     // conf and then the children
     if (work instanceof Operator) {
@@ -783,7 +783,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
 
     if (jsonOutput) {
       if (keyJSONObject != null) {
-        JSONObject ret = new JSONObject();
+        JSONObject ret = new JSONObject(new LinkedHashMap<>());
         ret.put(keyJSONObject, json);
         return ret;
       }
@@ -841,7 +841,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       throws Exception {
 
     boolean first = true;
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     if (out != null) {
       out.print(indentString(indent));
       out.print(task.getId());
@@ -946,7 +946,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       out.println("STAGE DEPENDENCIES:");
     }
 
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     for (Task task : tasks) {
       JSONObject jsonOut = outputDependencies(task, out, json, jsonOutput, appendTaskType, 2);
       if (jsonOutput && jsonOut != null) {
@@ -965,7 +965,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
       out.println("STAGE PLANS:");
     }
 
-    JSONObject json = jsonOutput ? new JSONObject() : null;
+    JSONObject json = jsonOutput ? new JSONObject(new LinkedHashMap<>()) : null;
     for (Task task : tasks) {
       outputPlan(task, out, json, isExtended, jsonOutput, 2);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 73263ee..3a6ec1a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -289,7 +290,8 @@ public class TezTask extends Task<TezWork> {
     DAG dag = DAG.create(work.getName());
 
     // set some info for the query
-    JSONObject json = new JSONObject().put("context", "Hive").put("description", ctx.getCmd());
+    JSONObject json = new JSONObject(new LinkedHashMap()).put("context", "Hive")
+        .put("description", ctx.getCmd());
     String dagInfo = json.toString();
 
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
index 87638da..2caa7ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
@@ -17,8 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.hooks;
 
-import java.io.Serializable;
-import java.util.List;
+import java.util.LinkedHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -26,15 +25,11 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
-import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
-import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.plan.ExplainWork;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
@@ -161,7 +156,7 @@ public class ATSHook implements ExecuteWithHookContext {
   TimelineEntity createPreHookEvent(String queryId, String query, JSONObject explainPlan,
       long startTime, String user, String requestuser, int numMrJobs, int numTezJobs, String opId) throws Exception {
 
-    JSONObject queryObj = new JSONObject();
+    JSONObject queryObj = new JSONObject(new LinkedHashMap<>());
     queryObj.put("queryText", query);
     queryObj.put("queryPlan", explainPlan);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out b/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out
index 8209c6a..b7ec209 100644
--- a/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/authorization_explain.q.java1.7.out
@@ -44,4 +44,4 @@ PREHOOK: query: explain formatted authorization use default
 PREHOOK: type: SWITCHDATABASE
 POSTHOOK: query: explain formatted authorization use default
 POSTHOOK: type: SWITCHDATABASE
-{"OUTPUTS":[],"INPUTS":["database:default"],"OPERATION":"SWITCHDATABASE","CURRENT_USER":"hive_test_user"}
+{"INPUTS":["database:default"],"OUTPUTS":[],"CURRENT_USER":"hive_test_user","OPERATION":"SWITCHDATABASE"}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out b/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out
index bb2bee9..b7ec209 100644
--- a/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/authorization_explain.q.java1.8.out
@@ -44,4 +44,4 @@ PREHOOK: query: explain formatted authorization use default
 PREHOOK: type: SWITCHDATABASE
 POSTHOOK: query: explain formatted authorization use default
 POSTHOOK: type: SWITCHDATABASE
-{"CURRENT_USER":"hive_test_user","OPERATION":"SWITCHDATABASE","INPUTS":["database:default"],"OUTPUTS":[]}
+{"INPUTS":["database:default"],"OUTPUTS":[],"CURRENT_USER":"hive_test_user","OPERATION":"SWITCHDATABASE"}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/explain_dependency.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_dependency.q.out b/ql/src/test/results/clientpositive/explain_dependency.q.out
index fa4dc59..dbfc482 100644
--- a/ql/src/test/results/clientpositive/explain_dependency.q.out
+++ b/ql/src/test/results/clientpositive/explain_dependency.q.out
@@ -70,14 +70,14 @@ POSTHOOK: query: -- Simple select queries, union queries and join queries
 EXPLAIN DEPENDENCY 
   SELECT key, count(1) FROM srcpart WHERE ds IS NOT NULL GROUP BY key
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: EXPLAIN DEPENDENCY 
   SELECT key, count(1) FROM (SELECT key, value FROM src) subq1 GROUP BY key
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY 
   SELECT key, count(1) FROM (SELECT key, value FROM src) subq1 GROUP BY key
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: EXPLAIN DEPENDENCY 
   SELECT * FROM (
     SELECT key, value FROM src UNION ALL SELECT key, value FROM srcpart WHERE ds IS NOT NULL
@@ -88,36 +88,36 @@ POSTHOOK: query: EXPLAIN DEPENDENCY
     SELECT key, value FROM src UNION ALL SELECT key, value FROM srcpart WHERE ds IS NOT NULL
   ) S1
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: EXPLAIN DEPENDENCY 
   SELECT S1.key, S2.value FROM src S1 JOIN srcpart S2 ON S1.key = S2.key WHERE ds IS NOT NULL
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY 
   SELECT S1.key, S2.value FROM src S1 JOIN srcpart S2 ON S1.key = S2.key WHERE ds IS NOT NULL
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: -- With views
 EXPLAIN DEPENDENCY SELECT * FROM V1
 PREHOOK: type: QUERY
 POSTHOOK: query: -- With views
 EXPLAIN DEPENDENCY SELECT * FROM V1
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@v1","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v1]"}]}
+{"input_tables":[{"tablename":"default@v1","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v1]"}],"input_partitions":[]}
 PREHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V2
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V2
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}]}
+{"input_tables":[{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V3
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V3
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@v3","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v3]"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v3]"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}]}
+{"input_tables":[{"tablename":"default@v3","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v3]"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v3]"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V4
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V4
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@v4","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@v1","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v4]"},{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v4]"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v4, default@v1]"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}]}
+{"input_tables":[{"tablename":"default@v4","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@v1","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v4]"},{"tablename":"default@v2","tabletype":"VIRTUAL_VIEW","tableParents":"[default@v4]"},{"tablename":"default@src","tabletype":"MANAGED_TABLE","tableParents":"[default@v4, default@v1]"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v2]"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: -- The table should show up in the explain dependency even if none
 -- of the partitions are selected.
 CREATE VIEW V5 as SELECT * FROM srcpart where ds = '10'
@@ -136,4 +136,4 @@ PREHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V5
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN DEPENDENCY SELECT * FROM V5
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@v5","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v5]"}]}
+{"input_tables":[{"tablename":"default@v5","tabletype":"VIRTUAL_VIEW"},{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE","tableParents":"[default@v5]"}],"input_partitions":[]}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/explain_dependency2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_dependency2.q.out b/ql/src/test/results/clientpositive/explain_dependency2.q.out
index cb4e6c3..7973a60 100644
--- a/ql/src/test/results/clientpositive/explain_dependency2.q.out
+++ b/ql/src/test/results/clientpositive/explain_dependency2.q.out
@@ -8,14 +8,14 @@ POSTHOOK: query: -- This test is used for testing EXPLAIN DEPENDENCY command
 -- select from a table which does not involve a map-reduce job
 EXPLAIN DEPENDENCY SELECT * FROM src
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: -- select from a table which involves a map-reduce job
 EXPLAIN DEPENDENCY SELECT count(*) FROM src
 PREHOOK: type: QUERY
 POSTHOOK: query: -- select from a table which involves a map-reduce job
 EXPLAIN DEPENDENCY SELECT count(*) FROM src
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@src","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: -- select from a partitioned table which does not involve a map-reduce job
 -- and some partitions are being selected
 EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds is not null
@@ -24,7 +24,7 @@ POSTHOOK: query: -- select from a partitioned table which does not involve a map
 -- and some partitions are being selected
 EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds is not null
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: -- select from a partitioned table which does not involve a map-reduce job
 -- and none of the partitions are being selected
 EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds = '1'
@@ -33,7 +33,7 @@ POSTHOOK: query: -- select from a partitioned table which does not involve a map
 -- and none of the partitions are being selected
 EXPLAIN DEPENDENCY SELECT * FROM srcpart where ds = '1'
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: -- select from a partitioned table which involves a map-reduce job
 -- and some partitions are being selected
 EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds is not null
@@ -42,7 +42,7 @@ POSTHOOK: query: -- select from a partitioned table which involves a map-reduce
 -- and some partitions are being selected
 EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds is not null
 POSTHOOK: type: QUERY
-{"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[{"partitionName":"default@srcpart@ds=2008-04-08/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-08/hr=12"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=11"},{"partitionName":"default@srcpart@ds=2008-04-09/hr=12"}]}
 PREHOOK: query: -- select from a partitioned table which involves a map-reduce job
 -- and none of the partitions are being selected
 EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds = '1'
@@ -51,7 +51,7 @@ POSTHOOK: query: -- select from a partitioned table which involves a map-reduce
 -- and none of the partitions are being selected
 EXPLAIN DEPENDENCY SELECT count(*) FROM srcpart where ds = '1'
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@srcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: create table tstsrcpart like srcpart
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -66,11 +66,11 @@ PREHOOK: type: QUERY
 POSTHOOK: query: -- select from a partitioned table with no partitions which does not involve a map-reduce job
 EXPLAIN DEPENDENCY SELECT * FROM tstsrcpart where ds is not null
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@tstsrcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@tstsrcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}
 PREHOOK: query: -- select from a partitioned table with no partitions which involves a map-reduce job
 EXPLAIN DEPENDENCY SELECT count(*) FROM tstsrcpart where ds is not null
 PREHOOK: type: QUERY
 POSTHOOK: query: -- select from a partitioned table with no partitions which involves a map-reduce job
 EXPLAIN DEPENDENCY SELECT count(*) FROM tstsrcpart where ds is not null
 POSTHOOK: type: QUERY
-{"input_partitions":[],"input_tables":[{"tablename":"default@tstsrcpart","tabletype":"MANAGED_TABLE"}]}
+{"input_tables":[{"tablename":"default@tstsrcpart","tabletype":"MANAGED_TABLE"}],"input_partitions":[]}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/input4.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/input4.q.java1.7.out b/ql/src/test/results/clientpositive/input4.q.java1.7.out
index dccf625..eaeedcb 100644
--- a/ql/src/test/results/clientpositive/input4.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/input4.q.java1.7.out
@@ -48,7 +48,7 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN FORMATTED
 SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 POSTHOOK: type: QUERY
-{"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"TableScan":{"alias:":"input4alias","children":{"Select Operator":{"expressions:":"value (type: string), key (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"ListSink":{}},"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE"}}}}},"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}}}
+{"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":{}}}}}}}}}}
 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/da0be3db/ql/src/test/results/clientpositive/input4.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/input4.q.java1.8.out b/ql/src/test/results/clientpositive/input4.q.java1.8.out
index bc408de..eaeedcb 100644
--- a/ql/src/test/results/clientpositive/input4.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/input4.q.java1.8.out
@@ -48,7 +48,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":{"Processor Tree:":{"TableScan":{"alias:":"input4alias","Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"ListSink":{}},"outputColumnNames:":["_col0","_col1"],"expressions:":"value (type: string), key (type: string)"}}}},"limit:":"-1"}}}}
+{"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":{}}}}}}}}}}
 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/da0be3db/ql/src/test/results/clientpositive/join0.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join0.q.java1.7.out b/ql/src/test/results/clientpositive/join0.q.java1.7.out
index fa55ffb..f7e52f6 100644
--- a/ql/src/test/results/clientpositive/join0.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/join0.q.java1.7.out
@@ -118,7 +118,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 PLANS":{"Stage-2":{"Map Reduce":{"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"],"children":{"File Output Operator":{"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}},"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE"}},"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"sort order:":"++++","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (ty
 pe: string), _col3 (type: string)"}}}}]}},"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Join Operator":{"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"children":{"File Output Operator":{"compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe","input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"}}},"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","condition map:":[{"":"Inner Join 0 to 1"}]}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 166 Data
  size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE"}},{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats
 : COMPLETE Column stats: NONE"}}]}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}},"STAGE DEPENDENCIES":{"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}}}
+{"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: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","outp
 ut 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: 182 Data size: 1939 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: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoo
 p.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
 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/da0be3db/ql/src/test/results/clientpositive/join0.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join0.q.java1.8.out b/ql/src/test/results/clientpositive/join0.q.java1.8.out
index 443f6a3..f7e52f6 100644
--- a/ql/src/test/results/clientpositive/join0.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/join0.q.java1.8.out
@@ -1,5 +1,6 @@
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: -- JAVA_VERSION_SPECIFIC_OUTPUT
+-- SORT_QUERY_RESULTS
 
 EXPLAIN
 SELECT src1.key as k1, src1.value as v1, 
@@ -10,6 +11,7 @@ SELECT src1.key as k1, src1.value as v1,
   SORT BY k1, v1, k2, v2
 PREHOOK: type: QUERY
 POSTHOOK: query: -- JAVA_VERSION_SPECIFIC_OUTPUT
+-- SORT_QUERY_RESULTS
 
 EXPLAIN
 SELECT src1.key as k1, src1.value as v1, 
@@ -116,7 +118,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":{"Reduce Operator Tree:":{"Join Operator":{"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","keys:":{},"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"}}},"condition map:":[{"":"Inner Join 0 to 1"}],"outputColumnNames:":["_col0","_col1","_col2","_col3"]}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"Statist
 ics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"value expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"outputColumnNames:":["_col0","_col1"],"expressions:":"key (type: string), value (type: string)"}},"predicate:":"(key < 10) (type: boolean)"}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"value expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"
 outputColumnNames:":["_col0","_col1"],"expressions:":"key (type: string), value (type: string)"}},"predicate:":"(key < 10) (type: boolean)"}}}}]}},"Stage-2":{"Map Reduce":{"Reduce Operator Tree:":{"Select Operator":{"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)"}},"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"sort order:":"++++","Statistics:":"Num rows: 182 
 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)"}}}}]}},"Stage-0":{"Fetch Operator":{"Processor Tree:":{"ListSink":{}},"limit:":"-1"}}}}
+{"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: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","outp
 ut 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: 182 Data size: 1939 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: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoo
 p.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
 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/da0be3db/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 0f57a51..3aef936 100644
--- a/ql/src/test/results/clientpositive/parallel_join0.q.out
+++ b/ql/src/test/results/clientpositive/parallel_join0.q.out
@@ -116,7 +116,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 PLANS":{"Stage-2":{"Map Reduce":{"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"],"children":{"File Output Operator":{"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}},"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE"}},"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"sort order:":"++++","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (ty
 pe: string), _col3 (type: string)"}}}}]}},"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Join Operator":{"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"children":{"File Output Operator":{"compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe","input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"}}},"Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","condition map:":[{"":"Inner Join 0 to 1"}]}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 166 Data
  size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE"}},{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats
 : COMPLETE Column stats: NONE"}}]}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}},"STAGE DEPENDENCIES":{"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}}}
+{"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: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","table:":{"input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","outp
 ut 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: 182 Data size: 1939 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: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE","table:":{"input format:":"org.apache.hadoo
 p.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
 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/da0be3db/ql/src/test/results/clientpositive/plan_json.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/plan_json.q.java1.7.out b/ql/src/test/results/clientpositive/plan_json.q.java1.7.out
index 0979c70..2faa08a 100644
--- a/ql/src/test/results/clientpositive/plan_json.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/plan_json.q.java1.7.out
@@ -10,4 +10,4 @@ POSTHOOK: query: -- explain plan json:  the query gets the formatted json output
 
 EXPLAIN FORMATTED SELECT count(1) FROM src
 POSTHOOK: type: QUERY
-{"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Group By Operator":{"mode:":"mergepartial","aggregations:":["count(VALUE._col0)"],"outputColumnNames:":["_col0"],"children":{"File Output Operator":{"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE"}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","children":{"Select Operator":{"children":{"Group By Operator":{"mode:":"hash","aggregations:":["count(1)"],"outputColumnNames:":["_col0"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: bigint)","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
 "}},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE"}},"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE"}}]}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}},"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-1"}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-1"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Select Operator":{"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Group By Operator":{"aggregations:":["count(1)"],"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","value expressions:":"_col0 (type: bigint)"}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column 
 stats: COMPLETE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/plan_json.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/plan_json.q.java1.8.out b/ql/src/test/results/clientpositive/plan_json.q.java1.8.out
index 13842b4..2faa08a 100644
--- a/ql/src/test/results/clientpositive/plan_json.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/plan_json.q.java1.8.out
@@ -10,4 +10,4 @@ POSTHOOK: query: -- explain plan json:  the query gets the formatted json output
 
 EXPLAIN FORMATTED SELECT count(1) FROM src
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-1"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"mode:":"mergepartial","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Select Operator":{"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}},"outputColumnNames:":["_col0"],"expressions:":"_col0 (type: bigint)"}},"outputColumnNames:":["_col0"]}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size
 : 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Select Operator":{"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Group By Operator":{"aggregations:":["count(_col0)"],"mode:":"hash","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Reduce Output Operator":{"value expressions:":"_col0 (type: bigint)","sort order:":"","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE"}},"outputColumnNames:":["_col0"]}},"outputColumnNames:":["_col0"],"expressions:":"1 (type: int)"}}}}]}},"Stage-0":{"Fetch Operator":{"Processor Tree:":{"ListSink":{}},"limit:":"-1"}}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-1"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Select Operator":{"Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Group By Operator":{"aggregations:":["count(1)"],"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","value expressions:":"_col0 (type: bigint)"}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column 
 stats: COMPLETE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE","table:":{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}

http://git-wip-us.apache.org/repos/asf/hive/blob/da0be3db/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out b/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out
index b2b2371..0bc964b 100644
--- a/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out
+++ b/ql/src/test/results/clientpositive/tez/constprog_dpp.q.out
@@ -54,10 +54,10 @@ Stage-0
          File Output Operator [FS_16]
             compressed:false
             Statistics:Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-            table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
+            table:{"input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"}
             Merge Join Operator [MERGEJOIN_20]
             |  condition map:[{"":"Left Outer Join0 to 1"}]
-            |  keys:{"1":"_col0 (type: int)","0":"id (type: int)"}
+            |  keys:{"0":"id (type: int)","1":"_col0 (type: int)"}
             |  outputColumnNames:["_col0"]
             |  Statistics:Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             |<-Map 6 [SIMPLE_EDGE]