Browse Source

HADOOP-4084. Add explain plan capabilities to Hive Query Language.
(Ashish Thusoo via dhruba)



git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@696736 13f79535-47bb-0310-9956-ffa450edef68

Dhruba Borthakur 16 years ago
parent
commit
1ed5959b1f
100 changed files with 1585 additions and 785 deletions
  1. 3 0
      CHANGES.txt
  2. 1 1
      src/contrib/hive/build-common.xml
  3. 2 1
      src/contrib/hive/ql/build.xml
  4. 29 22
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
  5. 0 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java
  6. 0 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
  7. 1 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
  8. 330 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
  9. 82 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java
  10. 322 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
  11. 9 7
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
  12. 16 3
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
  13. 0 74
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/UDAFRegistry.java
  14. 0 238
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/UDFRegistry.java
  15. 14 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
  16. 3 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
  17. 1 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
  18. 54 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
  19. 0 60
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionInfo.java
  20. 15 3
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
  21. 3 3
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
  22. 0 3
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/PartitionPruner.java
  23. 2 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java
  24. 15 16
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
  25. 1 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
  26. 5 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeInfo.java
  27. 0 316
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeRegistry.java
  28. 7 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
  29. 25 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/aggregationDesc.java
  30. 20 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/alterTableDesc.java
  31. 1 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/collectDesc.java
  32. 5 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/copyWork.java
  33. 24 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java
  34. 13 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/descTableDesc.java
  35. 2 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/dropTableDesc.java
  36. 28 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/explain.java
  37. 79 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/explainWork.java
  38. 1 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/explosionDesc.java
  39. 6 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeColumnDesc.java
  40. 14 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeConstantDesc.java
  41. 9 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeDesc.java
  42. 6 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeFieldDesc.java
  43. 54 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeFuncDesc.java
  44. 6 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeIndexDesc.java
  45. 7 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeNullDesc.java
  46. 1 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/extractDesc.java
  47. 5 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/fileSinkDesc.java
  48. 2 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/filterDesc.java
  49. 1 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/forwardDesc.java
  50. 19 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/groupByDesc.java
  51. 29 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/joinCond.java
  52. 46 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/joinDesc.java
  53. 2 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/loadDesc.java
  54. 4 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/loadFileDesc.java
  55. 4 2
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/loadTableDesc.java
  56. 17 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredWork.java
  57. 4 1
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/moveWork.java
  58. 5 4
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/partitionDesc.java
  59. 6 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/reduceSinkDesc.java
  60. 5 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/scriptDesc.java
  61. 2 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/selectDesc.java
  62. 12 0
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/showTablesDesc.java
  63. 14 3
      src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/tableDesc.java
  64. 34 7
      src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
  65. 1 1
      src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/exec/TestCompositeHiveObject.java
  66. 2 2
      src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
  67. 1 1
      src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/exec/TestJEXL.java
  68. 1 1
      src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
  69. 5 0
      src/contrib/hive/ql/src/test/queries/clientpositive/cast1.q
  70. 5 0
      src/contrib/hive/ql/src/test/queries/clientpositive/groupby1.q
  71. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/groupby2.q
  72. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/groupby3.q
  73. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/groupby4.q
  74. 6 0
      src/contrib/hive/ql/src/test/queries/clientpositive/groupby5.q
  75. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/groupby6.q
  76. 5 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input1.q
  77. 5 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input10.q
  78. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input11.q
  79. 6 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input12.q
  80. 7 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input13.q
  81. 9 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input14.q
  82. 3 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input15.q
  83. 3 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input2.q
  84. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input3.q
  85. 2 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input4.q
  86. 9 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input5.q
  87. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input6.q
  88. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input7.q
  89. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input8.q
  90. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input9.q
  91. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input_part1.q
  92. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q
  93. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/input_testxpath.q
  94. 5 1
      src/contrib/hive/ql/src/test/queries/clientpositive/inputddl1.q
  95. 2 0
      src/contrib/hive/ql/src/test/queries/clientpositive/inputddl2.q
  96. 2 0
      src/contrib/hive/ql/src/test/queries/clientpositive/inputddl3.q
  97. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/join1.q
  98. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/join2.q
  99. 4 0
      src/contrib/hive/ql/src/test/queries/clientpositive/join3.q
  100. 15 0
      src/contrib/hive/ql/src/test/queries/clientpositive/join4.q

+ 3 - 0
CHANGES.txt

@@ -625,6 +625,9 @@ Trunk (unreleased changes)
     HADOOP-4200. Fix a bug in the test-patch.sh script.
     (Ramya R via nigel)
 
+    HADOOP-4084. Add explain plan capabilities to Hive Query Language.
+    (Ashish Thusoo via dhruba)
+
 Release 0.18.1 - 2008-09-17
 
   IMPROVEMENTS

+ 1 - 1
src/contrib/hive/build-common.xml

@@ -205,7 +205,7 @@
     <mkdir dir="${test.log.dir}"/>
     <junit showoutput="${test.output}" printsummary="yes" haltonfailure="no"
            fork="yes" maxmemory="256m" dir="${basedir}" timeout="${test.timeout}"
-      errorProperty="tests.failed" failureProperty="tests.failed" filtertrace="off" >
+           errorProperty="tests.failed" failureProperty="tests.failed" filtertrace="off">
       <!--
       <jvmarg value="-Xdebug"/>
       <jvmarg value="-Xrunjdwp:transport=dt_socket,address=8000,server=y,suspend=y"/> -->

+ 2 - 1
src/contrib/hive/ql/build.xml

@@ -61,7 +61,8 @@
               templatePath="${ql.test.template.dir}" template="TestCliDriver.vm" 
               queryDirectory="${ql.test.query.dir}/clientpositive" 
               queryFile="${qfile}"
-              resultsDirectory="${ql.test.results.dir}/clientpositive" className="TestCliDriver"/>
+              resultsDirectory="${ql.test.results.dir}/clientpositive" className="TestCliDriver"
+              logFile="${test.log.dir}/testclidrivergen.log"/>
 
   </target>
 

+ 29 - 22
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/Driver.java

@@ -18,12 +18,9 @@
 
 package org.apache.hadoop.hive.ql;
 
-import java.io.File;
 import java.io.InputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
+import java.io.Serializable;
 import java.util.*;
-
 import org.antlr.runtime.tree.CommonTree;
 
 import org.apache.commons.lang.StringUtils;
@@ -37,6 +34,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.MapRedTask;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.ExecDriver;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -65,15 +63,15 @@ public class Driver implements CommandProcessor {
     return terminator;
   }
   
