You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jv...@apache.org on 2010/07/12 03:21:43 UTC

svn commit: r963173 - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/ ql/src/test/queries/clientnegative/ ql/src/test/queries/clientpo...

Author: jvs
Date: Mon Jul 12 01:21:41 2010
New Revision: 963173

URL: http://svn.apache.org/viewvc?rev=963173&view=rev
Log:
HIVE-287. Support count(*) and count distinct on multiple columns
(Arvind Prabhakar via jvs)


Added:
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/AbstractGenericUDAFResolver.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFParameterInfo.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver2.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/SimpleGenericUDAFParameterInfo.java
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_avg_syntax.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_max_syntax.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_min_syntax.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_std_syntax.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_sum_syntax.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q
    hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_variance_syntax.q
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_avg_syntax.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_max_syntax.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_min_syntax.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_std_syntax.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_stddev_samp_syntax.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_sum_syntax.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_var_samp_syntax.q.out
    hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_variance_syntax.q.out
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBridge.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMin.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
    hadoop/hive/trunk/ql/src/test/queries/clientpositive/udf_count.q
    hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_count.q.out

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Mon Jul 12 01:21:41 2010
@@ -136,6 +136,9 @@ Release 0.6.0 -  Unreleased
     HIVE-1255. Add mathematical UDFs PI, E, degrees, radians, tan,
     sign, and atan.  (Edward Capriolo via jvs)
 
+    HIVE-287. Support count(*) and count distinct on multiple columns
+    (Arvind Prabhakar via jvs)
+
   IMPROVEMENTS
     HIVE-983. Function from_unixtime takes long.
     (Ning Zhang via zshao)

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java Mon Jul 12 01:21:41 2010
@@ -137,8 +137,10 @@ import org.apache.hadoop.hive.ql.udf.gen
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFHistogramNumeric;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMin;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFParameterInfo;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentileApprox;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver2;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStd;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStdSample;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum;
@@ -168,6 +170,7 @@ import org.apache.hadoop.hive.ql.udf.gen
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFExplode;
+import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo;
 import org.apache.hadoop.hive.ql.udf.xml.GenericUDFXPath;
 import org.apache.hadoop.hive.ql.udf.xml.UDFXPathBoolean;
 import org.apache.hadoop.hive.ql.udf.xml.UDFXPathDouble;
