You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/01/08 02:08:25 UTC

[2/6] calcite git commit: [CALCITE-1039] Assign a SqlKind value for each built-in aggregate function

[CALCITE-1039] Assign a SqlKind value for each built-in aggregate function

Also, deprecate getParameterTypes and getReturnType methods in several aggregate functions.


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

Branch: refs/heads/master
Commit: 8531ab11715109716215c0006d28014754e566f9
Parents: 808d540
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 4 02:47:17 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jan 5 15:16:35 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     | 22 +----
 .../org/apache/calcite/sql/SqlAggFunction.java  | 14 +++
 .../java/org/apache/calcite/sql/SqlKind.java    | 97 ++++++++++++++++++--
 .../org/apache/calcite/sql/SqlRankFunction.java | 17 ----
 .../calcite/sql/fun/SqlAvgAggFunction.java      | 48 ++++------
 .../calcite/sql/fun/SqlCountAggFunction.java    |  2 +-
 .../calcite/sql/fun/SqlCovarAggFunction.java    | 38 +++-----
 .../sql/fun/SqlFirstLastValueAggFunction.java   | 14 ++-
 .../sql/fun/SqlHistogramAggFunction.java        | 10 +-
 .../calcite/sql/fun/SqlLeadLagAggFunction.java  | 23 +++--
 .../calcite/sql/fun/SqlMinMaxAggFunction.java   | 49 ++++++----
 .../calcite/sql/fun/SqlNtileAggFunction.java    | 13 +--
 .../sql/fun/SqlSingleValueAggFunction.java      |  4 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    | 52 +++++------
 .../calcite/sql/fun/SqlSumAggFunction.java      |  4 +-
 .../sql/fun/SqlSumEmptyIsZeroAggFunction.java   |  2 +-
 16 files changed, 229 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index a7b3e08..e658749 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -64,11 +64,9 @@ import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.MultisetSqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -376,18 +374,9 @@ public abstract class RelOptUtil {
       // doing a min(TRUE)
 
       ret = createProject(ret, ImmutableList.of(extraExpr), null);
-      final List<RelDataType> argTypes =
-          ImmutableList.of(
-              typeFactory.createSqlType(SqlTypeName.BOOLEAN));
-
-      SqlAggFunction minFunction =
-          new SqlMinMaxAggFunction(
-              argTypes,
-              true,
-              SqlMinMaxAggFunction.MINMAX_COMPARABLE);
 
       final AggregateCall aggCall =
-          AggregateCall.create(minFunction,
+          AggregateCall.create(SqlStdOperatorTable.MIN,
               false,
               ImmutableList.of(0),
               -1,
@@ -457,15 +446,8 @@ public abstract class RelOptUtil {
 
       ret = createProject(ret, exprs, null);
 
-      final List<RelDataType> argTypes =
-          ImmutableList.of(typeFactory.createSqlType(SqlTypeName.BOOLEAN));
-
-      final SqlAggFunction minFunction =
-          new SqlMinMaxAggFunction(argTypes, true,
-              SqlMinMaxAggFunction.MINMAX_COMPARABLE);
-
       final AggregateCall aggCall =
-          AggregateCall.create(minFunction,
+          AggregateCall.create(SqlStdOperatorTable.MIN,
               false,
               ImmutableList.of(projectedKeyCount),
               -1,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
index 83fe248..d58987d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
@@ -17,12 +17,16 @@
 package org.apache.calcite.sql;
 
 import org.apache.calcite.plan.Context;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
+import java.util.List;
+
 /**
  * Abstract base class for the definition of an aggregate function: an operator
  * which aggregates sets of values into a result.
@@ -119,6 +123,16 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
   public final boolean requiresOver() {
     return requiresOver;
   }
+
+  @Deprecated // to be removed before 2.0
+  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before calcite-2.0");
+  }
+
+  @Deprecated // to be removed before 2.0
+  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before calcite-2.0");
+  }
 }
 
 // End SqlAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 5a8a2fa..78a9d7d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -625,6 +625,74 @@ public enum SqlKind {
   /** The internal {@code GROUP_ID()} function. */
   GROUP_ID,
 
+  // Aggregate functions
+
+  /** The {@code COUNT} aggregate function. */
+  COUNT,
+
+  /** The {@code SUM} aggregate function. */
+  SUM,
+
+  /** The {@code SUM0} aggregate function. */
+  SUM0,
+
+  /** The {@code MIN} aggregate function. */
+  MIN,
+
+  /** The {@code MAX} aggregate function. */
+  MAX,
+
+  /** The {@code LEAD} aggregate function. */
+  LEAD,
+
+  /** The {@code LAG} aggregate function. */
+  LAG,
+
+  /** The {@code FIRST_VALUE} aggregate function. */
+  FIRST_VALUE,
+
+  /** The {@code LAST_VALUE} aggregate function. */
+  LAST_VALUE,
+
+  /** The {@code COVAR_POP} aggregate function. */
+  COVAR_POP,
+
+  /** The {@code COVAR_SAMP} aggregate function. */
+  COVAR_SAMP,
+
+  /** The {@code REGR_SXX} aggregate function. */
+  REGR_SXX,
+
+  /** The {@code REGR_SYY} aggregate function. */
+  REGR_SYY,
+
+  /** The {@code AVG} aggregate function. */
+  AVG,
+
+  /** The {@code STDDEV_POP} aggregate function. */
+  STDDEV_POP,
+
+  /** The {@code STDDEV_SAMP} aggregate function. */
+  STDDEV_SAMP,
+
+  /** The {@code VAR_POP} aggregate function. */
+  VAR_POP,
+
+  /** The {@code VAR_SAMP} aggregate function. */
+  VAR_SAMP,
+
+  /** The {@code NTILE} aggregate function. */
+  NTILE,
+
+  /** The {@code COLLECT} aggregate function. */
+  COLLECT,
+
+  /** The {@code FUSION} aggregate function. */
+  FUSION,
+
+  /** The {@code SINGLE_VALUE} aggregate function. */
+  SINGLE_VALUE,
+
   // DDL and session control statements follow. The list is not exhaustive: feel
   // free to add more.
 
@@ -699,10 +767,21 @@ public enum SqlKind {
       EnumSet.of(UNION, INTERSECT, EXCEPT);
 
   /**
+   * Category consisting of all built-in aggregate functions.
+   */
+  public static final EnumSet<SqlKind> AGGREGATE =
+      EnumSet.of(COUNT, SUM, SUM0, MIN, MAX, LEAD, LAG, FIRST_VALUE,
+          LAST_VALUE, COVAR_POP, COVAR_SAMP, REGR_SXX, REGR_SYY,
+          AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP, NTILE, COLLECT,
+          FUSION, SINGLE_VALUE);
+
+  /**
    * Category consisting of all expression operators.
    *
    * <p>A node is an expression if it is NOT one of the following:
    * {@link #AS},
+   * {@link #ARGUMENT_ASSIGNMENT},
+   * {@link #DEFAULT},
    * {@link #DESCENDING},
    * {@link #SELECT},
    * {@link #JOIN},
@@ -715,17 +794,19 @@ public enum SqlKind {
    * {@link #FOLLOWING},
    * {@link #ORDER_BY},
    * {@link #COLLECTION_TABLE},
-   * {@link #TABLESAMPLE}.
+   * {@link #TABLESAMPLE},
+   * or an aggregate function.
    */
   public static final Set<SqlKind> EXPRESSION =
       EnumSet.complementOf(
-          EnumSet.of(
-              AS, ARGUMENT_ASSIGNMENT, DEFAULT,
-              DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND,
-              SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM, FLOOR, CEIL,
-              LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
-              NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
-              VALUES, WITH, WITH_ITEM));
+          concat(
+              EnumSet.of(AS, ARGUMENT_ASSIGNMENT, DEFAULT,
+                  DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND,
+                  SELECT, JOIN, OTHER_FUNCTION, CAST, TRIM, FLOOR, CEIL,
+                  LITERAL_CHAIN, JDBC_FN, PRECEDING, FOLLOWING, ORDER_BY,
+                  NULLS_FIRST, NULLS_LAST, COLLECTION_TABLE, TABLESAMPLE,
+                  VALUES, WITH, WITH_ITEM),
+              AGGREGATE));
 
   /**
    * Category consisting of all DML operators.

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
index e2a2490..738add1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
@@ -16,23 +16,13 @@
  */
 package org.apache.calcite.sql;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
 /**
  * Operator which aggregates sets of values into a result.
  */
 public class SqlRankFunction extends SqlAggFunction {
-  //~ Instance fields --------------------------------------------------------
-
-  private final RelDataType type = null;
-
   //~ Constructors -----------------------------------------------------------
 
   public SqlRankFunction(String name, boolean requiresOrder) {
@@ -54,13 +44,6 @@ public class SqlRankFunction extends SqlAggFunction {
     return false;
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
-  }
 }
 
 // End SqlRankFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
index b86d49c..95f8049 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlAvgAggFunction.java
@@ -17,16 +17,13 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
+import com.google.common.base.Preconditions;
 
 /**
  * <code>Avg</code> is an aggregator which returns the average of the values
@@ -35,56 +32,49 @@ import java.util.List;
  * double</code>), and the result is the same type.
  */
 public class SqlAvgAggFunction extends SqlAggFunction {
-  //~ Instance fields --------------------------------------------------------
-
-  private final RelDataType type;
-  private final Subtype subtype;
-
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a SqlAvgAggFunction
-   *
-   * @param type    Data type
-   * @param subtype Specific function, e.g. AVG or STDDEV_POP
+   * Creates a SqlAvgAggFunction.
    */
-  public SqlAvgAggFunction(
-      RelDataType type,
-      Subtype subtype) {
-    super(
-        subtype.name(),
+  public SqlAvgAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC,
         SqlFunctionCategory.NUMERIC,
         false,
         false);
-    this.type = type;
-    this.subtype = subtype;
+    Preconditions.checkArgument(kind == SqlKind.AVG
+        || kind == SqlKind.STDDEV_POP
+        || kind == SqlKind.STDDEV_SAMP
+        || kind == SqlKind.VAR_POP
+        || kind == SqlKind.VAR_SAMP);
   }
 
-  //~ Methods ----------------------------------------------------------------
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
+  @Deprecated // to be removed before 2.0
+  public SqlAvgAggFunction(
+      RelDataType type,
+      Subtype subtype) {
+    this(SqlKind.valueOf(subtype.name()));
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
+    //~ Methods ----------------------------------------------------------------
 
   /**
    * Returns the specific function, e.g. AVG or STDDEV_POP.
    *
    * @return Subtype
    */
+  @Deprecated // to be removed before 2.0
   public Subtype getSubtype() {
-    return subtype;
+    return Subtype.valueOf(kind.name());
   }
 
   /** Sub-type of aggregate function. */
+  @Deprecated // to be removed before 2.0
   public enum Subtype {
     AVG,
     STDDEV_POP,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
index e91f30b..dac4b21 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
@@ -48,7 +48,7 @@ public class SqlCountAggFunction extends SqlAggFunction {
     super(
         "COUNT",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.COUNT,
         ReturnTypes.BIGINT,
         null,
         SqlValidator.STRICT

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
index df36095..ea23300 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCovarAggFunction.java
@@ -17,16 +17,13 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
+import com.google.common.base.Preconditions;
 
 /**
  * <code>Covar</code> is an aggregator which returns the Covariance of the
@@ -37,53 +34,48 @@ import java.util.List;
 public class SqlCovarAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
-  private final RelDataType type;
-  private final Subtype subtype;
-
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Creates a SqlCovarAggFunction.
-   *
-   * @param type    Data type
-   * @param subtype Specific function, e.g. COVAR_POP
    */
-  public SqlCovarAggFunction(RelDataType type, Subtype subtype) {
-    super(subtype.name(),
+  public SqlCovarAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC_NUMERIC,
         SqlFunctionCategory.NUMERIC,
         false,
         false);
-    this.type = type;
-    this.subtype = subtype;
+    Preconditions.checkArgument(kind == SqlKind.COVAR_POP
+        || kind == SqlKind.COVAR_SAMP
+        || kind == SqlKind.REGR_SXX
+        || kind == SqlKind.REGR_SYY);
   }
 
-  //~ Methods ----------------------------------------------------------------
-
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    return ImmutableList.of(type);
+  @Deprecated // to be removed before 2.0
+  public SqlCovarAggFunction(RelDataType type, Subtype subtype) {
+    this(SqlKind.valueOf(subtype.name()));
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return type;
-  }
+  //~ Methods ----------------------------------------------------------------
 
   /**
    * Returns the specific function, e.g. COVAR_POP or COVAR_SAMP.
    *
    * @return Subtype
    */
+  @Deprecated // to be removed before 2.0
   public Subtype getSubtype() {
-    return subtype;
+    return Subtype.valueOf(kind.name());
   }
 
   /**
    * Enum for defining specific types.
    */
+  @Deprecated // to be removed before 2.0
   public enum Subtype {
     COVAR_POP,
     COVAR_SAMP,

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
index bc75424..edc6499 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlFirstLastValueAggFunction.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -37,17 +38,24 @@ import java.util.List;
 public class SqlFirstLastValueAggFunction extends SqlAggFunction {
   //~ Constructors -----------------------------------------------------------
 
-  public SqlFirstLastValueAggFunction(boolean firstFlag) {
+  public SqlFirstLastValueAggFunction(SqlKind kind) {
     super(
-        firstFlag ? "FIRST_VALUE" : "LAST_VALUE",
+        kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.ANY,
         SqlFunctionCategory.NUMERIC,
         false,
         true);
+    Preconditions.checkArgument(kind == SqlKind.FIRST_VALUE
+        || kind == SqlKind.LAST_VALUE);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public SqlFirstLastValueAggFunction(boolean firstFlag) {
+    this(firstFlag ? SqlKind.FIRST_VALUE : SqlKind.LAST_VALUE);
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
index 8572338..1ba3821 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlHistogramAggFunction.java
@@ -29,15 +29,16 @@ import com.google.common.collect.ImmutableList;
 import java.util.List;
 
 /**
- * <code>HistogramAgg</code> is base operator which supports the Histogram
- * MIN/MAX aggregatoin functions. which returns the sum of the values which go
+ * <code>HISTOGRAM</code> is the base operator that supports the Histogram
+ * MIN/MAX aggregate functions. It returns the sum of the values which go
  * into it. It has precisely one argument of numeric type (<code>int</code>,
- * <code>long</code>, <code>float</code>, <code>double</code>) results are
- * retrived with (<code>HistogramMin</code>) and (<code>HistogramMax</code>)
+ * <code>long</code>, <code>float</code>, <code>double</code>); results are
+ * retrieved using (<code>HistogramMin</code>) and (<code>HistogramMax</code>).
  */
 public class SqlHistogramAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -62,6 +63,7 @@ public class SqlHistogramAggFunction extends SqlAggFunction {
     return ImmutableList.of(type);
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
index a2b0c45..5e5498b 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLeadLagAggFunction.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
@@ -31,6 +30,7 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeTransform;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -73,30 +73,29 @@ public class SqlLeadLagAggFunction extends SqlAggFunction {
         }
       });
 
-  public SqlLeadLagAggFunction(boolean isLead) {
-    super(
-        isLead ? "LEAD" : "LAG",
+  public SqlLeadLagAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         RETURN_TYPE,
         null,
         OPERAND_TYPES,
         SqlFunctionCategory.NUMERIC,
         false,
         true);
+    Preconditions.checkArgument(kind == SqlKind.LEAD
+        || kind == SqlKind.LAG);
   }
 
-  @Override public boolean allowsFraming() {
-    return false;
+  @Deprecated // to be removed before 2.0
+  public SqlLeadLagAggFunction(boolean isLead) {
+    this(isLead ? SqlKind.LEAD : SqlKind.LAG);
   }
 
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
+  @Override public boolean allowsFraming() {
+    return false;
   }
 
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
 }
 
 // End SqlLeadLagAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
index 27a1a8b..391c09a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
@@ -26,6 +26,9 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 
 /**
@@ -57,62 +60,72 @@ public class SqlMinMaxAggFunction extends SqlAggFunction {
 
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public final List<RelDataType> argTypes;
-  private final boolean isMin;
-  private final int kind;
+  private final int minMaxKind;
 
   //~ Constructors -----------------------------------------------------------
 
-  public SqlMinMaxAggFunction(
-      List<RelDataType> argTypes,
-      boolean isMin,
-      int kind) {
-    super(
-        isMin ? "MIN" : "MAX",
+  /** Creates a SqlMinMaxAggFunction. */
+  public SqlMinMaxAggFunction(SqlKind kind) {
+    super(kind.name(),
         null,
-        SqlKind.OTHER_FUNCTION,
+        kind,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.COMPARABLE_ORDERED,
         SqlFunctionCategory.SYSTEM,
         false,
         false);
-    this.argTypes = argTypes;
-    this.isMin = isMin;
-    this.kind = kind;
+    this.argTypes = ImmutableList.of();
+    this.minMaxKind = MINMAX_COMPARABLE;
+    Preconditions.checkArgument(kind == SqlKind.MIN
+        || kind == SqlKind.MAX);
+  }
+
+  @Deprecated // to be removed before 2.0
+  public SqlMinMaxAggFunction(
+      List<RelDataType> argTypes,
+      boolean isMin,
+      int minMaxKind) {
+    this(isMin ? SqlKind.MIN : SqlKind.MAX);
+    assert argTypes.isEmpty();
+    assert minMaxKind == MINMAX_COMPARABLE;
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   public boolean isMin() {
-    return isMin;
+    return kind == SqlKind.MIN;
   }
 
+  @Deprecated // to be removed before 2.0
   public int getMinMaxKind() {
-    return kind;
+    return minMaxKind;
   }
 
   public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    switch (kind) {
+    switch (minMaxKind) {
     case MINMAX_PRIMITIVE:
     case MINMAX_COMPARABLE:
       return argTypes;
     case MINMAX_COMPARATOR:
       return argTypes.subList(1, 2);
     default:
-      throw Util.newInternal("bad kind: " + kind);
+      throw Util.newInternal("bad kind: " + minMaxKind);
     }
   }
 
   public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    switch (kind) {
+    switch (minMaxKind) {
     case MINMAX_PRIMITIVE:
     case MINMAX_COMPARABLE:
       return argTypes.get(0);
     case MINMAX_COMPARATOR:
       return argTypes.get(1);
     default:
-      throw Util.newInternal("bad kind: " + kind);
+      throw Util.newInternal("bad kind: " + minMaxKind);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
index 524e7d6..55eb8cb 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlNtileAggFunction.java
@@ -16,16 +16,12 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import java.util.List;
-
 /**
  * <code>NTILE</code> aggregate function
  * return the value of given expression evaluated at given offset.
@@ -35,7 +31,7 @@ public class SqlNtileAggFunction extends SqlAggFunction {
     super(
         "NTILE",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.NTILE,
         ReturnTypes.INTEGER,
         null,
         OperandTypes.POSITIVE_INTEGER_LITERAL,
@@ -44,13 +40,6 @@ public class SqlNtileAggFunction extends SqlAggFunction {
         true);
   }
 
-  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
-
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    throw new UnsupportedOperationException("remove before calcite-0.9");
-  }
 }
 
 // End SqlNtileAggFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
index a27e180..c0c977c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
@@ -35,6 +35,7 @@ import java.util.List;
 public class SqlSingleValueAggFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -44,7 +45,7 @@ public class SqlSingleValueAggFunction extends SqlAggFunction {
     super(
         "SINGLE_VALUE",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SINGLE_VALUE,
         ReturnTypes.ARG0,
         null,
         OperandTypes.ANY,
@@ -64,6 +65,7 @@ public class SqlSingleValueAggFunction extends SqlAggFunction {
     return type;
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index fc524c6..3311f50 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -16,7 +16,6 @@
  */
 package org.apache.calcite.sql.fun;
 
-import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlAsOperator;
 import org.apache.calcite.sql.SqlBinaryOperator;
@@ -51,8 +50,6 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlModality;
 
-import com.google.common.collect.ImmutableList;
-
 /**
  * Implementation of {@link org.apache.calcite.sql.SqlOperatorTable} containing
  * the standard operators and functions.
@@ -688,43 +685,37 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <code>MIN</code> aggregate function.
    */
   public static final SqlAggFunction MIN =
-      new SqlMinMaxAggFunction(
-          ImmutableList.<RelDataType>of(),
-          true,
-          SqlMinMaxAggFunction.MINMAX_COMPARABLE);
+      new SqlMinMaxAggFunction(SqlKind.MIN);
 
   /**
    * <code>MAX</code> aggregate function.
    */
   public static final SqlAggFunction MAX =
-      new SqlMinMaxAggFunction(
-          ImmutableList.<RelDataType>of(),
-          false,
-          SqlMinMaxAggFunction.MINMAX_COMPARABLE);
+      new SqlMinMaxAggFunction(SqlKind.MAX);
 
   /**
    * <code>LAST_VALUE</code> aggregate function.
    */
   public static final SqlAggFunction LAST_VALUE =
-      new SqlFirstLastValueAggFunction(false);
+      new SqlFirstLastValueAggFunction(SqlKind.LAST_VALUE);
 
   /**
    * <code>FIRST_VALUE</code> aggregate function.
    */
   public static final SqlAggFunction FIRST_VALUE =
-      new SqlFirstLastValueAggFunction(true);
+      new SqlFirstLastValueAggFunction(SqlKind.FIRST_VALUE);
 
   /**
    * <code>LEAD</code> aggregate function.
    */
   public static final SqlAggFunction LEAD =
-      new SqlLeadLagAggFunction(true);
+      new SqlLeadLagAggFunction(SqlKind.LEAD);
 
   /**
    * <code>LAG</code> aggregate function.
    */
   public static final SqlAggFunction LAG =
-      new SqlLeadLagAggFunction(false);
+      new SqlLeadLagAggFunction(SqlKind.LAG);
 
   /**
    * <code>NTILE</code> aggregate function.
@@ -742,55 +733,55 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <code>AVG</code> aggregate function.
    */
   public static final SqlAggFunction AVG =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.AVG);
+      new SqlAvgAggFunction(SqlKind.AVG);
 
   /**
    * <code>STDDEV_POP</code> aggregate function.
    */
   public static final SqlAggFunction STDDEV_POP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.STDDEV_POP);
+      new SqlAvgAggFunction(SqlKind.STDDEV_POP);
 
   /**
    * <code>REGR_SXX</code> aggregate function.
    */
   public static final SqlAggFunction REGR_SXX =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.REGR_SXX);
+      new SqlCovarAggFunction(SqlKind.REGR_SXX);
 
   /**
    * <code>REGR_SYY</code> aggregate function.
    */
   public static final SqlAggFunction REGR_SYY =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.REGR_SYY);
+      new SqlCovarAggFunction(SqlKind.REGR_SYY);
 
   /**
    * <code>COVAR_POP</code> aggregate function.
    */
   public static final SqlAggFunction COVAR_POP =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.COVAR_POP);
+      new SqlCovarAggFunction(SqlKind.COVAR_POP);
 
   /**
    * <code>COVAR_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction COVAR_SAMP =
-      new SqlCovarAggFunction(null, SqlCovarAggFunction.Subtype.COVAR_SAMP);
+      new SqlCovarAggFunction(SqlKind.COVAR_SAMP);
 
   /**
    * <code>STDDEV_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction STDDEV_SAMP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.STDDEV_SAMP);
+      new SqlAvgAggFunction(SqlKind.STDDEV_SAMP);
 
   /**
    * <code>VAR_POP</code> aggregate function.
    */
   public static final SqlAggFunction VAR_POP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.VAR_POP);
+      new SqlAvgAggFunction(SqlKind.VAR_POP);
 
   /**
    * <code>VAR_SAMP</code> aggregate function.
    */
   public static final SqlAggFunction VAR_SAMP =
-      new SqlAvgAggFunction(null, SqlAvgAggFunction.Subtype.VAR_SAMP);
+      new SqlAvgAggFunction(SqlKind.VAR_SAMP);
 
   //-------------------------------------------------------------
   // WINDOW Aggregate Functions
@@ -1509,24 +1500,25 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    */
   public static final SqlAggFunction COLLECT =
       new SqlAggFunction("COLLECT",
-          SqlKind.OTHER_FUNCTION,
+          null,
+          SqlKind.COLLECT,
           ReturnTypes.TO_MULTISET,
           null,
           OperandTypes.ANY,
-          SqlFunctionCategory.SYSTEM) {
+          SqlFunctionCategory.SYSTEM, false, false) {
       };
 
   /**
    * The FUSION operator. Multiset aggregator function.
    */
   public static final SqlFunction FUSION =
-      new SqlFunction(
-          "FUSION",
-          SqlKind.OTHER_FUNCTION,
+      new SqlAggFunction("FUSION", null,
+          SqlKind.FUSION,
           ReturnTypes.ARG0,
           null,
           OperandTypes.MULTISET,
-          SqlFunctionCategory.SYSTEM);
+          SqlFunctionCategory.SYSTEM, false, false) {
+      };
 
   /**
    * The sequence next value function: <code>NEXT VALUE FOR sequence</code>

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
index 48e8ea4..5bf124d 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
@@ -39,6 +39,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
 
   //~ Instance fields --------------------------------------------------------
 
+  @Deprecated // to be removed before 2.0
   private final RelDataType type;
 
   //~ Constructors -----------------------------------------------------------
@@ -47,7 +48,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
     super(
         "SUM",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SUM,
         ReturnTypes.AGG_SUM,
         null,
         OperandTypes.NUMERIC,
@@ -63,6 +64,7 @@ public class SqlSumAggFunction extends SqlAggFunction {
     return ImmutableList.of(type);
   }
 
+  @Deprecated // to be removed before 2.0
   public RelDataType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/8531ab11/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
index 5b673e2..e8f7844 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
@@ -42,7 +42,7 @@ public class SqlSumEmptyIsZeroAggFunction extends SqlAggFunction {
   SqlSumEmptyIsZeroAggFunction() {
     super("$SUM0",
         null,
-        SqlKind.OTHER_FUNCTION,
+        SqlKind.SUM0,
         ReturnTypes.AGG_SUM_EMPTY_IS_ZERO,
         null,
         OperandTypes.NUMERIC,