-  public int countJobs(Collection tasks) {
+  public int countJobs(List<Task<? extends Serializable>> tasks) {
     if (tasks == null)
       return 0;
     int jobs = 0;
-    for (Object task: tasks) {
+    for (Task<? extends Serializable> task: tasks) {
       if ((task instanceof ExecDriver) || (task instanceof MapRedTask)) {
         jobs++;
       }
-      jobs += countJobs(((Task) task).getChildTasks());
+      jobs += countJobs(task.getChildTasks());
     }
     return jobs;
   }
@@ -104,11 +102,13 @@ public class Driver implements CommandProcessor {
     conf.setVar(HiveConf.ConfVars.HIVEQUERYID, command);
 
     try {
+      
+      TaskFactory.resetId();
+
       BaseSemanticAnalyzer sem;
       LOG.info("Starting command: " + command);
 
-      if (resStream != null)
-      {
+      if (resStream != null) {
         resStream.close();
         resStream = null;
       }
@@ -126,21 +126,32 @@ public class Driver implements CommandProcessor {
       // Do semantic analysis and plan generation
       sem.analyze(tree, ctx);
       LOG.info("Semantic Analysis Completed");
-      for(Task rootTask: sem.getRootTasks()) {
-        rootTask.initialize(conf);
-      }
 
       jobs = countJobs(sem.getRootTasks());
       if (jobs > 0) {
         console.printInfo("Total MapReduce jobs = " + jobs);
       }
+      
+ 
+      String jobname = Utilities.abbreviate(command, maxlen - 6);
+      int curJob = 0;
+      for(Task<? extends Serializable> rootTask: sem.getRootTasks()) {
+        // assumption that only top level tasks are map-reduce tasks
+        if ((rootTask instanceof ExecDriver) || (rootTask instanceof MapRedTask)) {
+          curJob ++;
+          if(noName) {
+            conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, jobname + "(" + curJob + "/" + jobs + ")");
+          }
+        }
+        rootTask.initialize(conf);
+      }
 
       // A very simple runtime that keeps putting runnable takss
       // on a list and when a job completes, it puts the children at the back of the list
       // while taking the job to run from the front of the list
-      Queue<Task> runnable = new LinkedList<Task>();
+      Queue<Task<? extends Serializable>> runnable = new LinkedList<Task<? extends Serializable>>();
 
-      for(Task rootTask:sem.getRootTasks()) {
+      for(Task<? extends Serializable> rootTask:sem.getRootTasks()) {
         if (runnable.offer(rootTask) == false) {
           LOG.error("Could not insert the first task into the queue");
           return (1);
@@ -148,11 +159,7 @@ public class Driver implements CommandProcessor {
       }
 
       while(runnable.peek() != null) {
-        Task tsk = runnable.remove();
-
-        if(noName) {
-          conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, Utilities.abbreviate(command, maxlen));
-        }
+        Task<? extends Serializable> tsk = runnable.remove();
 
         int exitVal = tsk.execute();
         if (exitVal != 0) {
@@ -166,13 +173,13 @@ public class Driver implements CommandProcessor {
           continue;
         }
 
-        for(Object child: tsk.getChildTasks()) {
+        for(Task<? extends Serializable> child: tsk.getChildTasks()) {
           // Check if the child is runnable
-          if (!((Task)child).isRunnable()) {
+          if (!child.isRunnable()) {
             continue;
           }
 
-          if (runnable.offer((Task)child) == false) {
+          if (runnable.offer(child) == false) {
             LOG.error("Could not add child task to queue");
           }
         }

+ 0 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java

@@ -24,8 +24,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.plan.copyWork;
 import org.apache.hadoop.hive.ql.parse.LoadSemanticAnalyzer;
 import org.apache.hadoop.util.StringUtils;

+ 0 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java

@@ -47,8 +47,6 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
-import org.apache.hadoop.util.StringUtils;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import com.facebook.thrift.TException;

+ 1 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java

@@ -214,7 +214,7 @@ public class ExecDriver extends Task<mapredWork> implements Serializable {
     }
     return (returnVal);
   }
-
+  
   private static void printUsage() {
     System.out.println("ExecDriver -plan <plan-file> [-jobconf k1=v1 [-jobconf k2=v2] ...]");
     System.exit(1);

+ 330 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java

@@ -0,0 +1,330 @@
+/**
+ * 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.exec;
+
+import java.io.FileOutputStream;
+import java.io.PrintStream;
+import java.io.Serializable;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hive.ql.plan.explain;
+import org.apache.hadoop.hive.ql.plan.explainWork;
+import org.apache.hadoop.util.StringUtils;
+
+
+/**
+ * ExplainTask implementation
+ * 
+ **/
+public class ExplainTask extends Task<explainWork> implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  public int execute() {
+    
+    try {
+      // If this is an explain plan then return from here
+      PrintStream out = new PrintStream(new FileOutputStream(work.getResFile()));
+
+      // Print out the parse AST
+      outputAST(work.getAstStringTree(), out, 0);
+      out.println();
+      
+      outputDependencies(out, work.getRootTasks(), 0);
+      out.println();
+      
+      // Go over all the tasks and dump out the plans
+      outputStagePlans(out, work.getRootTasks(), 0);
+
+      return (0);
+    }
+    catch (Exception e) {
+      console.printError("Failed with exception " +   e.getMessage(), "\n" + StringUtils.stringifyException(e));
+      return (1);
+    }
+  }
+
+  private String indentString(int indent) {
+    StringBuilder sb = new StringBuilder();
+    for(int i=0; i<indent; ++i) {
+      sb.append(" ");
+    }
+    
+    return sb.toString();
+  }
+
+  private void outputMap(Map<?, ?> mp, String header,
+                         PrintStream out, boolean extended, int indent) 
+    throws Exception {
+
+    boolean first_el = true;
+    for(Entry<?,?> ent: mp.entrySet()) {
+      if (first_el) {
+        out.println(header);
+      }
+      first_el = false;
+              
+      // Print the key
+      out.print(indentString(indent));
+      out.printf("%s ", ent.getKey().toString());
+      // Print the value
+      if (isPrintable(ent.getValue())) {
+        out.print(ent.getValue());
+        out.println();
+      }
+      else if (ent.getValue() instanceof Serializable) {
+        out.println();
+        outputPlan((Serializable)ent.getValue(), out, extended, indent+2);
+      }
+    }
+  }
+
+  private void outputList(List<?> l, String header,
+                          PrintStream out, boolean extended, int indent) 
+    throws Exception {
+  
+    boolean first_el = true;
+    boolean nl = false;
+    for(Object o: l) {
+      if (first_el) {
+        out.print(header);
+      }
+              
+      if (isPrintable(o)) {
+        if (!first_el) {
+          out.print(", ");
+        } else {
+          out.print(" ");
+        }
+                
+        out.print(o);
+        nl = true;
+      }
+      else if (o instanceof Serializable) {
+        if (first_el) {
+          out.println();
+        }
+        outputPlan((Serializable)o, out, extended, indent+2);
+      }
+              
+      first_el = false;
+    }
+            
+    if (nl) {
+      out.println();
+    }
+  }
+
+  private boolean isPrintable(Object val) {
+    if (val instanceof String ||
+        val instanceof Integer ||
+        val instanceof Byte ||
+        val instanceof Float ||
+        val instanceof Double) {
+      return true;
+    }
+
+    if (val.getClass().isPrimitive()) {
+      return true;
+    }
+    
+    return false;
+  }
+
+  private void outputPlan(Serializable work, PrintStream out, boolean extended, int indent) 
+    throws Exception {
+    // Check if work has an explain annotation
+    Annotation note = work.getClass().getAnnotation(explain.class);
+    
+    if (note instanceof explain) {
+      explain xpl_note = (explain)note;
+      if (extended || xpl_note.normalExplain()) {
+        out.print(indentString(indent));
+        out.println(xpl_note.displayName());
+      }
+    }
+
+    // If this is an operator then we need to call the plan generation on the conf and then
+    // the children
+    if (work instanceof Operator) {
+      Operator<? extends Serializable> operator = (Operator<? extends Serializable>) work;
+      if (operator.getConf() != null) {
+        outputPlan(operator.getConf(), out, extended, indent);
+      }
+      if (operator.getChildOperators() != null) {
+        for(Operator<? extends Serializable> op: operator.getChildOperators()) {
+          outputPlan(op, out, extended, indent+2);
+        }
+      }
+      return;
+    }
+    
+    // We look at all methods that generate values for explain
+    for(Method m: work.getClass().getMethods()) {
+      int prop_indents = indent+2;
+      note = m.getAnnotation(explain.class);
+
+      if (note instanceof explain) {
+        explain xpl_note = (explain)note;
+
+        if (extended || xpl_note.normalExplain()) {
+          
+          Object val = m.invoke(work);
+
+          if (val == null) {
+            continue;
+          }
+          
+          String header = null;
+          if (!xpl_note.displayName().equals("")){
+            header = indentString(prop_indents) + xpl_note.displayName() +":";
+          } else {
+            prop_indents = indent;
+            header = indentString(prop_indents);
+          }
+
+          if (isPrintable(val)) {
+            
+            out.printf("%s ", header);
+            out.println(val);
+            continue;
+          }
+          // Try this as a map
+          try {
+            // Go through the map and print out the stuff
+            Map<?,?> mp = (Map<?,?>)val;
+            outputMap(mp, header, out, extended, prop_indents+2);
+            continue;
+          }
+          catch (ClassCastException ce) {
+            // Ignore - all this means is that this is not a map
+          }
+
+          // Try this as a list
+          try {
+            List<?> l = (List<?>)val;
+            outputList(l, header, out, extended, prop_indents+2);
+            
+            continue;
+          }
+          catch (ClassCastException ce) {
+            // Ignore
+          }
+          
+
+          // Finally check if it is serializable
+          try {
+            Serializable s = (Serializable)val;
+            out.println(header);
+            outputPlan(s, out, extended, prop_indents+2);
+            
+            continue;
+          }
+          catch (ClassCastException ce) {
+            // Ignore
+          }
+        }
+      }
+    }
+  }
+  
+  private void outputPlan(Task<? extends Serializable> task, PrintStream out, 
+                          boolean extended, HashSet<Task<? extends Serializable>> displayedSet,
+                          int indent) 
+    throws Exception {
+  
+    if (displayedSet.contains(task)) {
+      return;
+    }
+    displayedSet.add(task);
+    
+    out.print(indentString(indent));
+    out.printf("Stage: %s\n", task.getId());
+    // Start by getting the work part of the task and call the output plan for the work
+    outputPlan(task.getWork(), out, extended, indent+2);
+    out.println();
+    if (task.getChildTasks() != null) {
+      for(Task<? extends Serializable> child: task.getChildTasks()) {
+        outputPlan(child, out, extended, displayedSet, indent);
+      }
+    }
+  }
+
+  private void outputDependencies(Task<? extends Serializable> task, PrintStream out, int indent) 
+    throws Exception {
+    
+    out.print(indentString(indent));
+    out.printf("%s", task.getId());
+    if (task.getParentTasks() == null || task.getParentTasks().isEmpty()) {
+      out.print(" is a root stage");
+    }
+    else {
+      out.print(" depends on stages: ");
+      boolean first = true;
+      for(Task<? extends Serializable> parent: task.getParentTasks()) {
+        if (!first) {
+          out.print(", ");
+        }
+        first = false;
+        out.print(parent.getId());
+      }
+    }
+    out.println();
+    
+    if (task.getChildTasks() != null) {
+      for(Task<? extends Serializable> child: task.getChildTasks()) {
+        outputDependencies(child, out, indent);
+      }
+    }
+  }
+
+  public void outputAST(String treeString, PrintStream out, int indent) {
+    out.print(indentString(indent));
+    out.println("ABSTRACT SYNTAX TREE:");
+    out.print(indentString(indent+2));
+    out.println(treeString);    
+  }
+
+  public void outputDependencies(PrintStream out, 
+                                 List<Task<? extends Serializable>> rootTasks,
+                                 int indent) 
+    throws Exception {
+    out.print(indentString(indent));
+    out.println("STAGE DEPENDENCIES:");
+    for(Task<? extends Serializable> rootTask: rootTasks) {
+      outputDependencies(rootTask, out, indent+2);
+    }
+  }
+
+  public void outputStagePlans(PrintStream out, 
+                               List<Task<? extends Serializable>> rootTasks,
+                               int indent) 
+    throws Exception {
+    out.print(indentString(indent));
+    out.println("STAGE PLANS:");
+    for(Task<? extends Serializable> rootTask: rootTasks) {
+      outputPlan(rootTask, out, work.getExtended(),
+                 new HashSet<Task<? extends Serializable>>(), indent+2);
+    }
+  }
+}

+ 82 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java

@@ -0,0 +1,82 @@
+/**
+ * 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.exec;
+
+public class FunctionInfo {
+
+  private String displayName;
+
+  private OperatorType opType;
+
+  private boolean isOperator;
+  
+  private Class<? extends UDF> udfClass;
+
+  private Class<? extends UDAF> udafClass;
+
+  public static enum OperatorType { NO_OP, PREFIX, INFIX, POSTFIX };
+
+  public FunctionInfo(String displayName, Class<? extends UDF> udfClass, Class<? extends UDAF> udafClass) {
+    assert(udfClass == null || udafClass == null);
+    this.displayName = displayName;
+    opType = OperatorType.NO_OP;
+    isOperator = false;
+    this.udfClass = udfClass;
+    this.udafClass = udafClass;
+  }
+
+  public FunctionInfo(String displayName, OperatorType opType, Class<? extends UDF> udfClass) {
+    this.displayName = displayName;
+    this.opType = opType;
+    this.udfClass = udfClass;
+    this.udafClass = null;
+  }
+
+  public boolean isAggFunction() {
+    return (udafClass != null && udfClass == null);
+  }
+
+  public boolean isOperator() {
+    return isOperator;
+  }
+
+  public void setIsOperator(boolean val) {
+    isOperator = val;
+  }
+  
+  public void setOpType(OperatorType opt) {
+    opType = opt;
+  }
+  
+  public OperatorType getOpType() {
+    return opType;
+  }
+
+  public Class<? extends UDF> getUDFClass() {
+    return udfClass;
+  }
+
+  public Class<? extends UDAF> getUDAFClass() {
+    return udafClass;
+  }
+  
+  public String getDisplayName() {
+    return displayName;
+  }
+}

+ 322 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java

@@ -0,0 +1,322 @@
+/**
+ * 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.exec;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.lang.Void;
+
+import org.apache.hadoop.hive.ql.exec.FunctionInfo.OperatorType;
+import org.apache.hadoop.hive.ql.parse.TypeInfo;
+import org.apache.hadoop.hive.ql.udf.*;
+
+public class FunctionRegistry {
+
+  private static Log LOG = LogFactory.getLog("org.apache.hadoop.hive.ql.exec.FunctionRegistry");
+
+  /**
+   * The mapping from expression function names to expression classes.
+   */
+  static HashMap<String, FunctionInfo> mFunctions;
+  static {
+    mFunctions = new HashMap<String, FunctionInfo>();
+    registerUDF("default_sample_hashfn", UDFDefaultSampleHashFn.class, 
+                OperatorType.PREFIX, false);
+    registerUDF("concat", UDFConcat.class, OperatorType.PREFIX, false);
+    registerUDF("substr", UDFSubstr.class, OperatorType.PREFIX, false);
+    registerUDF("str_eq", UDFStrEq.class, OperatorType.PREFIX, false);
+    registerUDF("str_ne", UDFStrNe.class, OperatorType.PREFIX, false);
+    registerUDF("str_gt", UDFStrGt.class, OperatorType.PREFIX, false);
+    registerUDF("str_lt", UDFStrLt.class, OperatorType.PREFIX, false);
+    registerUDF("str_ge", UDFStrGe.class, OperatorType.PREFIX, false);
+    registerUDF("str_le", UDFStrLe.class, OperatorType.PREFIX, false);
+
+    registerUDF("upper", UDFUpper.class, OperatorType.PREFIX, false);
+    registerUDF("lower", UDFLower.class, OperatorType.PREFIX, false);
+    registerUDF("ucase", UDFUpper.class, OperatorType.PREFIX, false);
+    registerUDF("lcase", UDFLower.class, OperatorType.PREFIX, false);
+    registerUDF("trim", UDFTrim.class, OperatorType.PREFIX, false);
+    registerUDF("ltrim", UDFLTrim.class, OperatorType.PREFIX, false);
+    registerUDF("rtrim", UDFRTrim.class, OperatorType.PREFIX, false);
+
+    registerUDF("like", UDFLike.class, OperatorType.INFIX, true);
+    registerUDF("rlike", UDFRegExp.class, OperatorType.INFIX, true);
+    registerUDF("regexp", UDFRegExp.class, OperatorType.INFIX, true);
+    registerUDF("regexp_replace", UDFRegExpReplace.class, OperatorType.PREFIX, false);
+
+    registerUDF("+", UDFOPPlus.class, OperatorType.INFIX, true);
+    registerUDF("-", UDFOPMinus.class, OperatorType.INFIX, true);
+    registerUDF("*", UDFOPMultiply.class, OperatorType.INFIX, true);
+    registerUDF("/", UDFOPDivide.class, OperatorType.INFIX, true);
+    registerUDF("%", UDFOPMod.class, OperatorType.INFIX, true);
+
+    registerUDF("&", UDFOPBitAnd.class, OperatorType.INFIX, true);
+    registerUDF("|", UDFOPBitOr.class, OperatorType.INFIX, true);
+    registerUDF("^", UDFOPBitXor.class, OperatorType.INFIX, true);
+    registerUDF("~", UDFOPBitNot.class, OperatorType.INFIX, true);
+
+    registerUDF("=", UDFOPEqual.class, OperatorType.INFIX, true);
+    registerUDF("==", UDFOPEqual.class, OperatorType.INFIX, true, "=");
+    registerUDF("<>", UDFOPNotEqual.class, OperatorType.INFIX, true);
+    registerUDF("<", UDFOPLessThan.class, OperatorType.INFIX, true);
+    registerUDF("<=", UDFOPEqualOrLessThan.class, OperatorType.INFIX, true);
+    registerUDF(">", UDFOPGreaterThan.class, OperatorType.INFIX, true);
+    registerUDF(">=", UDFOPEqualOrGreaterThan.class, OperatorType.INFIX, true);
+
+    registerUDF("and", UDFOPAnd.class, OperatorType.INFIX, true);
+    registerUDF("&&", UDFOPAnd.class, OperatorType.INFIX, true, "and");
+    registerUDF("or", UDFOPOr.class, OperatorType.INFIX, true);
+    registerUDF("||", UDFOPOr.class, OperatorType.INFIX, true, "or");
+    registerUDF("not", UDFOPNot.class, OperatorType.INFIX, true);
+    registerUDF("!", UDFOPNot.class, OperatorType.INFIX, true, "not");
+
+    registerUDF("isnull", UDFOPNull.class, OperatorType.POSTFIX, true, "is null");
+    registerUDF("isnotnull", UDFOPNotNull.class, OperatorType.POSTFIX, true, "is not null");
+
+    // Aliases for Java Class Names
+    // These are used in getImplicitConvertUDFMethod
+    registerUDF(Boolean.class.getName(), UDFToBoolean.class, OperatorType.PREFIX, false,
+                UDFToBoolean.class.getSimpleName());
+    registerUDF(Byte.class.getName(), UDFToByte.class, OperatorType.PREFIX, false,
+                UDFToByte.class.getSimpleName());
+    registerUDF(Integer.class.getName(), UDFToInteger.class, OperatorType.PREFIX, false,
+                UDFToInteger.class.getSimpleName());
+    registerUDF(Long.class.getName(), UDFToLong.class, OperatorType.PREFIX, false,
+                UDFToLong.class.getSimpleName());
+    registerUDF(Float.class.getName(), UDFToFloat.class, OperatorType.PREFIX, false,
+                UDFToFloat.class.getSimpleName());
+    registerUDF(Double.class.getName(), UDFToDouble.class, OperatorType.PREFIX, false,
+                UDFToDouble.class.getSimpleName());
+    registerUDF(String.class.getName(), UDFToString.class, OperatorType.PREFIX, false,
+                UDFToString.class.getSimpleName());
+    registerUDF(java.sql.Date.class.getName(), UDFToDate.class, OperatorType.PREFIX, false,
+                UDFToDate.class.getSimpleName());
+
+    // Aggregate functions
+    registerUDAF("sum", UDAFSum.class);
+    registerUDAF("count", UDAFCount.class);
+    registerUDAF("max", UDAFMax.class);
+    registerUDAF("min", UDAFMin.class);
+    registerUDAF("avg", UDAFAvg.class);    
+  }
+
+  public static FunctionInfo getInfo(Class<?> fClass) {
+    for(Map.Entry<String, FunctionInfo> ent: mFunctions.entrySet()) {
+      FunctionInfo val = ent.getValue();
+      if (val.getUDFClass() == fClass ||
+          val.getUDAFClass() == fClass) {
+        return val;
+      }
+    }
+    
+    return null;
+  }
+  
+  public static void registerUDF(String functionName, Class<? extends UDF> UDFClass,
+                                 FunctionInfo.OperatorType opt, boolean isOperator) {
+    if (UDF.class.isAssignableFrom(UDFClass)) { 
+      FunctionInfo fI = new FunctionInfo(functionName.toLowerCase(), UDFClass, null);
+      fI.setIsOperator(isOperator);
+      fI.setOpType(opt);
+      mFunctions.put(functionName.toLowerCase(), fI);
+    } else {
+      throw new RuntimeException("Registering UDF Class " + UDFClass + " which does not extends " + UDF.class);
+    }
+  }
+  
+  public static void registerUDF(String functionName, Class<? extends UDF> UDFClass,
+                                 FunctionInfo.OperatorType opt, boolean isOperator,
+                                 String displayName) {
+    if (UDF.class.isAssignableFrom(UDFClass)) { 
+      FunctionInfo fI = new FunctionInfo(displayName, UDFClass, null);
+      fI.setIsOperator(isOperator);
+      fI.setOpType(opt);
+      mFunctions.put(functionName.toLowerCase(), fI);
+    } else {
+      throw new RuntimeException("Registering UDF Class " + UDFClass + " which does not extends " + UDF.class);
+    }
+  }
+
+  public static Class<? extends UDF> getUDFClass(String functionName) {
+    LOG.debug("Looking up: " + functionName);
+    FunctionInfo finfo = mFunctions.get(functionName.toLowerCase());
+    if (finfo == null) {
+      return null;
+    }
+    Class<? extends UDF> result = finfo.getUDFClass();
+    return result;
+  }
+
+  static Map<Class<?>, Integer> numericTypes;
+  static {
+    numericTypes = new HashMap<Class<?>, Integer>();
+    numericTypes.put(Byte.class, 1);
+    numericTypes.put(Integer.class, 2);
+    numericTypes.put(Long.class, 3);
+    numericTypes.put(Float.class, 4);
+    numericTypes.put(Double.class, 5);
+    numericTypes.put(String.class, 6);
+  } 
+
+  /**
+   * Find a common class that objects of both Class a and Class b can convert to.
+   * @return null if no common class could be found.
+   */
+  public static Class<?> getCommonClass(Class<?> a, Class<?> b) {
+    // Equal
+    if (a.equals(b)) return a;
+    // Java class inheritance hierarchy
+    if (a.isAssignableFrom(b)) return a;
+    if (b.isAssignableFrom(a)) return b;
+    // Prefer String to Number conversion before implicit conversions
+    if (Number.class.isAssignableFrom(a) && b.equals(String.class)) return Double.class;
+    if (Number.class.isAssignableFrom(b) && a.equals(String.class)) return Double.class;
+    // implicit conversions
+    if (FunctionRegistry.implicitConvertable(a, b)) return b;
+    if (FunctionRegistry.implicitConvertable(b, a)) return a;
+    return null;
+  }
+
+  /** Returns whether it is possible to implicitly convert an object of Class from to Class to.
+   */
+  public static boolean implicitConvertable(Class<?> from, Class<?> to) {
+    assert(!from.equals(to));
+    // Allow implicit String to Double conversion
+    if (from.equals(String.class) && to.equals(Double.class)) {
+      return true;
+    }
+    if (from.equals(String.class) && to.equals(java.sql.Date.class)) {
+      return true;
+    }
+    if (from.equals(java.sql.Date.class) && to.equals(String.class)) {
+      return true;
+    }
+    // Allow implicit conversion from Byte -> Integer -> Long -> Float -> Double -> String
+    Integer f = numericTypes.get(from);
+    Integer t = numericTypes.get(to);
+    if (f == null || t == null) return false;
+    if (f.intValue() > t.intValue()) return false;
+    return true;
+  }
+
+  /**
+   * Get the UDF method for the name and argumentClasses. 
+   * @param name the name of the UDF
+   * @param argumentClasses 
+   * @param exact  if true, we don't allow implicit type conversions. 
+   * @return
+   */
+  public static Method getUDFMethod(String name, boolean exact, List<Class<?>> argumentClasses) {
+    Class<? extends UDF> udf = getUDFClass(name);
+    if (udf == null) return null;
+    return getMethodInternal(udf, "evaluate", exact, argumentClasses);    
+  }
+
+  /**
+   * This method is shared between UDFRegistry and UDAFRegistry.
+   * methodName will be "evaluate" for UDFRegistry, and "aggregate" for UDAFRegistry. 
+   */
+  public static <T> Method getMethodInternal(Class<? extends T> udfClass, String methodName, boolean exact, List<Class<?>> argumentClasses) {
+    int leastImplicitConversions = Integer.MAX_VALUE;
+    Method udfMethod = null;
+
+    for(Method m: Arrays.asList(udfClass.getMethods())) {
+      if (m.getName().equals(methodName)) {
+
+        Class<?>[] argumentTypeInfos = m.getParameterTypes();
+
+        boolean match = (argumentTypeInfos.length == argumentClasses.size());
+        int implicitConversions = 0;
+
+        for(int i=0; i<argumentClasses.size() && match; i++) {
+          if (argumentClasses.get(i) == Void.class) continue;
+          Class<?> accepted = TypeInfo.generalizePrimitive(argumentTypeInfos[i]);
+          if (accepted.isAssignableFrom(argumentClasses.get(i))) {
+            // do nothing if match
+          } else if (!exact && implicitConvertable(argumentClasses.get(i), accepted)) {
+            implicitConversions ++;
+          } else {
+            match = false;
+          }
+        }
+
+        if (match) {
+          // Always choose the function with least implicit conversions.
+          if (implicitConversions < leastImplicitConversions) {
+            udfMethod = m;
+            leastImplicitConversions = implicitConversions;
+            // Found an exact match
+            if (leastImplicitConversions == 0) break;
+          } else if (implicitConversions == leastImplicitConversions){
+            // Ambiguous call: two methods with the same number of implicit conversions 
+            udfMethod = null;
+          } else {
+            // do nothing if implicitConversions > leastImplicitConversions
+          }
+        }
+      }
+    }
+    return udfMethod;
+  }
+
+  public static Method getUDFMethod(String name, boolean exact, Class<?> ... argumentClasses) {
+    return getUDFMethod(name, exact, Arrays.asList(argumentClasses));
+  }
+
+  public static void registerUDAF(String functionName, Class<? extends UDAF> UDAFClass) {
+
+    if (UDAF.class.isAssignableFrom(UDAFClass)) {
+      mFunctions.put(functionName.toLowerCase(), new FunctionInfo(functionName
+                                                                  .toLowerCase(), null, UDAFClass));
+    } else {
+      throw new RuntimeException("Registering UDAF Class " + UDAFClass
+                                 + " which does not extends " + UDAF.class);
+    }
+    mFunctions.put(functionName.toLowerCase(), new FunctionInfo(functionName
+                                                                .toLowerCase(), null, UDAFClass));
+  }
+
+  public static Class<? extends UDAF> getUDAF(String functionName) {
+    LOG.debug("Looking up UDAF: " + functionName);
+    FunctionInfo finfo = mFunctions.get(functionName.toLowerCase());
+    if (finfo == null) {
+      return null;
+    }
+    Class<? extends UDAF> result = finfo.getUDAFClass();
+    return result;
+  }
+
+  public static Method getUDAFMethod(String name, List<Class<?>> argumentClasses) {
+    Class<? extends UDAF> udaf = getUDAF(name);
+    if (udaf == null)
+      return null;
+    return FunctionRegistry.getMethodInternal(udaf, "aggregate", false,
+                                         argumentClasses);
+  }
+
+  public static Method getUDAFMethod(String name, Class<?>... argumentClasses) {
+    return getUDAFMethod(name, Arrays.asList(argumentClasses));
+  }
+}

+ 9 - 7
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
+import org.apache.hadoop.hive.ql.plan.explain;
 
 /**
  * Base operator implementation
@@ -38,6 +38,7 @@ public abstract class Operator <T extends Serializable> implements Serializable
   // Bean methods
 
   private static final long serialVersionUID = 1L;
+  
   protected List<Operator<? extends Serializable>> childOperators;
 
   public Operator() {}
@@ -57,6 +58,7 @@ public abstract class Operator <T extends Serializable> implements Serializable
     this.conf = conf;
   }
 
+  @explain
   public T getConf() {
     return conf;
   }
@@ -96,7 +98,7 @@ public abstract class Operator <T extends Serializable> implements Serializable
     if(childOperators == null)
       return;
 
-    for(Operator op: childOperators) {
+    for(Operator<? extends Serializable> op: childOperators) {
       op.setOutputCollector(out);
     }
   }
@@ -148,9 +150,9 @@ public abstract class Operator <T extends Serializable> implements Serializable
 
 
 
-  public Map<Enum, Long> getStats() {
-    HashMap<Enum, Long> ret = new HashMap<Enum, Long> ();
-    for(Enum one: statsMap.keySet()) {
+  public Map<Enum<?>, Long> getStats() {
+    HashMap<Enum<?>, Long> ret = new HashMap<Enum<?>, Long> ();
+    for(Enum<?> one: statsMap.keySet()) {
       ret.put(one, Long.valueOf(statsMap.get(one).get()));
     }
     return(ret);
@@ -226,7 +228,7 @@ public abstract class Operator <T extends Serializable> implements Serializable
   }
 
   public void resetStats() {
-    for(Enum e: statsMap.keySet()) {
+    for(Enum<?> e: statsMap.keySet()) {
       statsMap.get(e).set(0L);
     }
   }
@@ -246,7 +248,7 @@ public abstract class Operator <T extends Serializable> implements Serializable
   }
 
   public void logStats () {
-    for(Enum e: statsMap.keySet()) {
+    for(Enum<?> e: statsMap.keySet()) {
       l4j.info(e.toString() + ":" + statsMap.get(e).toString());
     }    
   }

+ 16 - 3
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java

@@ -41,22 +41,32 @@ public class TaskFactory {
 
   public static ArrayList<taskTuple<? extends Serializable>> taskvec;
   static {
+    id = 0;
     taskvec = new ArrayList<taskTuple<? extends Serializable>>();
     taskvec.add(new taskTuple<moveWork>(moveWork.class, MoveTask.class));
     taskvec.add(new taskTuple<copyWork>(copyWork.class, CopyTask.class));
     taskvec.add(new taskTuple<DDLWork>(DDLWork.class, DDLTask.class));
+    taskvec.add(new taskTuple<explainWork>(explainWork.class, ExplainTask.class));
     // we are taking this out to allow us to instantiate either MapRedTask or
     // ExecDriver dynamically at run time based on configuration
     // taskvec.add(new taskTuple<mapredWork>(mapredWork.class, ExecDriver.class));
   }
 
+  private static int id;
+  
+  public static void resetId() {
+    id = 0;
+  }
+  
   @SuppressWarnings("unchecked")
   public static <T extends Serializable> Task<T> get(Class<T> workClass, HiveConf conf) {
       
     for(taskTuple<? extends Serializable> t: taskvec) {
       if(t.workClass == workClass) {
         try {
-          return (Task<T>)t.taskClass.newInstance();
+          Task<T> ret = (Task<T>)t.taskClass.newInstance();
+          ret.setId("Stage-" + Integer.toString(id++));
+          return ret;
         } catch (Exception e) {
           throw new RuntimeException(e);
         }
@@ -71,12 +81,15 @@ public class TaskFactory {
 
         // in local mode - or if otherwise so configured - always submit
         // jobs via separate jvm
+        Task<T> ret = null;
         if(conf.getVar(HiveConf.ConfVars.HADOOPJT).equals("local") ||
            viachild.equals("true")) {
-          return (Task<T>)MapRedTask.class.newInstance();
+          ret = (Task<T>)MapRedTask.class.newInstance();
         } else {
-          return (Task<T>)ExecDriver.class.newInstance();
+          ret = (Task<T>)ExecDriver.class.newInstance();
         }
+        ret.setId("Stage-" + Integer.toString(id++));
+        return ret;
       } catch (Exception e) {
         throw new RuntimeException (e.getMessage(), e);
       }

+ 0 - 74
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/UDAFRegistry.java

@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec;
-
-import java.lang.reflect.Method;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.parse.TypeInfo;
-import org.apache.hadoop.hive.ql.udf.*;
-
-public class UDAFRegistry {
-
-  private static Log LOG = LogFactory.getLog("org.apache.hadoop.hive.ql.exec.UDAFRegistry");
-
-  /**
-   * The mapping from aggregation function names to aggregation classes.
-   */
-  static HashMap<String, Class<? extends UDAF>> mUDAFs;
-  static {
-    mUDAFs = new HashMap<String, Class<? extends UDAF>>();
-    registerUDAF("sum", UDAFSum.class);
-    registerUDAF("count", UDAFCount.class);
-    registerUDAF("max", UDAFMax.class);
-    registerUDAF("min", UDAFMin.class);
-    registerUDAF("avg", UDAFAvg.class);
-  }
-
-  public static void registerUDAF(String functionName, Class<? extends UDAF> UDAFClass) {
-    if (UDAF.class.isAssignableFrom(UDAFClass)) { 
-      mUDAFs.put(functionName.toLowerCase(), UDAFClass);
-    } else {
-      throw new RuntimeException("Registering UDAF Class " + UDAFClass + " which does not extends " + UDAF.class);
-    }
-    mUDAFs.put(functionName.toLowerCase(), UDAFClass);
-  }
-  
-  public static Class<? extends UDAF> getUDAF(String functionName) {
-    LOG.debug("Looking up UDAF: " + functionName);
-    Class<? extends UDAF> result = mUDAFs.get(functionName.toLowerCase());
-    return result;
-  }
-
-  public static Method getUDAFMethod(String name, List<Class<?>> argumentClasses) {
-    Class<? extends UDAF> udaf = getUDAF(name);
-    if (udaf == null) return null;
-    return UDFRegistry.getMethodInternal(udaf, "aggregate", false, argumentClasses);
-  }
-
-  public static Method getUDAFMethod(String name, Class<?> ... argumentClasses) {
-    return getUDAFMethod(name, Arrays.asList(argumentClasses));
-  }
-  
-  
-}

+ 0 - 238
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/UDFRegistry.java

@@ -1,238 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.lang.reflect.Method;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.lang.Void;
-
-import org.apache.hadoop.hive.ql.parse.TypeInfo;
-import org.apache.hadoop.hive.ql.udf.*;
-
-public class UDFRegistry {
-
-  private static Log LOG = LogFactory.getLog("org.apache.hadoop.hive.ql.exec.UDFRegistry");
-
-  /**
-   * The mapping from expression function names to expression classes.
-   */
-  static HashMap<String, Class<? extends UDF>> mUDFs;
-  static {
-    mUDFs = new HashMap<String, Class<? extends UDF>>();
-    registerUDF("default_sample_hashfn", UDFDefaultSampleHashFn.class);
-    registerUDF("concat", UDFConcat.class);
-    registerUDF("substr", UDFSubstr.class);
-    registerUDF("str_eq", UDFStrEq.class);
-    registerUDF("str_ne", UDFStrNe.class);
-    registerUDF("str_gt", UDFStrGt.class);
-    registerUDF("str_lt", UDFStrLt.class);
-    registerUDF("str_ge", UDFStrGe.class);
-    registerUDF("str_le", UDFStrLe.class);
-
-    registerUDF("upper", UDFUpper.class);
-    registerUDF("lower", UDFLower.class);
-    registerUDF("ucase", UDFUpper.class);
-    registerUDF("lcase", UDFLower.class);
-    registerUDF("trim", UDFTrim.class);
-    registerUDF("ltrim", UDFLTrim.class);
-    registerUDF("rtrim", UDFRTrim.class);
-    
-    registerUDF("like", UDFLike.class);
-    registerUDF("rlike", UDFRegExp.class);
-    registerUDF("regexp", UDFRegExp.class);
-    registerUDF("regexp_replace", UDFRegExpReplace.class);
-    
-    registerUDF("+", UDFOPPlus.class);
-    registerUDF("-", UDFOPMinus.class);
-    registerUDF("*", UDFOPMultiply.class);
-    registerUDF("/", UDFOPDivide.class);
-    registerUDF("%", UDFOPMod.class);
-    
-    registerUDF("&", UDFOPBitAnd.class);
-    registerUDF("|", UDFOPBitOr.class);
-    registerUDF("^", UDFOPBitXor.class);
-    registerUDF("~", UDFOPBitNot.class);
-
-    registerUDF("=", UDFOPEqual.class);
-    registerUDF("==", UDFOPEqual.class);
-    registerUDF("<>", UDFOPNotEqual.class);
-    registerUDF("<", UDFOPLessThan.class);
-    registerUDF("<=", UDFOPEqualOrLessThan.class);
-    registerUDF(">", UDFOPGreaterThan.class);
-    registerUDF(">=", UDFOPEqualOrGreaterThan.class);
-
-    registerUDF("and", UDFOPAnd.class);
-    registerUDF("&&", UDFOPAnd.class);
-    registerUDF("or", UDFOPOr.class);
-    registerUDF("||", UDFOPOr.class);
-    registerUDF("not", UDFOPNot.class);
-    registerUDF("!", UDFOPNot.class);
-
-    registerUDF("isnull", UDFOPNull.class);
-    registerUDF("isnotnull", UDFOPNotNull.class);
-    
-    // Aliases for Java Class Names
-    // These are used in getImplicitConvertUDFMethod
-    registerUDF(Boolean.class.getName(), UDFToBoolean.class);
-    registerUDF(Byte.class.getName(), UDFToByte.class);
-    registerUDF(Integer.class.getName(), UDFToInteger.class);
-    registerUDF(Long.class.getName(), UDFToLong.class);
-    registerUDF(Float.class.getName(), UDFToFloat.class);
-    registerUDF(Double.class.getName(), UDFToDouble.class);
-    registerUDF(String.class.getName(), UDFToString.class);
-    registerUDF(java.sql.Date.class.getName(), UDFToDate.class);
-  }
-
-  public static void registerUDF(String functionName, Class<? extends UDF> UDFClass) {
-    if (UDF.class.isAssignableFrom(UDFClass)) { 
-      mUDFs.put(functionName.toLowerCase(), UDFClass);
-    } else {
-      throw new RuntimeException("Registering UDF Class " + UDFClass + " which does not extends " + UDF.class);
-    }
-  }
-  
-  public static Class<? extends UDF> getUDFClass(String functionName) {
-    LOG.debug("Looking up: " + functionName);
-    Class<? extends UDF> result = mUDFs.get(functionName.toLowerCase());
-    return result;
-  }
-
-  static Map<Class<?>, Integer> numericTypes;
-  static {
-    numericTypes = new HashMap<Class<?>, Integer>();
-    numericTypes.put(Byte.class, 1);
-    numericTypes.put(Integer.class, 2);
-    numericTypes.put(Long.class, 3);
-    numericTypes.put(Float.class, 4);
-    numericTypes.put(Double.class, 5);
-    numericTypes.put(String.class, 6);
-  } 
-  
-  /**
-   * Find a common class that objects of both Class a and Class b can convert to.
-   * @return null if no common class could be found.
-   */
-  public static Class<?> getCommonClass(Class<?> a, Class<?> b) {
-    // Equal
-    if (a.equals(b)) return a;
-    // Java class inheritance hierarchy
-    if (a.isAssignableFrom(b)) return a;
-    if (b.isAssignableFrom(a)) return b;
-    // Prefer String to Number conversion before implicit conversions
-    if (Number.class.isAssignableFrom(a) && b.equals(String.class)) return Double.class;
-    if (Number.class.isAssignableFrom(b) && a.equals(String.class)) return Double.class;
-    // implicit conversions
-    if (UDFRegistry.implicitConvertable(a, b)) return b;
-    if (UDFRegistry.implicitConvertable(b, a)) return a;
-    return null;
-  }
-  
-  /** Returns whether it is possible to implicitly convert an object of Class from to Class to.
-   */
-  public static boolean implicitConvertable(Class<?> from, Class<?> to) {
-    assert(!from.equals(to));
-    // Allow implicit String to Double conversion
-    if (from.equals(String.class) && to.equals(Double.class)) {
-      return true;
-    }
-    if (from.equals(String.class) && to.equals(java.sql.Date.class)) {
-      return true;
-    }
-    if (from.equals(java.sql.Date.class) && to.equals(String.class)) {
-      return true;
-    }
-    // Allow implicit conversion from Byte -> Integer -> Long -> Float -> Double -> String
-    Integer f = numericTypes.get(from);
-    Integer t = numericTypes.get(to);
-    if (f == null || t == null) return false;
-    if (f.intValue() > t.intValue()) return false;
-    return true;
-  }
-
-  /**
-   * Get the UDF method for the name and argumentClasses. 
-   * @param name the name of the UDF
-   * @param argumentClasses 
-   * @param exact  if true, we don't allow implicit type conversions. 
-   * @return
-   */
-  public static Method getUDFMethod(String name, boolean exact, List<Class<?>> argumentClasses) {
-    Class<? extends UDF> udf = getUDFClass(name);
-    if (udf == null) return null;
-    return getMethodInternal(udf, "evaluate", exact, argumentClasses);    
-  }
-  
-  /**
-   * This method is shared between UDFRegistry and UDAFRegistry.
-   * methodName will be "evaluate" for UDFRegistry, and "aggregate" for UDAFRegistry. 
-   */
-  public static <T> Method getMethodInternal(Class<? extends T> udfClass, String methodName, boolean exact, List<Class<?>> argumentClasses) {
-    int leastImplicitConversions = Integer.MAX_VALUE;
-    Method udfMethod = null;
-    
-    for(Method m: Arrays.asList(udfClass.getMethods())) {
-      if (m.getName().equals(methodName)) {
-        
-        Class<?>[] argumentTypeInfos = m.getParameterTypes();
-        
-        boolean match = (argumentTypeInfos.length == argumentClasses.size());
-        int implicitConversions = 0;
-        
-        for(int i=0; i<argumentClasses.size() && match; i++) {
-          if (argumentClasses.get(i) == Void.class) continue;
-          Class<?> accepted = TypeInfo.generalizePrimitive(argumentTypeInfos[i]);
-          if (accepted.isAssignableFrom(argumentClasses.get(i))) {
-            // do nothing if match
-          } else if (!exact && implicitConvertable(argumentClasses.get(i), accepted)) {
-            implicitConversions ++;
-          } else {
-            match = false;
-          }
-        }
-
-        if (match) {
-          // Always choose the function with least implicit conversions.
-          if (implicitConversions < leastImplicitConversions) {
-            udfMethod = m;
-            leastImplicitConversions = implicitConversions;
-            // Found an exact match
-            if (leastImplicitConversions == 0) break;
-          } else if (implicitConversions == leastImplicitConversions){
-            // Ambiguous call: two methods with the same number of implicit conversions 
-            udfMethod = null;
-          } else {
-            // do nothing if implicitConversions > leastImplicitConversions
-          }
-        }
-      }
-    }
-    return udfMethod;
-  }
-
-  public static Method getUDFMethod(String name, boolean exact, Class<?> ... argumentClasses) {
-    return getUDFMethod(name, exact, Arrays.asList(argumentClasses));
-  }
-  
-}

+ 14 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe;
 
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.plan.*;
 import org.apache.hadoop.hive.ql.plan.PlanUtils.ExpressionTypes;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -90,6 +91,19 @@ public class Utilities {
     }
   }
 
+  public static List<String> getFieldSchemaString(List<FieldSchema> fl) {
+    if (fl == null) {
+      return null;
+    }
+    
+    ArrayList<String> ret = new ArrayList<String>();
+    for(FieldSchema f: fl) {
+      ret.add(f.getName() + " " + f.getType() + 
+              (f.getComment() != null ? (" " + f.getComment()) : ""));
+    }
+    return ret;
+  }
+  
   /**
    * Java 1.5 workaround. 
    * From http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=5015403

+ 3 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.metadata.*;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 
 public abstract class BaseSemanticAnalyzer {
@@ -44,6 +45,7 @@ public abstract class BaseSemanticAnalyzer {
   protected final LogHelper console;
 
   protected Context ctx;
+  
   public BaseSemanticAnalyzer(HiveConf conf) throws SemanticException {
     try {
       this.conf = conf;
@@ -62,7 +64,7 @@ public abstract class BaseSemanticAnalyzer {
   }
 
   public abstract void analyze(CommonTree ast, Context ctx) throws SemanticException;
-
+  
   public List<Task<? extends Serializable>> getRootTasks() {
     return rootTasks;
   }

+ 1 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java

@@ -65,6 +65,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     super(conf);
   }
 
+  @Override
   public void analyze(CommonTree ast, Context ctx) throws SemanticException {
     this.ctx = ctx;
     if (ast.getToken().getType() == HiveParser.TOK_CREATETABLE)

+ 54 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java

@@ -0,0 +1,54 @@
+/**
+ * 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.parse;
+
+import java.io.File;
+
+import org.antlr.runtime.tree.CommonTree;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.plan.explainWork;
+
+public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
+
+  
+  public ExplainSemanticAnalyzer(HiveConf conf) throws SemanticException {
+    super(conf);
+  }
+
+  public void analyze(CommonTree ast, Context ctx) throws SemanticException {
+    
+    // Create a semantic analyzer for the query
+    BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, (CommonTree)ast.getChild(0));
+    sem.analyze((CommonTree)ast.getChild(0), ctx);
+    
+    boolean extended = false;
+    if (ast.getChildCount() > 1) {
+      extended = true;
+    }
+    
+    ctx.setResFile(new File(getTmpFileName()));
+    
+    rootTasks.add(TaskFactory.get(new explainWork(ctx.getResFile(),
+                                                  sem.getRootTasks(),
+                                                  ((CommonTree)ast.getChild(0)).toStringTree(),
+                                                  extended), this.conf));
+  }
+}

+ 0 - 60
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionInfo.java

@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.parse;
-
-import java.lang.Class;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * The type information returned by the TypeRegistry
- *
- **/
-
-public class FunctionInfo {
-  private TypeInfo returnType;
-  private String UDFName;
-  private boolean hasUDF;
-
-  @SuppressWarnings("unused")
-  private static final Log LOG = LogFactory.getLog(FunctionInfo.class.getName());
-  
-  public FunctionInfo(TypeInfo returnType, String UDFName) {
-    this.returnType = returnType;
-    this.UDFName = UDFName;
-    this.hasUDF = (this.UDFName != null);
-  }
-
-  public FunctionInfo(Class<?> returnClass, String UDFName) {
-    this(new TypeInfo(returnClass), UDFName);
-  }
-
-  public TypeInfo getReturnType() {
-    return returnType;
-  }
-
-  public String getUDFName() {
-    return UDFName;
-  }
-
-  public boolean hasUDF() {
-    return hasUDF;
-  }
-}

+ 15 - 3
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g

@@ -97,6 +97,7 @@ TOK_TABCOLNAME;
 TOK_TABLELOCATION;
 TOK_TABLESAMPLE;
 TOK_TMP_FILE;
+TOK_EXPLAIN;
 }
 
 
@@ -115,9 +116,18 @@ catch (RecognitionException e) {
  
 // starting rule
 statement
-    : queryStatementExpression EOF
-    | loadStatement EOF
-    | ddlStatement EOF
+	: explainStatement EOF
+	| execStatement EOF
+	;
+
+explainStatement
+	: KW_EXPLAIN (isExtended=KW_EXTENDED)? execStatement -> ^(TOK_EXPLAIN execStatement $isExtended?)
+	;
+		
+execStatement
+    : queryStatementExpression
+    | loadStatement
+    | ddlStatement
     ;
 
 loadStatement
@@ -766,6 +776,8 @@ KW_ADD: 'ADD';
 KW_COLUMNS: 'COLUMNS';
 KW_RLIKE: 'RLIKE';
 KW_REGEXP: 'REGEXP';
+KW_EXPLAIN: 'EXPLAIN';
+KW_EXTENDED: 'EXTENDED';
 
 // Operators
 

+ 3 - 3
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.parse;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
@@ -28,8 +29,6 @@ import java.util.List;
 import org.antlr.runtime.tree.CommonTree;
 import org.antlr.runtime.tree.Tree;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -156,6 +155,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
+  @Override
   public void analyze(CommonTree ast, Context ctx) throws SemanticException {
     isLocal = isOverWrite = false;
     Tree from_t = ast.getChild(0);
@@ -191,7 +191,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     // make sure the arguments make sense
     applyConstraints(fromURI, toURI, from_t, isLocal);
 
-    Task rTask = null;
+    Task<? extends Serializable> rTask = null;
 
     // create copy work
     if(isLocal) {

+ 0 - 3
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/PartitionPruner.java

@@ -29,14 +29,11 @@ package org.apache.hadoop.hive.ql.parse;
 import java.util.*;
 import org.antlr.runtime.tree.*;
 
-import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
 import org.apache.hadoop.hive.ql.exec.HiveObject;
 import org.apache.hadoop.hive.ql.exec.LabeledCompositeHiveObject;
 import org.apache.hadoop.hive.ql.exec.PrimitiveHiveObject;
-import org.apache.hadoop.hive.ql.exec.UDF;
-import org.apache.hadoop.hive.ql.exec.UDFRegistry;
 import org.apache.hadoop.hive.ql.metadata.*;
 import org.apache.hadoop.hive.ql.plan.exprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.exprNodeConstantDesc;

+ 2 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java

@@ -143,12 +143,12 @@ public class RowResolver {
   public String toString() {
     StringBuffer sb = new StringBuffer();
     
-    for(Map.Entry e: rslvMap.entrySet()) {
+    for(Map.Entry<String, LinkedHashMap<String,ColumnInfo>> e: rslvMap.entrySet()) {
       String tab = (String)e.getKey();
       sb.append(tab + "{");
       HashMap<String, ColumnInfo> f_map = (HashMap<String, ColumnInfo>)e.getValue();
       if (f_map != null)
-        for(Map.Entry entry: f_map.entrySet()) {
+        for(Map.Entry<String, ColumnInfo> entry: f_map.entrySet()) {
           sb.append("(" + (String)entry.getKey() + "," + entry.getValue().toString() + ")");
         }
       sb.append("} ");

+ 15 - 16
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java

@@ -134,7 +134,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       assert (expressionTree.getChildCount() != 0);
       assert (expressionTree.getChild(0).getType() == HiveParser.Identifier);
       String functionName = expressionTree.getChild(0).getText();
-      if (UDAFRegistry.getUDAF(functionName) != null) {
+      if (FunctionRegistry.getUDAF(functionName) != null) {
         aggregations.put(expressionTree.toStringTree(), expressionTree);
         return;
       }
@@ -987,7 +987,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     for (Map.Entry<String, CommonTree> entry : aggregationTrees.entrySet()) {
       CommonTree value = entry.getValue();
       String aggName = value.getChild(0).getText();
-      Class<? extends UDAF> aggClass = UDAFRegistry.getUDAF(aggName);
+      Class<? extends UDAF> aggClass = FunctionRegistry.getUDAF(aggName);
       assert (aggClass != null);
       ArrayList<exprNodeDesc> aggParameters = new ArrayList<exprNodeDesc>();
       ArrayList<Class<?>> aggClasses = new ArrayList<Class<?>>();
@@ -1006,7 +1006,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         aggClasses.add(paraExprInfo.getType().getPrimitiveClass());
       }
 
-      if (null == UDAFRegistry.getUDAFMethod(aggName, aggClasses)) {
+      if (null == FunctionRegistry.getUDAFMethod(aggName, aggClasses)) {
         String reason = "Looking for UDAF \"" + aggName + "\" with parameters " + aggClasses;
         throw new SemanticException(ErrorMsg.INVALID_FUNCTION_SIGNATURE.getMsg((CommonTree)value.getChild(0), reason));
       }
@@ -1059,7 +1059,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     for (Map.Entry<String, CommonTree> entry : aggregationTrees.entrySet()) {
       CommonTree value = entry.getValue();
       String aggName = value.getChild(0).getText();
-      Class<? extends UDAF> aggClass = UDAFRegistry.getUDAF(aggName);
+      Class<? extends UDAF> aggClass = FunctionRegistry.getUDAF(aggName);
       assert (aggClass != null);
       ArrayList<exprNodeDesc> aggParameters = new ArrayList<exprNodeDesc>();
       ArrayList<Class<?>> aggClasses = new ArrayList<Class<?>>();
@@ -1078,7 +1078,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         aggClasses.add(paraExprInfo.getType().getPrimitiveClass());
       }
 
-      if (null == UDAFRegistry.getUDAFMethod(aggName, aggClasses)) {
+      if (null == FunctionRegistry.getUDAFMethod(aggName, aggClasses)) {
         String reason = "Looking for UDAF \"" + aggName + "\" with parameters " + aggClasses;
         throw new SemanticException(ErrorMsg.INVALID_FUNCTION_SIGNATURE.getMsg((CommonTree)value.getChild(0), reason));
       }
@@ -1288,8 +1288,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     int inputField = reduceKeys.size();
     HashMap<String, CommonTree> aggregationTrees = parseInfo
         .getAggregationExprsForClause(dest);
-    for (Map.Entry entry : aggregationTrees.entrySet()) {
-      String key = (String)entry.getKey();
+    for (Map.Entry<String, CommonTree> entry : aggregationTrees.entrySet()) {
       reduceValues.add(new exprNodeColumnDesc(TypeInfo.getPrimitiveTypeInfo(String.class),
           (Integer.valueOf(inputField)).toString()));
       inputField++;
@@ -1336,7 +1335,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     for (Map.Entry<String, CommonTree> entry : aggregationTrees.entrySet()) {
       CommonTree value = entry.getValue();
       String aggName = value.getChild(0).getText();
-      Class<? extends UDAF> aggClass = UDAFRegistry.getUDAF(aggName);
+      Class<? extends UDAF> aggClass = FunctionRegistry.getUDAF(aggName);
       assert (aggClass != null);
       ArrayList<exprNodeDesc> aggParameters = new ArrayList<exprNodeDesc>();
       String text = entry.getKey();
@@ -1535,7 +1534,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     // We have the table object here - go over the row resolver
     // and check all the types are the same
-    Vector<ColumnInfo> srcOpns = input.get(0).getRowResolver().getColumnInfos();
+    input.get(0).getRowResolver().getColumnInfos();
 
     Vector<ColumnInfo> insOpns = new Vector<ColumnInfo>();
     for (SerDeField field : dest_tab.getFields(null)) {
@@ -1836,7 +1835,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       for(int i=1; i<right.length; i++) {
         Class<?> a = commonClass;
         Class<?> b = keys.get(i).get(k).getTypeInfo().getPrimitiveClass(); 
-        commonClass = UDFRegistry.getCommonClass(a, b);
+        commonClass = FunctionRegistry.getCommonClass(a, b);
         if (commonClass == null) {
           throw new SemanticException("Cannot do equality join on different types: " + a.getClass() + " and " + b.getClass());
         }
@@ -2693,7 +2692,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       assert(childTypeInfo != null);
       argumentClasses.add(childTypeInfo.getPrimitiveClass());
     }
-    Method udfMethod = UDFRegistry.getUDFMethod(udfName, false, argumentClasses);
+    Method udfMethod = FunctionRegistry.getUDFMethod(udfName, false, argumentClasses);
     if (udfMethod == null) return null;
 
     ArrayList<exprNodeDesc> ch = new ArrayList<exprNodeDesc>();
@@ -2713,10 +2712,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         // must be implicit type conversion
         Class<?> from = argumentClasses.get(i);
         Class<?> to = pType;
-        assert(UDFRegistry.implicitConvertable(from, to));
-        Method m = UDFRegistry.getUDFMethod(to.getName(), true, from);
+        assert(FunctionRegistry.implicitConvertable(from, to));
+        Method m = FunctionRegistry.getUDFMethod(to.getName(), true, from);
         assert(m != null);
-        Class c = UDFRegistry.getUDFClass(to.getName());
+        Class<? extends UDF> c = FunctionRegistry.getUDFClass(to.getName());
         assert(c != null);
 
         // get the conversion method
@@ -2730,7 +2729,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     exprNodeFuncDesc desc = new exprNodeFuncDesc(
       TypeInfo.getPrimitiveTypeInfo(TypeInfo.generalizePrimitive(udfMethod.getReturnType())),
-      UDFRegistry.getUDFClass(udfName),
+      FunctionRegistry.getUDFClass(udfName),
       udfMethod, ch);
     return desc;
   }
@@ -2895,7 +2894,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       desc = new exprNodeIndexDesc(t, children.get(0), children.get(1));
     } else {
       // other operators or functions
-      Class<? extends UDF> udf = UDFRegistry.getUDFClass(funcText);
+      Class<? extends UDF> udf = FunctionRegistry.getUDFClass(funcText);
       if (udf == null) {
         throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg((CommonTree)expr.getChild(0)));
       }

+ 1 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java

@@ -29,6 +29,7 @@ public class SemanticAnalyzerFactory {
       throw new RuntimeException ("Empty Syntax Tree");
     } else {
       switch (tree.getToken().getType()) {
+      case HiveParser.TOK_EXPLAIN: return new ExplainSemanticAnalyzer(conf);
       case HiveParser.TOK_LOAD: return new LoadSemanticAnalyzer(conf);
       case HiveParser.TOK_CREATETABLE: 
       case HiveParser.TOK_CREATEEXTTABLE: 

+ 5 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeInfo.java

@@ -232,7 +232,7 @@ public class TypeInfo {
   
 
   public static Class<?> generalizePrimitive(Class<?> primitiveClass) {
-    if (primitiveClass == Boolean.TYPE)    primitiveClass = Boolean.class;
+    if (primitiveClass == Boolean.TYPE)   primitiveClass = Boolean.class;
     if (primitiveClass == Byte.TYPE)      primitiveClass = Byte.class; 
     if (primitiveClass == Character.TYPE) primitiveClass = Character.class; 
     if (primitiveClass == Short.TYPE)     primitiveClass = Short.class; 
@@ -244,4 +244,8 @@ public class TypeInfo {
     return primitiveClass;
   }
   
+  public String getTypeString() {
+    // TODO: Change this to print out a better user visible string
+    return toString();
+  }
 }

+ 0 - 316
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeRegistry.java

@@ -1,316 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.parse;
-
-import java.util.HashMap;
-import java.lang.Number;
-import java.lang.Boolean;
-import java.lang.String;
-import java.util.Date;
-import java.lang.Void;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * Class that stores the type mappings for all builtin functions, udfs, udafs
- * and builtin operators. The mappings are (name, list of paramerter types) -> return type
- *
- **/
-
-public class TypeRegistry {
-  private static final Log LOG = LogFactory.getLog("hive.ql.parse.TypeRegistry");
-
-  private static HashMap<InputSignature, FunctionInfo> typeMap;
-  static {
-    typeMap = new HashMap<InputSignature, FunctionInfo>();
-    typeMap.put(new InputSignature("+", Number.class, Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("+", Void.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("+", Void.class, Void.class), 
-        new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("+", String.class, Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("+", Void.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("+", Number.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("+", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("+", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("+", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-    typeMap.put(new InputSignature("-", Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", Number.class, Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", Void.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", Void.class, Void.class), 
-        new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", String.class, Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", Void.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", String.class, Void.class), 
-        new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", Number.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", Number.class, Void.class), 
-        new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("-", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-    typeMap.put(new InputSignature("/", Number.class, Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("/", Void.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("/", Void.class, Void.class), 
-        new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("/", String.class, Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("/", Void.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("/", Number.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("/", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("/", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("/", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-    typeMap.put(new InputSignature("*", Number.class, Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("*", Void.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("*", Void.class, Void.class), 
-        new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("*", String.class, Void.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("*", Void.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("*", Number.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("*", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("*", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("*", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-    typeMap.put(new InputSignature("%", Number.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("%", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("%", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("%", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-
-    typeMap.put(new InputSignature("&", Number.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("&", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("&", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("&", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-    typeMap.put(new InputSignature("~", Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("~", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("~", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("~", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-    typeMap.put(new InputSignature("|", Number.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("|", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("|", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("|", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-    typeMap.put(new InputSignature("^", Number.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("^", String.class, String.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("^", String.class, Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("^", Number.class, String.class), 
-                new FunctionInfo(Number.class, null));
-
-
-    typeMap.put(new InputSignature("=", Number.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("=", String.class, String.class), 
-                new FunctionInfo(Boolean.class, "str_eq"));
-    typeMap.put(new InputSignature("=", Number.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("=", String.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("=", Date.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("=", Date.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("=", String.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("=", Boolean.class, Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("=", Void.class, Void.class), 
-                new FunctionInfo(Boolean.class, null));
-
-    typeMap.put(new InputSignature("<>", Number.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<>", String.class, String.class), 
-                new FunctionInfo(Boolean.class, "str_ne"));
-    typeMap.put(new InputSignature("<>", Number.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<>", String.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<>", Date.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<>", Date.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<>", String.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<>", Boolean.class, Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-
-    typeMap.put(new InputSignature("<=", Number.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<=", String.class, String.class), 
-                new FunctionInfo(Boolean.class, "str_le"));
-    typeMap.put(new InputSignature("<=", Number.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<=", String.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<=", Date.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<=", Date.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<=", String.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<=", Boolean.class, Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-
-    typeMap.put(new InputSignature("<", Number.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<", String.class, String.class), 
-                new FunctionInfo(Boolean.class, "str_lt"));
-    typeMap.put(new InputSignature("<", Number.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<", String.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<", Date.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<", Date.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<", String.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("<", Boolean.class, Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-
-    typeMap.put(new InputSignature(">=", Number.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">=", String.class, String.class), 
-                new FunctionInfo(Boolean.class, "str_ge"));
-    typeMap.put(new InputSignature(">=", Number.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">=", String.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">=", Date.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">=", Date.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">=", String.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">=", Boolean.class, Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-
-    typeMap.put(new InputSignature(">", Number.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">", String.class, String.class), 
-                new FunctionInfo(Boolean.class, "str_gt"));
-    typeMap.put(new InputSignature(">", Number.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">", String.class, Number.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">", Date.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">", Date.class, String.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">", String.class, Date.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature(">", Boolean.class, Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-
-    typeMap.put(new InputSignature("AND", Boolean.class, Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("OR", Boolean.class, Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-    typeMap.put(new InputSignature("NOT", Boolean.class), 
-                new FunctionInfo(Boolean.class, null));
-
-    typeMap.put(new InputSignature("sum", Number.class), 
-                new FunctionInfo(Number.class, null));
-    typeMap.put(new InputSignature("count", Object.class), 
-                new FunctionInfo(Boolean.class, null));
-
-    typeMap.put(new InputSignature("concat", String.class, String.class), 
-                new FunctionInfo(String.class, null));
-    typeMap.put(new InputSignature("substr", String.class, Number.class), 
-                new FunctionInfo(String.class, null));
-    typeMap.put(new InputSignature("substr", String.class, Number.class, Number.class), 
-                new FunctionInfo(String.class, null));
-    typeMap.put(new InputSignature("NULL", Void.class), 
-                new FunctionInfo(Void.class, null));
-    typeMap.put(new InputSignature("NULL", String.class), 
-                new FunctionInfo(Void.class, null));
-  }
-
-  public static FunctionInfo getTypeInfo(InputSignature sgn) {
-    LOG.info("Looking up: " + sgn.toString());
-    return typeMap.get(sgn);
-  }
-
-  public static boolean isValidFunction(String functionName) {
-    for(InputSignature inp: typeMap.keySet()) {
-      if (inp.getName().equalsIgnoreCase(functionName)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-}

+ 7 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java

@@ -68,6 +68,7 @@ public class DDLWork implements Serializable {
   /**
    * @return the createTblDesc
    */
+  @explain(displayName="Create Table Operator")
   public createTableDesc getCreateTblDesc() {
     return createTblDesc;
   }
@@ -82,6 +83,7 @@ public class DDLWork implements Serializable {
   /**
    * @return the dropTblDesc
    */
+  @explain(displayName="Drop Table Operator")
   public dropTableDesc getDropTblDesc() {
     return dropTblDesc;
   }
@@ -96,6 +98,7 @@ public class DDLWork implements Serializable {
   /**
    * @return the alterTblDesc
    */
+  @explain(displayName="Alter Table Operator")
   public alterTableDesc getAlterTblDesc() {
     return alterTblDesc;
   }
@@ -110,6 +113,7 @@ public class DDLWork implements Serializable {
   /**
    * @return the showTblsDesc
    */
+  @explain(displayName="Show Table Operator")
   public showTablesDesc getShowTblsDesc() {
     return showTblsDesc;
   }
@@ -124,6 +128,7 @@ public class DDLWork implements Serializable {
   /**
    * @return the descTblDesc
    */
+  @explain(displayName="Describe Table Operator")
   public descTableDesc getDescTblDesc() {
     return descTblDesc;
   }
@@ -133,5 +138,6 @@ public class DDLWork implements Serializable {
    */
   public void setDescTblDesc(descTableDesc descTblDesc) {
     this.descTblDesc = descTblDesc;
-  }  
+  }
+  
 }

+ 25 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/aggregationDesc.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.UDAF;
 
 public class aggregationDesc implements java.io.Serializable {
@@ -52,4 +54,27 @@ public class aggregationDesc implements java.io.Serializable {
   public void setDistinct(final boolean distinct) {
     this.distinct = distinct;
   }
+  
+  @explain(displayName="expr")
+  public String getExprString() {
+    FunctionInfo fI = FunctionRegistry.getInfo(aggregationClass);
+    StringBuilder sb = new StringBuilder();
+    sb.append(fI.getDisplayName());
+    sb.append("(");
+    if (distinct) {
+      sb.append("DISTINCT ");
+    }
+    boolean first = true;
+    for(exprNodeDesc exp: parameters) {
+      if (!first) {
+        sb.append(", ");
+      }
+      
+      sb.append(exp.getExprString());
+      first = false;
+    }
+    sb.append(")");
+    
+    return sb.toString();
+  }
 }

+ 20 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/alterTableDesc.java

@@ -19,10 +19,13 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 
+@explain(displayName="Alter Table")
 public class alterTableDesc extends ddlDesc implements Serializable 
 {
   private static final long serialVersionUID = 1L;
@@ -56,6 +59,7 @@ public class alterTableDesc extends ddlDesc implements Serializable
   /**
    * @return the old name of the table
    */
+  @explain(displayName="old name")
   public String getOldName() {
     return oldName;
   }
@@ -70,6 +74,7 @@ public class alterTableDesc extends ddlDesc implements Serializable
   /**
    * @return the newName
    */
+  @explain(displayName="new name")
   public String getNewName() {
     return newName;
   }
@@ -88,6 +93,17 @@ public class alterTableDesc extends ddlDesc implements Serializable
     return op;
   }
 
+  @explain(displayName="type")
+  public String getAlterTableTypeString() {
+    switch(op) {
+    case RENAME:
+      return "rename";
+    case ADDCOLS:
+      return "add columns";
+    }
+    
+    return "unknown";
+  }
   /**
    * @param op the op to set
    */
@@ -102,6 +118,10 @@ public class alterTableDesc extends ddlDesc implements Serializable
     return newCols;
   }
 
+  @explain(displayName="new columns")
+  public List<String> getNewColsString() {
+    return Utilities.getFieldSchemaString(getNewCols());
+  }
   /**
    * @param newCols the newCols to set
    */

+ 1 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/collectDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Collect")
 public class collectDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   Integer bufferSize;

+ 5 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/copyWork.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
-
+@explain(displayName="Copy")
 public class copyWork implements Serializable {
   private static final long serialVersionUID = 1L;
   private String fromPath;
@@ -33,12 +33,16 @@ public class copyWork implements Serializable {
     this.fromPath = fromPath;
     this.toPath = toPath;
   }
+  
+  @explain(displayName="source")
   public String getFromPath() {
     return this.fromPath;
   }
   public void setFromPath(final String fromPath) {
     this.fromPath = fromPath;
   }
+  
+  @explain(displayName="destination")
   public String getToPath() {
     return this.toPath;
   }

+ 24 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/createTableDesc.java

@@ -22,7 +22,9 @@ import java.io.Serializable;
 import java.util.List;
 
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 
+@explain(displayName="Create Table")
 public class createTableDesc extends ddlDesc implements Serializable 
 {
   private static final long serialVersionUID = 1L;
@@ -65,6 +67,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.partCols        = partCols;
   }
 
+  @explain(displayName="name")
   public String getTableName() {
     return tableName;
   }
@@ -77,6 +80,11 @@ public class createTableDesc extends ddlDesc implements Serializable
     return cols;
   }
 
+  @explain(displayName="columns")
+  public List<String> getColsString() {
+    return Utilities.getFieldSchemaString(getCols());
+  }
+ 
   public void setCols(List<FieldSchema> cols) {
     this.cols = cols;
   }
@@ -85,10 +93,16 @@ public class createTableDesc extends ddlDesc implements Serializable
     return partCols;
   }
 
+  @explain(displayName="partition columns")
+  public List<String> getPartColsString() {
+    return Utilities.getFieldSchemaString(getPartCols());
+  }
+
   public void setPartCols(List<FieldSchema> partCols) {
     this.partCols = partCols;
   }
 
+  @explain(displayName="bucket columns")
   public List<String> getBucketCols() {
     return bucketCols;
   }
@@ -97,6 +111,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.bucketCols = bucketCols;
   }
 
+  @explain(displayName="# buckets")
   public int getNumBuckets() {
     return numBuckets;
   }
@@ -105,6 +120,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.numBuckets = numBuckets;
   }
 
+  @explain(displayName="field delimiter")
   public String getFieldDelim() {
     return fieldDelim;
   }
@@ -113,6 +129,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.fieldDelim = fieldDelim;
   }
 
+  @explain(displayName="collection delimiter")
   public String getCollItemDelim() {
     return collItemDelim;
   }
@@ -121,6 +138,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.collItemDelim = collItemDelim;
   }
 
+  @explain(displayName="map key delimiter")
   public String getMapKeyDelim() {
     return mapKeyDelim;
   }
@@ -129,6 +147,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.mapKeyDelim = mapKeyDelim;
   }
 
+  @explain(displayName="line delimiter")
   public String getLineDelim() {
     return lineDelim;
   }
@@ -137,6 +156,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.lineDelim = lineDelim;
   }
 
+  @explain(displayName="comment")
   public String getComment() {
     return comment;
   }
@@ -145,6 +165,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.comment = comment;
   }
 
+  @explain(displayName="isCompressed")
   public boolean isCompressed() {
     return isCompressed;
   }
@@ -153,6 +174,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.isCompressed = isCompressed;
   }
 
+  @explain(displayName="location")
   public String getLocation() {
     return location;
   }
@@ -161,6 +183,7 @@ public class createTableDesc extends ddlDesc implements Serializable
     this.location = location;
   }
 
+  @explain(displayName="isExternal")
   public boolean isExternal() {
     return isExternal;
   }
@@ -172,6 +195,7 @@ public class createTableDesc extends ddlDesc implements Serializable
   /**
    * @return the sortCols
    */
+  @explain(displayName="sort columns")
   public List<String> getSortCols() {
     return sortCols;
   }

+ 13 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/descTableDesc.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import java.io.File;
+import java.io.IOException;
 import java.io.Serializable;
 
+@explain(displayName="Describe Table")
 public class descTableDesc extends ddlDesc implements Serializable 
 {
   private static final long serialVersionUID = 1L;
@@ -37,6 +39,7 @@ public class descTableDesc extends ddlDesc implements Serializable
     this.tableName = tableName;
   }
 
+  @explain(displayName="table")
   public String getTableName() {
     return tableName;
   }
@@ -52,6 +55,16 @@ public class descTableDesc extends ddlDesc implements Serializable
     return resFile;
   }
 
+  @explain(displayName="result file", normalExplain=false)
+  public String getResFileString() {
+    try {
+      return getResFile().getCanonicalPath();
+    }
+    catch (IOException ioe) {
+      return "error";
+    }
+  }
+  
   /**
    * @param resFile the resFile to set
    */

+ 2 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/dropTableDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Drop Table")
 public class dropTableDesc extends ddlDesc implements Serializable 
 {
   private static final long serialVersionUID = 1L;
@@ -36,6 +37,7 @@ public class dropTableDesc extends ddlDesc implements Serializable
   /**
    * @return the tableName
    */
+  @explain(displayName="table")
   public String getTableName() {
     return tableName;
   }

+ 28 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/explain.java

@@ -0,0 +1,28 @@
+/**
+ * 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.plan;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+@Retention(RetentionPolicy.RUNTIME)
+public @interface explain {
+  String displayName() default "";
+  boolean normalExplain() default true;
+}

+ 79 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/explainWork.java

@@ -0,0 +1,79 @@
+/**
+ * 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.plan;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.Task;
+
+public class explainWork implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private File resFile;
+  private List<Task<? extends Serializable>> rootTasks;
+  private String astStringTree;
+  boolean extended;
+  
+  public explainWork() { }
+  
+  public explainWork(File resFile, 
+                     List<Task<? extends Serializable>> rootTasks,
+                     String astStringTree,
+                     boolean extended) {
+    this.resFile = resFile;
+    this.rootTasks = rootTasks;
+    this.astStringTree = astStringTree;
+    this.extended = extended;
+  }
+  
+  public File getResFile() {
+    return resFile;
+  }
+  
+  public void setResFile(File resFile) {
+    this.resFile = resFile;
+  }
+  
+  public List<Task<? extends Serializable>> getRootTasks() {
+    return rootTasks;
+  }
+  
+  public void setRootTasks(List<Task<? extends Serializable>> rootTasks) {
+    this.rootTasks = rootTasks;
+  }
+  
+  public String getAstStringTree() {
+    return astStringTree;
+  }
+  
+  public void setAstStringTree(String astStringTree) {
+    this.astStringTree = astStringTree;
+  }
+  
+  public boolean getExtended() {
+    return extended;
+  }
+  
+  public void setExtended(boolean extended) {
+    this.extended = extended;
+  }
+}
+

+ 1 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/explosionDesc.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Explosion")
 public class explosionDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private String fieldName;

+ 6 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeColumnDesc.java

@@ -44,4 +44,10 @@ public class exprNodeColumnDesc extends exprNodeDesc implements Serializable {
   public String toString() {
     return "Column[" + column + "]";
   }
+  
+  @explain(displayName="expr")
+  @Override
+  public String getExprString() {
+    return getColumn();
+  }
 }

+ 14 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeConstantDesc.java

@@ -50,4 +50,18 @@ public class exprNodeConstantDesc extends exprNodeDesc implements Serializable {
     return "Const " + typeInfo.toString() + " " + value;
   }
   
+  @explain(displayName="expr")
+  @Override
+  public String getExprString() {
+    if (value == null) {
+      return "null";
+    }
+
+    if (typeInfo.getPrimitiveClass() == String.class) {
+      return "'" + value.toString() + "'";
+    }
+    else {
+      return value.toString();
+    }
+  }
 }

+ 9 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeDesc.java

@@ -38,5 +38,13 @@ public class exprNodeDesc implements Serializable {
     this.typeInfo = typeInfo;
   }
 
-
+  public String getExprString() {
+    assert(false);
+    return null;
+  }
+  
+  @explain(displayName="type")
+  public String getTypeString() {
+    return typeInfo.getTypeString();
+  }
 }

+ 6 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeFieldDesc.java

@@ -50,5 +50,10 @@ public class exprNodeFieldDesc extends exprNodeDesc implements Serializable {
   public String toString() {
     return this.desc.toString() + "." + this.fieldName;
   }
-  
+  
+  @explain(displayName="expr")
+  @Override
+  public String getExprString() {
+    return this.desc.getExprString() + "." + this.fieldName;
+  }
 }

+ 54 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeFuncDesc.java

@@ -24,6 +24,8 @@ import java.util.ArrayList;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.parse.TypeInfo;
 
 /**
@@ -79,4 +81,56 @@ public class exprNodeFuncDesc extends exprNodeDesc implements Serializable {
     sb.append(")");
     return sb.toString();
   }
+  
+  @explain(displayName="expr")
+  @Override
+  public String getExprString() {
+    FunctionInfo fI = FunctionRegistry.getInfo(UDFClass);
+    StringBuilder sb = new StringBuilder();
+    
+    if (fI.getOpType() == FunctionInfo.OperatorType.PREFIX ||
+        fI.isAggFunction()) {
+      sb.append(fI.getDisplayName());
+      if (!fI.isOperator()) {
+        sb.append("(");
+      }
+      else {
+        sb.append(" ");
+      }
+      
+      boolean first = true;
+      for(exprNodeDesc chld: children) {
+        if (!first) {
+          sb.append(", ");
+        }
+        first = false;
+        
+        sb.append(chld.getExprString());
+      }
+      
+      if(!fI.isOperator()) {
+        sb.append(")");
+      }
+    }
+    else if (fI.getOpType() == FunctionInfo.OperatorType.INFIX) {
+      // assert that this has only 2 children
+      assert(children.size() == 2);
+      sb.append("(");
+      sb.append(children.get(0).getExprString());
+      sb.append(" ");
+      sb.append(fI.getDisplayName());
+      sb.append(" ");
+      sb.append(children.get(1).getExprString());
+      sb.append(")");
+    }
+    else if (fI.getOpType() == FunctionInfo.OperatorType.POSTFIX) {
+      // assert for now as there should be no such case
+      assert(children.size() == 1);
+      sb.append(children.get(0).getExprString());
+      sb.append(" ");
+      sb.append(fI.getDisplayName());
+    }
+    
+    return sb.toString();
+  }
 }

+ 6 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeIndexDesc.java

@@ -50,5 +50,10 @@ public class exprNodeIndexDesc extends exprNodeDesc implements Serializable {
   public String toString() {
     return this.desc.toString() + "[" + this.index + "]";
   }
-  
+  
+  @explain(displayName="expr")
+  @Override
+  public String getExprString() {
+    return this.desc.getExprString() + "[" + this.index.getExprString() + "]";
+  }
 }

+ 7 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/exprNodeNullDesc.java

@@ -23,6 +23,8 @@ import java.io.Serializable;
 import org.apache.hadoop.hive.ql.parse.TypeInfo;
 
 public class exprNodeNullDesc extends exprNodeDesc implements Serializable {
+  
+  private static final long serialVersionUID = 1L;
 
   public exprNodeNullDesc() {
     super(new TypeInfo(Void.class));
@@ -32,4 +34,9 @@ public class exprNodeNullDesc extends exprNodeDesc implements Serializable {
     return null;
   }
 
+  @explain(displayName="expr")
+  @Override
+  public String getExprString() {
+    return "null";
+  }
 }

+ 1 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/extractDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Extract")
 public class extractDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private exprNodeDesc col;

+ 5 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/fileSinkDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="File Output Operator")
 public class fileSinkDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private String dirName;
@@ -33,12 +34,16 @@ public class fileSinkDesc implements Serializable {
     this.dirName = dirName;
     this.tableInfo = tableInfo;
   }
+  
+  @explain(displayName="directory", normalExplain=false)
   public String getDirName() {
     return this.dirName;
   }
   public void setDirName(final String dirName) {
     this.dirName = dirName;
   }
+  
+  @explain(displayName="table")
   public tableDesc getTableInfo() {
     return this.tableInfo;
   }

+ 2 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/filterDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Filter Operator")
 public class filterDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private org.apache.hadoop.hive.ql.plan.exprNodeDesc predicate;
@@ -28,6 +29,7 @@ public class filterDesc implements Serializable {
     final org.apache.hadoop.hive.ql.plan.exprNodeDesc predicate) {
     this.predicate = predicate;
   }
+  @explain(displayName="predicate")
   public org.apache.hadoop.hive.ql.plan.exprNodeDesc getPredicate() {
     return this.predicate;
   }

+ 1 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/forwardDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Forward")
 public class forwardDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   @SuppressWarnings("nls")

+ 19 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/groupByDesc.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
+@explain(displayName="Group By Operator")
 public class groupByDesc implements java.io.Serializable {
   /** Group-by Mode:
    *  COMPLETE: complete 1-phase aggregation: aggregate, evaluate
@@ -42,15 +43,33 @@ public class groupByDesc implements java.io.Serializable {
   public Mode getMode() {
     return this.mode;
   }
+  @explain(displayName="mode")
+  public String getModeString() {
+    switch(mode) {
+    case COMPLETE:
+      return "complete";
+    case PARTIAL1:
+      return "partial1";
+    case PARTIAL2:
+      return "partial2";
+    case HASH:
+      return "hash";
+    }
+  
+    return "unknown";
+  }
   public void setMode(final Mode mode) {
     this.mode = mode;
   }
+  @explain(displayName="keys")
   public java.util.ArrayList<exprNodeDesc> getKeys() {
     return this.keys;
   }
   public void setKeys(final java.util.ArrayList<exprNodeDesc> keys) {
     this.keys = keys;
   }
+  
+  @explain(displayName="")
   public java.util.ArrayList<org.apache.hadoop.hive.ql.plan.aggregationDesc> getAggregators() {
     return this.aggregators;
   }

+ 29 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/joinCond.java

@@ -77,4 +77,33 @@ public class joinCond implements Serializable {
   public void setType(final int type) {
     this.type = type;
   }
+  
+  @explain
+  public String getJoinCondString() {
+    StringBuilder sb = new StringBuilder();
+    
+    switch(type) {
+    case joinDesc.INNER_JOIN:
+      sb.append("Inner Join ");
+      break;
+    case joinDesc.FULL_OUTER_JOIN:
+      sb.append("Outer Join ");
+      break;
+    case joinDesc.LEFT_OUTER_JOIN:
+      sb.append("Left Outer Join");
+      break;
+    case joinDesc.RIGHT_OUTER_JOIN:
+      sb.append("Right Outer Join");
+      break;
+    default:
+      sb.append("Unknow Join");
+      break;
+    }
+    
+    sb.append(left);
+    sb.append(" to ");
+    sb.append(right);
+    
+    return sb.toString();
+  }
 }

+ 46 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/joinDesc.java

@@ -22,12 +22,15 @@ import java.io.Serializable;
 
 import org.apache.hadoop.hive.ql.plan.exprNodeDesc;
 import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 
 /**
  * Join operator Descriptor implementation.
  * 
  */
+@explain(displayName="Join Operator")
 public class joinDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   public static final int INNER_JOIN = 0;
@@ -67,6 +70,35 @@ public class joinDesc implements Serializable {
     return this.exprs;
   }
 
+  @explain(displayName="condition expressions")
+  public Map<Byte, String> getExprsStringMap() {
+    if (getExprs() == null) {
+      return null;
+    }
+    
+    LinkedHashMap<Byte, String> ret = new LinkedHashMap<Byte, String>();
+    
+    for(Map.Entry<Byte, ArrayList<exprNodeDesc>> ent: getExprs().entrySet()) {
+      StringBuilder sb = new StringBuilder();
+      boolean first = true;
+      if (ent.getValue() != null) {
+        for(exprNodeDesc expr: ent.getValue()) {
+          if (!first) {
+            sb.append(" ");
+          }
+          
+          first = false;
+          sb.append("{");
+          sb.append(expr.getExprString());
+          sb.append("}");
+        }
+      }
+      ret.put(ent.getKey(), sb.toString());
+    }
+    
+    return ret;
+  }
+  
   public void setExprs(final Map<Byte, ArrayList<exprNodeDesc>> exprs) {
     this.exprs = exprs;
   }
@@ -79,6 +111,20 @@ public class joinDesc implements Serializable {
     this.noOuterJoin = noOuterJoin;
   }
 
+  @explain(displayName="condition map")
+  public List<joinCond> getCondsList() {
+    if (conds == null) {
+      return null;
+    }
+
+    ArrayList<joinCond> l = new ArrayList<joinCond>();
+    for(joinCond cond: conds) {
+      l.add(cond);
+    }
+
+    return l;
+  }
+
   public joinCond[] getConds() {
     return this.conds;
   }

+ 2 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/loadDesc.java

@@ -30,6 +30,8 @@ public class loadDesc implements Serializable {
 
     this.sourceDir = sourceDir;
   }
+  
+  @explain(displayName="source", normalExplain=false)
   public String getSourceDir() {
     return this.sourceDir;
   }

+ 4 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/loadFileDesc.java

@@ -36,12 +36,16 @@ public class loadFileDesc extends loadDesc implements Serializable {
     this.targetDir = targetDir;
     this.isDfsDir = isDfsDir;
   }
+  
+  @explain(displayName="destination")
   public String getTargetDir() {
     return this.targetDir;
   }
   public void setTargetDir(final String targetDir) {
     this.targetDir=targetDir;
   }
+  
+  @explain(displayName="hdfs directory")
   public boolean getIsDfsDir() {
     return this.isDfsDir;
   }

+ 4 - 2
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/loadTableDesc.java

@@ -48,14 +48,15 @@ public class loadTableDesc extends org.apache.hadoop.hive.ql.plan.loadDesc imple
     this(sourceDir, table, partitionSpec, true);
   }
 
-
-
+  @explain(displayName="table")
   public tableDesc getTable() {
     return this.table;
   }
   public void setTable(final org.apache.hadoop.hive.ql.plan.tableDesc table) {
     this.table = table;
   }
+  
+  @explain(displayName="partition")
   public HashMap<String, String> getPartitionSpec() {
     return this.partitionSpec;
   }
@@ -63,6 +64,7 @@ public class loadTableDesc extends org.apache.hadoop.hive.ql.plan.loadDesc imple
     this.partitionSpec = partitionSpec;
   }
 
+  @explain(displayName="replace")
   public boolean getReplace() {
     return replace;
   }

+ 17 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/mapredWork.java

@@ -24,6 +24,7 @@ import java.io.*;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 
+@explain(displayName="Map Reduce")
 public class mapredWork implements Serializable {
   private static final long serialVersionUID = 1L;
   private String command;
@@ -31,18 +32,22 @@ public class mapredWork implements Serializable {
   //   use LinkedHashMap to make sure the iteration order is
   //   deterministic, to ease testing
   private LinkedHashMap<String,ArrayList<String>> pathToAliases;
+  
   private LinkedHashMap<String,partitionDesc> pathToPartitionInfo;
+  
   private HashMap<String,Operator<? extends Serializable>> aliasToWork;
 
   // map<->reduce interface
   // schema of the map-reduce 'key' object - this is homogeneous
   private schemaDesc keySchema;
+
   // schema of the map-reduce 'val' object - this is heterogeneous
   private HashMap<String,schemaDesc> aliasToSchema;
 
-
   private Operator<?> reducer;
+  
   private Integer numReduceTasks;
+  
   private boolean needsTagging;
 
   public mapredWork() { }
@@ -70,18 +75,24 @@ public class mapredWork implements Serializable {
   public void setCommand(final String command) {
     this.command = command;
   }
+
+  @explain(displayName="Path -> Alias", normalExplain=false)
   public LinkedHashMap<String,ArrayList<String>> getPathToAliases() {
     return this.pathToAliases;
   }
   public void setPathToAliases(final LinkedHashMap<String,ArrayList<String>> pathToAliases) {
     this.pathToAliases = pathToAliases;
   }
+
+  @explain(displayName="Path -> Partition", normalExplain=false)
   public LinkedHashMap<String,partitionDesc> getPathToPartitionInfo() {
     return this.pathToPartitionInfo;
   }
   public void setPathToPartitionInfo(final LinkedHashMap<String,partitionDesc> pathToPartitionInfo) {
     this.pathToPartitionInfo = pathToPartitionInfo;
   }
+  
+  @explain(displayName="Alias -> Map Operator Tree")
   public HashMap<String, Operator<? extends Serializable>> getAliasToWork() {
     return this.aliasToWork;
   }
@@ -101,12 +112,16 @@ public class mapredWork implements Serializable {
     this.aliasToSchema = aliasToSchema;
   }
 
+  @explain(displayName="Reduce Operator Tree")
   public Operator<?> getReducer() {
     return this.reducer;
   }
+
   public void setReducer(final Operator<?> reducer) {
     this.reducer = reducer;
   }
+
+  @explain(displayName="# Reducers")
   public Integer getNumReduceTasks() {
     return this.numReduceTasks;
   }
@@ -174,6 +189,7 @@ public class mapredWork implements Serializable {
     setAliases();
   }
 
+  @explain(displayName="Needs Tagging", normalExplain=false)
   public boolean getNeedsTagging() {
     return this.needsTagging;
   }

+ 4 - 1
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/moveWork.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.plan;
 import java.util.*;
 import java.io.*;
 
-
+@explain(displayName="Move Operator")
 public class moveWork implements Serializable {
   private static final long serialVersionUID = 1L;
   private List<loadTableDesc> loadTableWork;
@@ -34,12 +34,15 @@ public class moveWork implements Serializable {
     this.loadTableWork = loadTableWork;
     this.loadFileWork = loadFileWork;
   }
+  @explain(displayName="tables")
   public List<loadTableDesc> getLoadTableWork() {
     return this.loadTableWork;
   }
   public void setLoadTableWork(final List<loadTableDesc> loadTableWork) {
     this.loadTableWork = loadTableWork;
   }
+  
+  @explain(displayName="files")
   public List<loadFileDesc> getLoadFileWork() {
     return this.loadFileWork;
   }

+ 5 - 4
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/partitionDesc.java

@@ -19,11 +19,8 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
-import java.util.*;
-
-import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.metadata.Partition;
 
+@explain(displayName="Partition")
 public class partitionDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private tableDesc table;
@@ -35,12 +32,16 @@ public class partitionDesc implements Serializable {
     this.table = table;
     this.partSpec = partSpec;
   }
+  
+  @explain(displayName="")
   public tableDesc getTableDesc() {
     return this.table;
   }
   public void setTableDesc(final tableDesc table) {
     this.table = table;
   }
+  
+  @explain(displayName="partition values")
   public java.util.LinkedHashMap<String, String> getPartSpec() {
     return this.partSpec;
   }

+ 6 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/reduceSinkDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Reduce Output Operator")
 public class reduceSinkDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   // these are the expressions that go into the reduce key
@@ -56,6 +57,7 @@ public class reduceSinkDesc implements Serializable {
     this.numPartitionFields = numPartitionFields;
   }
 
+  @explain(displayName="key expressions")
   public java.util.ArrayList<exprNodeDesc> getKeyCols() {
     return this.keyCols;
   }
@@ -64,6 +66,7 @@ public class reduceSinkDesc implements Serializable {
     this.keyCols=keyCols;
   }
 
+  @explain(displayName="value expressions")
   public java.util.ArrayList<exprNodeDesc> getValueCols() {
     return this.valueCols;
   }
@@ -72,12 +75,15 @@ public class reduceSinkDesc implements Serializable {
     this.valueCols=valueCols;
   }
   
+  @explain(displayName="# partition fields")
   public int getNumPartitionFields() {
     return this.numPartitionFields;
   }
   public void setNumPartitionFields(int numPartitionFields) {
     this.numPartitionFields = numPartitionFields;
   }
+  
+  @explain(displayName="tag")
   public int getTag() {
     return this.tag;
   }

+ 5 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/scriptDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Transform Operator")
 public class scriptDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private String scriptCmd;
@@ -33,12 +34,16 @@ public class scriptDesc implements Serializable {
     this.scriptCmd = scriptCmd;
     this.scriptOutputInfo = scriptOutputInfo;
   }
+  
+  @explain(displayName="command")
   public String getScriptCmd() {
     return this.scriptCmd;
   }
   public void setScriptCmd(final String scriptCmd) {
     this.scriptCmd=scriptCmd;
   }
+  
+  @explain(displayName="output info")
   public tableDesc getScriptOutputInfo() {
     return this.scriptOutputInfo;
   }

+ 2 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/selectDesc.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+@explain(displayName="Select Operator")
 public class selectDesc implements Serializable {
   private static final long serialVersionUID = 1L;
   private java.util.ArrayList<org.apache.hadoop.hive.ql.plan.exprNodeDesc> colList;
@@ -28,6 +29,7 @@ public class selectDesc implements Serializable {
     final java.util.ArrayList<org.apache.hadoop.hive.ql.plan.exprNodeDesc> colList) {
     this.colList = colList;
   }
+  @explain(displayName="expressions")
   public java.util.ArrayList<org.apache.hadoop.hive.ql.plan.exprNodeDesc> getColList() {
     return this.colList;
   }

+ 12 - 0
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/showTablesDesc.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.hive.ql.plan;
 
 import java.io.File;
+import java.io.IOException;
 import java.io.Serializable;
 
+@explain(displayName="Show Tables")
 public class showTablesDesc extends ddlDesc implements Serializable 
 {
   private static final long serialVersionUID = 1L;
@@ -46,6 +48,7 @@ public class showTablesDesc extends ddlDesc implements Serializable
   /**
    * @return the pattern
    */
+  @explain(displayName="pattern")
   public String getPattern() {
     return pattern;
   }
@@ -64,6 +67,15 @@ public class showTablesDesc extends ddlDesc implements Serializable
     return resFile;
   }
 
+  @explain(displayName="result file", normalExplain=false)
+  public String getResFileString() {
+    try {
+      return getResFile().getCanonicalPath();
+    }
+    catch (IOException ioe) {
+      return "error";
+    }
+  }
   /**
    * @param resFile the resFile to set
    */

+ 14 - 3
src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/tableDesc.java

@@ -23,9 +23,6 @@ import java.io.Serializable;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.hive.serde.SerDe;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
 
 public class tableDesc implements Serializable {
   private static final long serialVersionUID = 1L;
@@ -64,6 +61,8 @@ public class tableDesc implements Serializable {
   public void setOutputFileFormatClass(final Class<? extends OutputFormat> outputFileFormatClass) {
     this.outputFileFormatClass=outputFileFormatClass;
   }
+  
+  @explain(displayName="properties", normalExplain=false)
   public java.util.Properties getProperties() {
     return this.properties;
   }
@@ -73,6 +72,7 @@ public class tableDesc implements Serializable {
   /**
    * @return the serdeClassName
    */
+  @explain(displayName="serde")
   public String getSerdeClassName() {
     return this.serdeClassName;
   }
@@ -83,7 +83,18 @@ public class tableDesc implements Serializable {
     this.serdeClassName = serdeClassName;
   }
   
+  @explain(displayName="name")
   public String getTableName() {
     return this.properties.getProperty(org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_NAME);
   }
+  
+  @explain(displayName="input format")
+  public String getInputFileFormatClassName() {
+    return getInputFileFormatClass().getName();
+  }
+  
+  @explain(displayName="output format")
+  public String getOutputFileFormatClassName() {
+    return getOutputFileFormatClass().getName();
+  }
 }

+ 34 - 7
src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java

@@ -153,9 +153,9 @@ public class QTestUtil {
     testFiles = conf.get("test.data.files").replace('\\', '/').replace("c:", "");
     logDir = conf.get("test.log.dir");
 
-    String ow = System.getenv("overwrite");
+    String ow = System.getProperty("test.output.overwrite");
     overWrite = false;
-    if ((ow != null) && (ow.length() > 0)) {
+    if ((ow != null) && ow.equalsIgnoreCase("true")){
       overWrite = true;
     }
 
@@ -551,12 +551,26 @@ public class QTestUtil {
                          cmdArray[3] + " " + cmdArray[4] + " " + cmdArray[5]);
     }
     else {
+      System.out.println("overwritting");
       // Remove any existing output
       String [] cmdArray1 = new String[5];
       cmdArray1[0] = "rm";
       cmdArray1[1] = "-rf";
       cmdArray1[2] = (new File(outDir, tname)).getPath();
       System.out.println(cmdArray1[0] + " " + cmdArray1[1] + " " + cmdArray1[2]);
+
+      Process executor = Runtime.getRuntime().exec(cmdArray1);
+      
+      StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), null, System.out);
+      StreamPrinter errPrinter = new StreamPrinter(executor.getErrorStream(), null, System.err);
+      
+      outPrinter.start();
+      errPrinter.start();
+      int exitVal = executor.waitFor();
+      if (exitVal != 0) {
+        return exitVal;
+      }
+
       // Capture code
       cmdArray = new String[5];
       cmdArray[0] = "cp";
@@ -581,11 +595,24 @@ public class QTestUtil {
 
   public int checkCliDriverResults(String tname) throws Exception {
     String [] cmdArray;
-    cmdArray = new String[3];
-    cmdArray[0] = "diff";
-    cmdArray[1] = (new File(logDir, tname + ".out")).getPath();
-    cmdArray[2] = (new File(outDir, tname + ".out")).getPath();
-    System.out.println(cmdArray[0] + " " + cmdArray[1] + " " + cmdArray[2]);
+
+    if (!overWrite) {
+      cmdArray = new String[5];
+      cmdArray[0] = "diff";
+      cmdArray[1] = "-I";
+      cmdArray[2] = "\\|\\(tmp/hive-.*\\)";
+      cmdArray[3] = (new File(logDir, tname + ".out")).getPath();
+      cmdArray[4] = (new File(outDir, tname + ".out")).getPath();
+      System.out.println(cmdArray[0] + " " + cmdArray[1] + " " + cmdArray[2] + " " +
+                         cmdArray[3] + " " + cmdArray[4]);
+    }
+    else {
+      cmdArray = new String[3];
+      cmdArray[0] = "cp";
+      cmdArray[1] = (new File(logDir, tname + ".out")).getPath();
+      cmdArray[2] = (new File(outDir, tname + ".out")).getPath();
+      System.out.println(cmdArray[0] + " " + cmdArray[1] + " " + cmdArray[2]);
+    }
 
     Process executor = Runtime.getRuntime().exec(cmdArray);
 

+ 1 - 1
src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/exec/TestCompositeHiveObject.java

@@ -111,7 +111,7 @@ public class TestCompositeHiveObject extends TestCase {
       // evaluate on row
       op.process(cr);
 
-      Map<Enum, Long> results = op.getStats();
+      Map<Enum<?>, Long> results = op.getStats();
       assertEquals(results.get(FilterOperator.Counter.FILTERED), Long.valueOf(0));
       assertEquals(results.get(FilterOperator.Counter.PASSED), Long.valueOf(1));
       System.out.println("Filter Operator ok");

+ 2 - 2
src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java

@@ -150,8 +150,8 @@ public class TestExecDriver extends TestCase {
 
     exprNodeDesc desc = new exprNodeFuncDesc(
         TypeInfo.getPrimitiveTypeInfo(Boolean.class),
-        UDFRegistry.getUDFClass("<"),
-        UDFRegistry.getUDFMethod("<", true, String.class, Number.class),
+        FunctionRegistry.getUDFClass("<"),
+        FunctionRegistry.getUDFMethod("<", true, String.class, Number.class),
         children
     );
     return new filterDesc(desc);

+ 1 - 1
src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/exec/TestJEXL.java

@@ -68,7 +68,7 @@ public class TestJEXL extends TestCase {
       int basetimes = 100000;
 
       JexlContext jc = JexlHelper.createContext();
-      jc.getVars().put("__udf__concat", UDFRegistry.getUDFClass("concat").newInstance());
+      jc.getVars().put("__udf__concat", FunctionRegistry.getUDFClass("concat").newInstance());
       
       measureSpeed("1 + 2",
           basetimes * 100,

+ 1 - 1
src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java

@@ -85,7 +85,7 @@ public class TestOperators extends TestCase {
         op.process(oner);
       }
 
-      Map<Enum, Long> results = op.getStats();
+      Map<Enum<?>, Long> results = op.getStats();
       assertEquals(results.get(FilterOperator.Counter.FILTERED), Long.valueOf(4));
       assertEquals(results.get(FilterOperator.Counter.PASSED), Long.valueOf(1));
 

+ 5 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/cast1.q

@@ -1,4 +1,9 @@
 CREATE TABLE dest1(c1 INT, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 INT, c6 INT, c7 INT);
+
+EXPLAIN
+FROM src INSERT OVERWRITE TABLE dest1 SELECT 3 + 2, 3.0 + 2, 3 + 2.0, 3.0 + 2.0, 3 + CAST(2.0 AS INT), CAST(1 AS BOOLEAN), CAST(TRUE AS INT) WHERE src.key = 86;
+
 FROM src INSERT OVERWRITE TABLE dest1 SELECT 3 + 2, 3.0 + 2, 3 + 2.0, 3.0 + 2.0, 3 + CAST(2.0 AS INT), CAST(1 AS BOOLEAN), CAST(TRUE AS INT) WHERE src.key = 86;
+
 select dest1.* FROM dest1;
 

+ 5 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/groupby1.q

@@ -1,3 +1,8 @@
 CREATE TABLE dest1(key INT, value DOUBLE);
+
+EXPLAIN
 FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,4)) GROUP BY src.key;
+
+FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,4)) GROUP BY src.key;
+
 SELECT dest1.* FROM dest1;

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/groupby2.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(key STRING, c1 INT, c2 STRING);
 
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,0,1), count(DISTINCT substr(src.value,4)), concat(substr(src.key,0,1),sum(substr(src.value,4))) GROUP BY substr(src.key,0,1);
+
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,0,1), count(DISTINCT substr(src.value,4)), concat(substr(src.key,0,1),sum(substr(src.value,4))) GROUP BY substr(src.key,0,1);
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/groupby3.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(c1 INT, c2 INT, c3 INT, c4 INT, c5 INT);
 
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT sum(substr(src.value,4)), avg(substr(src.value,4)), avg(DISTINCT substr(src.value,4)), max(substr(src.value,4)), min(substr(src.value,4));
+
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT sum(substr(src.value,4)), avg(substr(src.value,4)), avg(DISTINCT substr(src.value,4)), max(substr(src.value,4)), min(substr(src.value,4));
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/groupby4.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(c1 STRING);
 
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,0,1) GROUP BY substr(src.key,0,1);
+
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,0,1) GROUP BY substr(src.key,0,1);
 

+ 6 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/groupby5.q

@@ -1,5 +1,11 @@
 CREATE TABLE dest1(key INT, value STRING);
 
+EXPLAIN
+INSERT OVERWRITE TABLE dest1 
+SELECT src.key, sum(substr(src.value,4)) 
+FROM src
+GROUP BY src.key;
+
 INSERT OVERWRITE TABLE dest1 
 SELECT src.key, sum(substr(src.value,4)) 
 FROM src

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/groupby6.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(c1 STRING);
 
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,4,1);
+
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,4,1);
 

+ 5 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input1.q

@@ -1,4 +1,9 @@
 CREATE TABLE TEST1(A INT, B FLOAT); 
+
+EXPLAIN
+DESCRIBE TEST1; 
+
 DESCRIBE TEST1; 
+
 DROP TABLE TEST1;
 

+ 5 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input10.q

@@ -1,4 +1,9 @@
 CREATE TABLE TEST10(key INT, value STRING) PARTITIONED BY(ds STRING, hr STRING);
+
+EXPLAIN
+DESCRIBE TEST10;
+
 DESCRIBE TEST10;
+
 DROP TABLE TEST10;
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input11.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(key INT, value STRING);
 
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100;
+
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100;
 

+ 6 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input12.q

@@ -2,6 +2,12 @@ CREATE TABLE dest1(key INT, value STRING);
 CREATE TABLE dest2(key INT, value STRING);
 CREATE TABLE dest3(key INT, value STRING) PARTITIONED BY(ds STRING, hr STRING);
 
+EXPLAIN
+FROM src 
+INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100
+INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200
+INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key WHERE src.key >= 200;
+
 FROM src 
 INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100
 INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200

+ 7 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input13.q

@@ -2,6 +2,13 @@ CREATE TABLE dest1(key INT, value STRING);
 CREATE TABLE dest2(key INT, value STRING);
 CREATE TABLE dest3(key INT, value STRING) PARTITIONED BY(ds STRING, hr STRING);
 
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100
+INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200
+INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key WHERE src.key >= 200 and src.key < 300
+INSERT OVERWRITE DIRECTORY '../../../../build/contrib/hive/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300;
+
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100
 INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200

+ 9 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input14.q

@@ -1,5 +1,14 @@
 CREATE TABLE dest1(key INT, value STRING);
 
+EXPLAIN
+FROM (
+  FROM src
+  SELECT TRANSFORM(src.key, src.value) AS (tkey, tvalue) 
+         USING '/bin/cat'
+  CLUSTER BY tkey 
+) tmap
+INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100;
+
 FROM (
   FROM src
   SELECT TRANSFORM(src.key, src.value) AS (tkey, tvalue) 

+ 3 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input15.q

@@ -1,4 +1,7 @@
+EXPLAIN
 CREATE TABLE TEST15(key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
+
 DESCRIBE TEST15;
+
 DROP TABLE TEST15;
 

+ 3 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input2.q

@@ -7,3 +7,6 @@ DROP TABLE TEST2a;
 SHOW TABLES;
 DROP TABLE TEST2b;
 
+EXPLAIN
+SHOW TABLES;
+

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input3.q

@@ -3,8 +3,12 @@ DESCRIBE TEST3a;
 CREATE TABLE TEST3b(A ARRAY<INT>, B FLOAT, C MAP<FLOAT, INT>); 
 DESCRIBE TEST3b; 
 SHOW TABLES;
+EXPLAIN
+ALTER TABLE TEST3b ADD COLUMNS (X FLOAT);
 ALTER TABLE TEST3b ADD COLUMNS (X FLOAT);
 DESCRIBE TEST3b; 
+EXPLAIN
+ALTER TABLE TEST3b RENAME TO TEST3c;
 ALTER TABLE TEST3b RENAME TO TEST3c;
 DESCRIBE TEST3c; 
 SHOW TABLES;

+ 2 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input4.q

@@ -1,4 +1,6 @@
 CREATE TABLE INPUT4(KEY STRING, VALUE STRING);
+EXPLAIN
+LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4;
 LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4;
 SELECT INPUT4.VALUE, INPUT4.KEY FROM INPUT4;
 DROP TABLE INPUT4;

+ 9 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input5.q

@@ -1,5 +1,14 @@
 CREATE TABLE dest1(key INT, value STRING);
 
+EXPLAIN
+FROM (
+  FROM src_thrift
+  SELECT TRANSFORM(src_thrift.lint, src_thrift.lintstring) AS (tkey, tvalue) 
+         USING '/bin/cat'
+  CLUSTER BY tkey 
+) tmap
+INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue;
+
 FROM (
   FROM src_thrift
   SELECT TRANSFORM(src_thrift.lint, src_thrift.lintstring) AS (tkey, tvalue) 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input6.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(key STRING, value STRING);
 
+EXPLAIN
+FROM src1
+INSERT OVERWRITE TABLE dest1 SELECT src1.key, src1.value WHERE src1.key is null;
+
 FROM src1
 INSERT OVERWRITE TABLE dest1 SELECT src1.key, src1.value WHERE src1.key is null;
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input7.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(c1 DOUBLE, c2 INT);
 
+EXPLAIN
+FROM src1
+INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key;
+
 FROM src1
 INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key;
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input8.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(c1 STRING, c2 INT, c3 DOUBLE);
 
+EXPLAIN
+FROM src1 
+INSERT OVERWRITE TABLE dest1 SELECT 4 + NULL, src1.key - NULL, NULL + NULL;
+
 FROM src1 
 INSERT OVERWRITE TABLE dest1 SELECT 4 + NULL, src1.key - NULL, NULL + NULL;
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input9.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(value STRING, key INT);
 
+EXPLAIN
+FROM src1
+INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key where NULL = NULL;
+
 FROM src1
 INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key where NULL = NULL;
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input_part1.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(key INT, value STRING, hr STRING, ds STRING);
 
+EXPLAIN
+FROM srcpart
+INSERT OVERWRITE TABLE dest1 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12';
+
 FROM srcpart
 INSERT OVERWRITE TABLE dest1 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12';
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input_testsequencefile.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest4_sequencefile(key INT, value STRING) STORED AS COMPRESSED;
 
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value;
+
 FROM src
 INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value;
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/input_testxpath.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(key INT, value STRING);
 
+EXPLAIN
+FROM src_thrift
+INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[1], src_thrift.lintstring[0].mystring;
+
 FROM src_thrift
 INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[1], src_thrift.lintstring[0].mystring;
 

+ 5 - 1
src/contrib/hive/ql/src/test/queries/clientpositive/inputddl1.q

@@ -1,4 +1,8 @@
+EXPLAIN
 CREATE TABLE INPUTDDL1(key INT, value STRING); 
+
+CREATE TABLE INPUTDDL1(key INT, value STRING); 
+
 SELECT INPUTDDL1.* from INPUTDDL1;
-DROP TABLE INPUTDDL1;
 
+DROP TABLE INPUTDDL1;

+ 2 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/inputddl2.q

@@ -1,3 +1,5 @@
+EXPLAIN
+CREATE TABLE INPUTDDL2(key INT, value STRING) PARTITIONED BY(ds DATETIME, country STRING);
 CREATE TABLE INPUTDDL2(key INT, value STRING) PARTITIONED BY(ds DATETIME, country STRING);
 DESCRIBE INPUTDDL2;
 DROP TABLE INPUTDDL2;

+ 2 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/inputddl3.q

@@ -1,3 +1,5 @@
+EXPLAIN
+CREATE TABLE INPUTDDL3(key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
 CREATE TABLE INPUTDDL3(key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
 DESCRIBE INPUTDDL3;
 DROP TABLE INPUTDDL3;

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/join1.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(key INT, value STRING);
 
+EXPLAIN
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value;
+
 FROM src src1 JOIN src src2 ON (src1.key = src2.key)
 INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value;
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/join2.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(key INT, value STRING);
 
+EXPLAIN
+FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value;
+
 FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key)
 INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value;
 

+ 4 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/join3.q

@@ -1,5 +1,9 @@
 CREATE TABLE dest1(key INT, value STRING);
 
+EXPLAIN
+FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key)
+INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value;
+
 FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key)
 INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value;
 

+ 15 - 0
src/contrib/hive/ql/src/test/queries/clientpositive/join4.q

@@ -1,5 +1,20 @@
 CREATE TABLE dest1(c1 INT, c2 STRING, c3 INT, c4 STRING);
 
+EXPLAIN
+FROM (
+ FROM 
+  (
+  FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20
+  ) a
+ LEFT OUTER JOIN 
+ (
+  FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25
+ ) b 
+ ON (a.c1 = b.c3)
+ SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4
+) c
+INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4;
+
 FROM (
  FROM 
   (

Some files were not shown because too many files changed in this diff