@@ -629,10 +632,13 @@ public final class FunctionRegistry {
    * @param argumentTypeInfos
    * @return The UDAF evaluator
    */
+  @SuppressWarnings("deprecation")
   public static GenericUDAFEvaluator getGenericUDAFEvaluator(String name,
-      List<TypeInfo> argumentTypeInfos) throws SemanticException {
-    GenericUDAFResolver udaf = getGenericUDAFResolver(name);
-    if (udaf == null) {
+      List<TypeInfo> argumentTypeInfos, boolean isDistinct,
+      boolean isAllColumns) throws SemanticException {
+
+    GenericUDAFResolver udafResolver = getGenericUDAFResolver(name);
+    if (udafResolver == null) {
       return null;
     }
 
@@ -640,7 +646,18 @@ public final class FunctionRegistry {
     for (int i = 0; i < parameters.length; i++) {
       parameters[i] = argumentTypeInfos.get(i);
     }
-    return udaf.getEvaluator(parameters);
+
+    GenericUDAFEvaluator udafEvaluator = null;
+    if (udafResolver instanceof GenericUDAFResolver2) {
+      GenericUDAFParameterInfo paramInfo =
+          new SimpleGenericUDAFParameterInfo(
+              parameters, isDistinct, isAllColumns);
+      udafEvaluator =
+          ((GenericUDAFResolver2) udafResolver).getEvaluator(paramInfo);
+    } else {
+      udafEvaluator = udafResolver.getEvaluator(parameters);
+    }
+    return udafEvaluator;
   }
 
   /**

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g Mon Jul 12 01:21:41 2010
@@ -27,6 +27,7 @@ TOK_ALLCOLREF;
 TOK_TABLE_OR_COL;
 TOK_FUNCTION;
 TOK_FUNCTIONDI;
+TOK_FUNCTIONSTAR;
 TOK_WHERE;
 TOK_OP_EQ;
 TOK_OP_NE;
@@ -1185,10 +1186,13 @@ function
     :
     functionName
     LPAREN
-      (dist=KW_DISTINCT)?
-      (expression (COMMA expression)*)?
-    RPAREN -> {$dist == null}? ^(TOK_FUNCTION functionName (expression+)?)
-                          -> ^(TOK_FUNCTIONDI functionName (expression+)?)
+      (
+        (star=STAR)
+        | (dist=KW_DISTINCT)? (expression (COMMA expression)*)?
+      )
+    RPAREN -> {$star != null}? ^(TOK_FUNCTIONSTAR functionName)
+           -> {$dist == null}? ^(TOK_FUNCTION functionName (expression+)?)
+                            -> ^(TOK_FUNCTIONDI functionName (expression+)?)
     ;
 
 functionName

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Mon Jul 12 01:21:41 2010
@@ -313,8 +313,10 @@ public class SemanticAnalyzer extends Ba
    */
   private void doPhase1GetAllAggregations(ASTNode expressionTree,
       HashMap<String, ASTNode> aggregations) {
-    if (expressionTree.getToken().getType() == HiveParser.TOK_FUNCTION
-        || expressionTree.getToken().getType() == HiveParser.TOK_FUNCTIONDI) {
+    int exprTokenType = expressionTree.getToken().getType();
+    if (exprTokenType == HiveParser.TOK_FUNCTION
+        || exprTokenType == HiveParser.TOK_FUNCTIONDI
+        || exprTokenType == HiveParser.TOK_FUNCTIONSTAR) {
       assert (expressionTree.getChildCount() != 0);
       if (expressionTree.getChild(0).getType() == HiveParser.Identifier) {
         String functionName = unescapeIdentifier(expressionTree.getChild(0)
@@ -1721,7 +1723,9 @@ public class SemanticAnalyzer extends Ba
     ASTNode udtfExpr = (ASTNode) selExprList.getChild(posn).getChild(0);
     GenericUDTF genericUDTF = null;
 
-    if (udtfExpr.getType() == HiveParser.TOK_FUNCTION) {
+    int udtfExprType = udtfExpr.getType();
+    if (udtfExprType == HiveParser.TOK_FUNCTION
+        || udtfExprType == HiveParser.TOK_FUNCTIONSTAR) {
       String funcName = TypeCheckProcFactory.DefaultExprProcessor
           .getFunctionText(udtfExpr, true);
       FunctionInfo fi = FunctionRegistry.getFunctionInfo(funcName);
@@ -1930,11 +1934,12 @@ public class SemanticAnalyzer extends Ba
    * for each GroupBy aggregation.
    */
   static GenericUDAFEvaluator getGenericUDAFEvaluator(String aggName,
-      ArrayList<ExprNodeDesc> aggParameters, ASTNode aggTree)
+      ArrayList<ExprNodeDesc> aggParameters, ASTNode aggTree,
+      boolean isDistinct, boolean isAllColumns)
       throws SemanticException {
     ArrayList<TypeInfo> originalParameterTypeInfos = getTypeInfo(aggParameters);
     GenericUDAFEvaluator result = FunctionRegistry.getGenericUDAFEvaluator(
-        aggName, originalParameterTypeInfos);
+        aggName, originalParameterTypeInfos, isDistinct, isAllColumns);
     if (null == result) {
       String reason = "Looking for UDAF Evaluator\"" + aggName
           + "\" with parameters " + originalParameterTypeInfos;
@@ -2078,9 +2083,10 @@ public class SemanticAnalyzer extends Ba
       }
 
       boolean isDistinct = value.getType() == HiveParser.TOK_FUNCTIONDI;
+      boolean isAllColumns = value.getType() == HiveParser.TOK_FUNCTIONSTAR;
       Mode amode = groupByDescModeToUDAFMode(mode, isDistinct);
       GenericUDAFEvaluator genericUDAFEvaluator = getGenericUDAFEvaluator(
-          aggName, aggParameters, value);
+          aggName, aggParameters, value, isDistinct, isAllColumns);
       assert (genericUDAFEvaluator != null);
       GenericUDAFInfo udaf = getGenericUDAFInfo(genericUDAFEvaluator, amode,
           aggParameters);
@@ -2200,12 +2206,13 @@ public class SemanticAnalyzer extends Ba
             .getIsPartitionCol()));
       }
       boolean isDistinct = (value.getType() == HiveParser.TOK_FUNCTIONDI);
+      boolean isAllColumns = value.getType() == HiveParser.TOK_FUNCTIONSTAR;
       Mode amode = groupByDescModeToUDAFMode(mode, isDistinct);
       GenericUDAFEvaluator genericUDAFEvaluator = null;
       // For distincts, partial aggregations have not been done
       if (distPartAgg) {
         genericUDAFEvaluator = getGenericUDAFEvaluator(aggName, aggParameters,
-            value);
+            value, isDistinct, isAllColumns);
         assert (genericUDAFEvaluator != null);
         genericUDAFEvaluators.put(entry.getKey(), genericUDAFEvaluator);
       } else {
@@ -2317,10 +2324,11 @@ public class SemanticAnalyzer extends Ba
       }
 
       boolean isDistinct = value.getType() == HiveParser.TOK_FUNCTIONDI;
+      boolean isAllColumns = value.getType() == HiveParser.TOK_FUNCTIONSTAR;
       Mode amode = groupByDescModeToUDAFMode(mode, isDistinct);
 
       GenericUDAFEvaluator genericUDAFEvaluator = getGenericUDAFEvaluator(
-          aggName, aggParameters, value);
+          aggName, aggParameters, value, isDistinct, isAllColumns);
       assert (genericUDAFEvaluator != null);
       GenericUDAFInfo udaf = getGenericUDAFInfo(genericUDAFEvaluator, amode,
           aggParameters);
@@ -2603,6 +2611,7 @@ public class SemanticAnalyzer extends Ba
       String aggName = value.getChild(0).getText();
 
       boolean isDistinct = value.getType() == HiveParser.TOK_FUNCTIONDI;
+      boolean isStar = value.getType() == HiveParser.TOK_FUNCTIONSTAR;
       Mode amode = groupByDescModeToUDAFMode(mode, isDistinct);
       GenericUDAFEvaluator genericUDAFEvaluator = genericUDAFEvaluators
           .get(entry.getKey());
@@ -3220,8 +3229,8 @@ public class SemanticAnalyzer extends Ba
           dpCtx = new DynamicPartitionCtx(dest_tab, partSpec,
               conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME),
               conf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTSPERNODE));
-        	qbm.setDPCtx(dest, dpCtx);
-      	}
+          qbm.setDPCtx(dest, dpCtx);
+        }
 
         if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONING)) { // allow DP
           // TODO: we should support merge files for dynamically generated partitions later
@@ -3239,7 +3248,7 @@ public class SemanticAnalyzer extends Ba
         }
         if (dpCtx.getSPPath() != null) {
           dest_path = new Path(dest_tab.getPath(), dpCtx.getSPPath());
-      	}
+        }
         if ((dest_tab.getNumBuckets() > 0) &&
             (conf.getBoolVar(HiveConf.ConfVars.HIVEENFORCEBUCKETING))) {
           dpCtx.setNumBuckets(dest_tab.getNumBuckets());
@@ -3470,10 +3479,10 @@ public class SemanticAnalyzer extends Ba
                 queryTmpdir,
                 table_desc,
                 conf.getBoolVar(HiveConf.ConfVars.COMPRESSRESULT),
-		            currentTableId,
-  		          rsCtx.isMultiFileSpray(),
-    		        rsCtx.getNumFiles(),
-      		      rsCtx.getTotalFiles(),
+                currentTableId,
+                rsCtx.isMultiFileSpray(),
+                rsCtx.getNumFiles(),
+                rsCtx.getTotalFiles(),
                 rsCtx.getPartnCols(),
                 dpCtx),
             fsRS, input), inputRR);

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/AbstractGenericUDAFResolver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/AbstractGenericUDAFResolver.java?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/AbstractGenericUDAFResolver.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/AbstractGenericUDAFResolver.java Mon Jul 12 01:21:41 2010
@@ -0,0 +1,49 @@
+/**
+ * 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.udf.generic;
+
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * An abstract class to help facilitate existing implementations of
+ * <tt>GenericUDAFResolver</tt> to migrate towards the newly introduced
+ * interface {@link GenericUDAFResolver2}. This class provides a default
+ * implementation of this new API and in turn calls
+ * the existing API {@link GenericUDAFResolver#getEvaluator(TypeInfo[])} by
+ * ignoring the extra parameter information available via the
+ * <tt>GenericUDAFParameterInfo</tt> interface.
+ *
+ */
+public abstract class AbstractGenericUDAFResolver
+    implements GenericUDAFResolver2
+{
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public GenericUDAFEvaluator getEvaluator(GenericUDAFParameterInfo info)
+    throws SemanticException {
+
+    if (info.isAllColumns()) {
+      throw new SemanticException(
+          "The specified syntax for UDAF invocation is invalid.");
+    }
+
+    return getEvaluator(info.getParameters());
+  }
+}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java Mon Jul 12 01:21:41 2010
@@ -45,7 +45,7 @@ import org.apache.hadoop.util.StringUtil
  *
  */
 @Description(name = "avg", value = "_FUNC_(x) - Returns the mean of a set of numbers")
-public class GenericUDAFAverage implements GenericUDAFResolver {
+public class GenericUDAFAverage extends AbstractGenericUDAFResolver {
 
   static final Log LOG = LogFactory.getLog(GenericUDAFAverage.class.getName());
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBridge.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBridge.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBridge.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBridge.java Mon Jul 12 01:21:41 2010
@@ -38,7 +38,7 @@ import org.apache.hadoop.util.Reflection
  * This class is a bridge between GenericUDAF and UDAF. Old UDAF can be used
  * with the GenericUDAF infrastructure through this bridge.
  */
-public class GenericUDAFBridge implements GenericUDAFResolver {
+public class GenericUDAFBridge extends AbstractGenericUDAFResolver {
 
   UDAF udaf;
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java Mon Jul 12 01:21:41 2010
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hive.ql.udf.generic;
 
 import org.apache.hadoop.hive.ql.exec.Description;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -30,35 +30,70 @@ import org.apache.hadoop.io.LongWritable
 /**
  * This class implements the COUNT aggregation function as in SQL.
  */
-@Description(name = "count", value = "_FUNC_(x) - Returns the count")
-public class GenericUDAFCount implements GenericUDAFResolver {
+@Description(name = "count",
+    value = "_FUNC_(*) - Returns the total number of retrieved rows, including "
+          +        "rows containing NULL values.\n"
+
+          + "_FUNC_(expr) - Returns the number of rows for which the supplied "
+          +        "expression is non-NULL.\n"
+
+          + "_FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for "
+          +        "which the supplied expression(s) are unique and non-NULL.")
+public class GenericUDAFCount implements GenericUDAFResolver2 {
 
   @Override
-  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
-    if (parameters.length != 1) {
-      throw new UDFArgumentTypeException(parameters.length - 1,
-          "Exactly one argument is expected.");
-    }
+  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+      throws SemanticException {
+    // This method implementation is preserved for backward compatibility.
     return new GenericUDAFCountEvaluator();
   }
 
+  @Override
+  public GenericUDAFEvaluator getEvaluator(GenericUDAFParameterInfo paramInfo)
+  throws SemanticException {
+
+    TypeInfo[] parameters = paramInfo.getParameters();
+
+    if (parameters.length == 0) {
+      if (!paramInfo.isAllColumns()) {
+        throw new UDFArgumentException("Argument expected");
+      }
+      assert !paramInfo.isDistinct() : "DISTINCT not supported with *";
+    } else {
+      if (parameters.length > 1 && !paramInfo.isDistinct()) {
+        throw new UDFArgumentException("DISTINCT keyword must be specified");
+      }
+      assert !paramInfo.isAllColumns() : "* not supported in expression list";
+    }
+
+    return new GenericUDAFCountEvaluator().setCountAllColumns(
+        paramInfo.isAllColumns());
+  }
+
   /**
    * GenericUDAFCountEvaluator.
    *
    */
   public static class GenericUDAFCountEvaluator extends GenericUDAFEvaluator {
-    private ObjectInspector inputOI;
+    private boolean countAllColumns = false;
+    private LongObjectInspector partialCountAggOI;
     private LongWritable result;
 
     @Override
-    public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+    public ObjectInspector init(Mode m, ObjectInspector[] parameters)
+    throws HiveException {
       super.init(m, parameters);
-      assert (parameters.length == 1);
-      inputOI = parameters[0];
+      partialCountAggOI =
+        PrimitiveObjectInspectorFactory.writableLongObjectInspector;
       result = new LongWritable(0);
       return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
     }
 
+    private GenericUDAFCountEvaluator setCountAllColumns(boolean countAllCols) {
+      countAllColumns = countAllCols;
+      return this;
+    }
+
     /** class for storing count value. */
     static class CountAgg implements AggregationBuffer {
       long value;
@@ -66,9 +101,9 @@ public class GenericUDAFCount implements
 
     @Override
     public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-      CountAgg result = new CountAgg();
-      reset(result);
-      return result;
+      CountAgg buffer = new CountAgg();
+      reset(buffer);
+      return buffer;
     }
 
     @Override
@@ -77,17 +112,31 @@ public class GenericUDAFCount implements
     }
 
     @Override
-    public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-      assert (parameters.length == 1);
-      if (parameters[0] != null) {
+    public void iterate(AggregationBuffer agg, Object[] parameters)
+      throws HiveException {
+      if (countAllColumns) {
+        assert parameters.length == 0;
         ((CountAgg) agg).value++;
+      } else {
+        assert parameters.length > 0;
+        boolean countThisRow = true;
+        for (Object nextParam : parameters) {
+          if (nextParam == null) {
+            countThisRow = false;
+            break;
+          }
+        }
+        if (countThisRow) {
+          ((CountAgg) agg).value++;
+        }
       }
     }
 
     @Override
-    public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+    public void merge(AggregationBuffer agg, Object partial)
+      throws HiveException {
       if (partial != null) {
-        long p = ((LongObjectInspector) inputOI).get(partial);
+        long p = partialCountAggOI.get(partial);
         ((CountAgg) agg).value += p;
       }
     }
@@ -102,7 +151,5 @@ public class GenericUDAFCount implements
     public Object terminatePartial(AggregationBuffer agg) throws HiveException {
       return terminate(agg);
     }
-
   }
-
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java Mon Jul 12 01:21:41 2010
@@ -62,7 +62,7 @@ import org.apache.hadoop.util.StringUtil
              + "bin widths. It offers no guarantees in terms of the mean-squared-error of the "
              + "histogram, but in practice is comparable to the histograms produced by the R/S-Plus "
              + "statistical computing packages." )
-public class GenericUDAFHistogramNumeric implements GenericUDAFResolver {
+public class GenericUDAFHistogramNumeric extends AbstractGenericUDAFResolver {
   // class static variables
   static final Log LOG = LogFactory.getLog(GenericUDAFHistogramNumeric.class.getName());
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java Mon Jul 12 01:21:41 2010
@@ -19,25 +19,24 @@ package org.apache.hadoop.hive.ql.udf.ge
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
 @Description(name = "max", value = "_FUNC_(expr) - Returns the maximum value of expr")
-public class GenericUDAFMax implements GenericUDAFResolver {
+public class GenericUDAFMax extends AbstractGenericUDAFResolver {
 
   static final Log LOG = LogFactory.getLog(GenericUDAFMax.class.getName());
 
   @Override
   public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
-      throws SemanticException {
+    throws SemanticException {
     if (parameters.length != 1) {
       throw new UDFArgumentTypeException(parameters.length - 1,
           "Exactly one argument is expected.");

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMin.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMin.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMin.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMin.java Mon Jul 12 01:21:41 2010
@@ -19,25 +19,24 @@ package org.apache.hadoop.hive.ql.udf.ge
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
 @Description(name = "min", value = "_FUNC_(expr) - Returns the minimum value of expr")
-public class GenericUDAFMin implements GenericUDAFResolver {
+public class GenericUDAFMin extends AbstractGenericUDAFResolver {
 
   static final Log LOG = LogFactory.getLog(GenericUDAFMin.class.getName());
 
   @Override
   public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
-      throws SemanticException {
+    throws SemanticException {
     if (parameters.length != 1) {
       throw new UDFArgumentTypeException(parameters.length - 1,
           "Exactly one argument is expected.");

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFParameterInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFParameterInfo.java?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFParameterInfo.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFParameterInfo.java Mon Jul 12 01:21:41 2010
@@ -0,0 +1,72 @@
+/**
+ * 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.udf.generic;
+
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * A callback interface used in conjunction with <tt>GenericUDAFResolver2</tt>
+ * interface that allows for a more extensible and flexible means of
+ * discovering the parameter types provided for UDAF invocation. Apart from
+ * allowing the function implementation to discover the <tt>TypeInfo</tt> of
+ * any types provided in the invocation, this also allows the implementation
+ * to determine if the parameters were qualified using <tt>DISTINCT</tt>. If
+ * no parameters were specified explicitly, it allows the function
+ * implementation to test if the invocation used the wildcard syntax
+ * such as <tt>FUNCTION(*)</tt>.
+ * <p>
+ * <b>Note:</b> The implementation of function does not have to handle the
+ * actual <tt>DISTINCT</tt> or wildcard implementation. This information is
+ * provided only to allow the function implementation to accept or reject
+ * such invocations. For example - the implementation of <tt>COUNT</tt> UDAF
+ * requires that the <tt>DISTINCT</tt> qualifier be supplied when more than
+ * one parameters are specified in the invocation. The actual filtering of
+ * data bound to parameter types for <tt>DISTINCT</tt> implementation is
+ * handled by the framework and not the <tt>COUNT</tt> UDAF implementation.
+ */
+public interface GenericUDAFParameterInfo {
+
+  /**
+   *
+   * @return the parameter type list passed into the UDAF.
+   */
+  TypeInfo[] getParameters();
+
+  /**
+   * Returns <tt>true</tt> if the UDAF invocation was qualified with
+   * <tt>DISTINCT</tt> keyword. Note that this is provided for informational
+   * purposes only and the function implementation is not expected to ensure
+   * the distinct property for the parameter values. That is handled by the
+   * framework.
+   * @return <tt>true</tt> if the UDAF invocation was qualified with
+   * <tt>DISTINCT</tt> keyword, <tt>false</tt> otherwise.
+   */
+  boolean isDistinct();
+
+  /**
+   * Returns <tt>true</tt> if the UDAF invocation was done via the wildcard
+   * syntax <tt>FUNCTION(*)</tt>. Note that this is provided for informational
+   * purposes only and the function implementation is not expected to ensure
+   * the wildcard handling of the target relation. That is handled by the
+   * framework.
+   * @return <tt>true</tt> if the UDAF invocation was done with a wildcard
+   * instead of explicit parameter list.
+   */
+  boolean isAllColumns();
+
+}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver.java Mon Jul 12 01:21:41 2010
@@ -24,28 +24,32 @@ import org.apache.hadoop.hive.serde2.typ
 /**
  * A Generic User-defined aggregation function (GenericUDAF) for the use with
  * Hive.
- * 
+ *
  * GenericUDAFResolver is used at compile time. We use GenericUDAFResolver to
  * find out the GenericUDAFEvaluator for the parameter types.
- * 
+ *
+ * @deprecated Use {@link GenericUDAFResolver2} instead.
  */
+@Deprecated
 public interface GenericUDAFResolver {
 
   /**
    * Get the evaluator for the parameter types.
-   * 
+   *
    * The reason that this function returns an object instead of a class is
-   * because it's possible that the object needs some configuration (that can be
-   * serialized). In that case the class of the object has to implement the
+   * because it is possible that the object needs some configuration (that can
+   * be serialized). In that case the class of the object has to implement the
    * Serializable interface. At execution time, we will deserialize the object
    * from the plan and use it to evaluate the aggregations.
-   * 
+   * <p>
    * If the class of the object does not implement Serializable, then we will
    * create a new instance of the class at execution time.
-   * 
+   * </p>
    * @param parameters
    *          The types of the parameters. We need the type information to know
    *          which evaluator class to use.
+   * @throws SemanticException
    */
-  GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException;
+  GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+    throws SemanticException;
 }

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver2.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver2.java?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver2.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFResolver2.java Mon Jul 12 01:21:41 2010
@@ -0,0 +1,62 @@
+/**
+ * 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.udf.generic;
+
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * This interface extends the <tt>GenericUDAFResolver</tt> interface and
+ * provides more flexibility in terms of discovering the parameter types
+ * supplied to the UDAF. Implementations that extend this interface will
+ * also have access to extra information such as the specification of the
+ * <tt>DISTINCT</tt> qualifier or the invocation with the special wildcard
+ * character.
+ * <p>
+ * <b>Note:</b> The implementation of function does not have to handle the
+ * actual <tt>DISTINCT</tt> or wildcard implementation. This information is
+ * provided only to allow the function implementation to accept or reject
+ * such invocations. For example - the implementation of <tt>COUNT</tt> UDAF
+ * requires that the <tt>DISTINCT</tt> qualifier be supplied when more than
+ * one parameters are specified in the invocation. The actual filtering of
+ * data bound to parameter types for <tt>DISTINCT</tt> implementation is
+ * handled by the framework and not the <tt>COUNT</tt> UDAF implementation.
+ */
+@SuppressWarnings("deprecation")
+public interface GenericUDAFResolver2 extends GenericUDAFResolver {
+
+  /**
+   * Get the evaluator for the parameter types.
+   *
+   * The reason that this function returns an object instead of a class is
+   * because it is possible that the object needs some configuration (that can
+   * be serialized). In that case the class of the object has to implement the
+   * Serializable interface. At execution time, we will deserialize the object
+   * from the plan and use it to evaluate the aggregations.
+   * <p>
+   * If the class of the object does not implement Serializable, then we will
+   * create a new instance of the class at execution time.
+   * </p>
+   *
+   * @param info The parameter information that is applicable to the UDAF being
+   *          invoked.
+   * @throws SemanticException
+   */
+  GenericUDAFEvaluator getEvaluator(
+      GenericUDAFParameterInfo info) throws SemanticException;
+}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java Mon Jul 12 01:21:41 2010
@@ -38,12 +38,13 @@ import org.apache.hadoop.util.StringUtil
  *
  */
 @Description(name = "sum", value = "_FUNC_(x) - Returns the sum of a set of numbers")
-public class GenericUDAFSum implements GenericUDAFResolver {
+public class GenericUDAFSum extends AbstractGenericUDAFResolver {
 
   static final Log LOG = LogFactory.getLog(GenericUDAFSum.class.getName());
 
   @Override
-  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+    throws SemanticException {
     if (parameters.length != 1) {
       throw new UDFArgumentTypeException(parameters.length - 1,
           "Exactly one argument is expected.");

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFVariance.java Mon Jul 12 01:21:41 2010
@@ -47,7 +47,7 @@ import org.apache.hadoop.util.StringUtil
  */
 @Description(name = "variance,var_pop",
     value = "_FUNC_(x) - Returns the variance of a set of numbers")
-public class GenericUDAFVariance implements GenericUDAFResolver {
+public class GenericUDAFVariance extends AbstractGenericUDAFResolver {
 
   static final Log LOG = LogFactory.getLog(GenericUDAFVariance.class.getName());
 

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/SimpleGenericUDAFParameterInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/SimpleGenericUDAFParameterInfo.java?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/SimpleGenericUDAFParameterInfo.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/SimpleGenericUDAFParameterInfo.java Mon Jul 12 01:21:41 2010
@@ -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.udf.generic;
+
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * A simple implementation of <tt>GenericUDAFParameterInfo</tt>.
+ *
+ */
+public class SimpleGenericUDAFParameterInfo implements GenericUDAFParameterInfo
+{
+
+  private final TypeInfo[] parameters;
+  private final boolean distinct;
+  private final boolean allColumns;
+
+  public SimpleGenericUDAFParameterInfo(TypeInfo[] params, boolean distinct,
+      boolean allColumns) {
+    this.parameters = params;
+    this.distinct = distinct;
+    this.allColumns = allColumns;
+  }
+
+  @Override
+  public TypeInfo[] getParameters() {
+    return parameters;
+  }
+
+  @Override
+  public boolean isDistinct() {
+    return distinct;
+  }
+
+  @Override
+  public boolean isAllColumns() {
+    return allColumns;
+  }
+}

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_avg_syntax.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_avg_syntax.q?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_avg_syntax.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_avg_syntax.q Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+SELECT avg(*) FROM src;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_max_syntax.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_max_syntax.q?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_max_syntax.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_max_syntax.q Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+SELECT max(*) FROM src;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_min_syntax.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_min_syntax.q?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_min_syntax.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_min_syntax.q Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+SELECT min(*) FROM src;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_std_syntax.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_std_syntax.q?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_std_syntax.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_std_syntax.q Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+SELECT std(*) FROM src;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+SELECT stddev_samp(*) FROM src;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_sum_syntax.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_sum_syntax.q?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_sum_syntax.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_sum_syntax.q Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+SELECT sum(*) FROM src;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+SELECT var_samp(*) FROM src;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_variance_syntax.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_variance_syntax.q?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_variance_syntax.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/invalid_variance_syntax.q Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+SELECT variance(*) FROM src;

Modified: hadoop/hive/trunk/ql/src/test/queries/clientpositive/udf_count.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/udf_count.q?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/udf_count.q (original)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/udf_count.q Mon Jul 12 01:21:41 2010
@@ -1,2 +1,17 @@
 DESCRIBE FUNCTION count;
 DESCRIBE FUNCTION EXTENDED count;
+
+EXPLAIN SELECT count(key) FROM src;
+SELECT count(key) FROM src;
+
+EXPLAIN SELECT count(DISTINCT key) FROM src;
+SELECT count(DISTINCT key) FROM src;
+
+EXPLAIN SELECT count(DISTINCT key, value) FROM src;
+SELECT count(DISTINCT key, value) FROM src;
+
+EXPLAIN SELECT count(*) FROM src;
+SELECT count(*) FROM src;
+
+EXPLAIN SELECT count(1) FROM src;
+SELECT count(1) FROM src;

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_avg_syntax.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_avg_syntax.q.out?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_avg_syntax.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_avg_syntax.q.out Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: The specified syntax for UDAF invocation is invalid.

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_max_syntax.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_max_syntax.q.out?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_max_syntax.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_max_syntax.q.out Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: The specified syntax for UDAF invocation is invalid.

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_min_syntax.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_min_syntax.q.out?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_min_syntax.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_min_syntax.q.out Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: The specified syntax for UDAF invocation is invalid.

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_std_syntax.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_std_syntax.q.out?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_std_syntax.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_std_syntax.q.out Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: The specified syntax for UDAF invocation is invalid.

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_stddev_samp_syntax.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_stddev_samp_syntax.q.out?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_stddev_samp_syntax.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_stddev_samp_syntax.q.out Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: The specified syntax for UDAF invocation is invalid.

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_sum_syntax.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_sum_syntax.q.out?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_sum_syntax.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_sum_syntax.q.out Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: The specified syntax for UDAF invocation is invalid.

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_var_samp_syntax.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_var_samp_syntax.q.out?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_var_samp_syntax.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_var_samp_syntax.q.out Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: The specified syntax for UDAF invocation is invalid.

Added: hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_variance_syntax.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_variance_syntax.q.out?rev=963173&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_variance_syntax.q.out (added)
+++ hadoop/hive/trunk/ql/src/test/results/clientnegative/invalid_variance_syntax.q.out Mon Jul 12 01:21:41 2010
@@ -0,0 +1 @@
+FAILED: Error in semantic analysis: The specified syntax for UDAF invocation is invalid.

Modified: hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_count.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_count.q.out?rev=963173&r1=963172&r2=963173&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_count.q.out (original)
+++ hadoop/hive/trunk/ql/src/test/results/clientpositive/udf_count.q.out Mon Jul 12 01:21:41 2010
@@ -2,9 +2,363 @@ PREHOOK: query: DESCRIBE FUNCTION count
 PREHOOK: type: DESCFUNCTION
 POSTHOOK: query: DESCRIBE FUNCTION count
 POSTHOOK: type: DESCFUNCTION
-count(x) - Returns the count
+count(*) - Returns the total number of retrieved rows, including rows containing NULL values.
+count(expr) - Returns the number of rows for which the supplied expression is non-NULL.
+count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.
 PREHOOK: query: DESCRIBE FUNCTION EXTENDED count
 PREHOOK: type: DESCFUNCTION
 POSTHOOK: query: DESCRIBE FUNCTION EXTENDED count
 POSTHOOK: type: DESCFUNCTION
-count(x) - Returns the count
+count(*) - Returns the total number of retrieved rows, including rows containing NULL values.
+count(expr) - Returns the number of rows for which the supplied expression is non-NULL.
+count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.
+PREHOOK: query: EXPLAIN SELECT count(key) FROM src
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT count(key) FROM src
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF src)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION count (TOK_TABLE_OR_COL key))))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        src 
+          TableScan
+            alias: src
+            Select Operator
+              expressions:
+                    expr: key
+                    type: string
+              outputColumnNames: key
+              Group By Operator
+                aggregations:
+                      expr: count(key)
+                bucketGroup: false
+                mode: hash
+                outputColumnNames: _col0
+                Reduce Output Operator
+                  sort order: 
+                  tag: -1
+                  value expressions:
+                        expr: _col0
+                        type: bigint
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: SELECT count(key) FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-26_584_2445455362948160825/10000
+POSTHOOK: query: SELECT count(key) FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-26_584_2445455362948160825/10000
+500
+PREHOOK: query: EXPLAIN SELECT count(DISTINCT key) FROM src
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT count(DISTINCT key) FROM src
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF src)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONDI count (TOK_TABLE_OR_COL key))))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        src 
+          TableScan
+            alias: src
+            Select Operator
+              expressions:
+                    expr: key
+                    type: string
+              outputColumnNames: key
+              Group By Operator
+                aggregations:
+                      expr: count(DISTINCT key)
+                bucketGroup: false
+                keys:
+                      expr: key
+                      type: string
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions:
+                        expr: _col0
+                        type: string
+                  sort order: +
+                  tag: -1
+                  value expressions:
+                        expr: _col1
+                        type: bigint
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(DISTINCT KEY._col0)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: SELECT count(DISTINCT key) FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-31_832_6294236640527541514/10000
+POSTHOOK: query: SELECT count(DISTINCT key) FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-31_832_6294236640527541514/10000
+309
+PREHOOK: query: EXPLAIN SELECT count(DISTINCT key, value) FROM src
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT count(DISTINCT key, value) FROM src
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF src)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONDI count (TOK_TABLE_OR_COL key) (TOK_TABLE_OR_COL value))))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        src 
+          TableScan
+            alias: src
+            Select Operator
+              expressions:
+                    expr: key
+                    type: string
+                    expr: value
+                    type: string
+              outputColumnNames: key, value
+              Group By Operator
+                aggregations:
+                      expr: count(DISTINCT key, value)
+                bucketGroup: false
+                keys:
+                      expr: key
+                      type: string
+                      expr: value
+                      type: string
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
+                Reduce Output Operator
+                  key expressions:
+                        expr: _col0
+                        type: string
+                        expr: _col1
+                        type: string
+                  sort order: ++
+                  tag: -1
+                  value expressions:
+                        expr: _col2
+                        type: bigint
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(DISTINCT KEY._col0, KEY._col1)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: SELECT count(DISTINCT key, value) FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-35_826_4104856806164432180/10000
+POSTHOOK: query: SELECT count(DISTINCT key, value) FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-35_826_4104856806164432180/10000
+309
+PREHOOK: query: EXPLAIN SELECT count(*) FROM src
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT count(*) FROM src
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF src)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count)))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        src 
+          TableScan
+            alias: src
+            Select Operator
+              Group By Operator
+                aggregations:
+                      expr: count()
+                bucketGroup: false
+                mode: hash
+                outputColumnNames: _col0
+                Reduce Output Operator
+                  sort order: 
+                  tag: -1
+                  value expressions:
+                        expr: _col0
+                        type: bigint
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: SELECT count(*) FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-40_398_2344399307637124134/10000
+POSTHOOK: query: SELECT count(*) FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-40_398_2344399307637124134/10000
+500
+PREHOOK: query: EXPLAIN SELECT count(1) FROM src
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT count(1) FROM src
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF src)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION count 1)))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        src 
+          TableScan
+            alias: src
+            Select Operator
+              Group By Operator
+                aggregations:
+                      expr: count(1)
+                bucketGroup: false
+                mode: hash
+                outputColumnNames: _col0
+                Reduce Output Operator
+                  sort order: 
+                  tag: -1
+                  value expressions:
+                        expr: _col0
+                        type: bigint
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: SELECT count(1) FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-45_028_714373071146042914/10000
+POSTHOOK: query: SELECT count(1) FROM src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: file:/var/folders/rF/rFg7A9swER0pyf9VBov+VU+++TM/-Tmp-/arvind/hive_2010-07-06_14-12-45_028_714373071146042914/10000
+500