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 2015/09/02 02:09:41 UTC

[05/18] incubator-calcite git commit: [CALCITE-820] Validate that window functions have OVER clause (Hsuan-Yi Chu)

[CALCITE-820] Validate that window functions have OVER clause (Hsuan-Yi Chu)

Close apache/incubator-calcite#113


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

Branch: refs/heads/master
Commit: 77674a7c64dfbe7f45a2fea12bce475a9f96161f
Parents: 5d8a90b
Author: Hsuan-Yi Chu <hs...@usc.edu>
Authored: Tue Jul 28 15:32:33 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Sep 1 16:17:14 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/runtime/CalciteResource.java |  3 ++
 .../org/apache/calcite/sql/SqlAggFunction.java  | 48 ++++++++++++++++++--
 .../org/apache/calcite/sql/SqlRankFunction.java | 11 ++---
 .../calcite/sql/fun/SqlAvgAggFunction.java      |  5 +-
 .../calcite/sql/fun/SqlCountAggFunction.java    |  5 +-
 .../calcite/sql/fun/SqlCovarAggFunction.java    |  5 +-
 .../sql/fun/SqlFirstLastValueAggFunction.java   | 12 ++---
 .../sql/fun/SqlHistogramAggFunction.java        |  5 +-
 .../calcite/sql/fun/SqlLeadLagAggFunction.java  |  9 ++--
 .../calcite/sql/fun/SqlMinMaxAggFunction.java   |  5 +-
 .../calcite/sql/fun/SqlNtileAggFunction.java    |  9 ++--
 .../sql/fun/SqlSingleValueAggFunction.java      |  5 +-
 .../calcite/sql/fun/SqlSumAggFunction.java      |  5 +-
 .../sql/fun/SqlSumEmptyIsZeroAggFunction.java   |  5 +-
 .../sql/validate/SqlUserDefinedAggFunction.java |  2 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  | 10 ++++
 .../calcite/runtime/CalciteResource.properties  |  1 +
 .../apache/calcite/test/SqlValidatorTest.java   | 26 ++++++++++-
 .../org/apache/calcite/tools/PlannerTest.java   |  4 +-
 19 files changed, 134 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index a511143..e74f915 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -243,6 +243,9 @@ public interface CalciteResource {
   ExInst<SqlValidatorException> naturalOrUsingColumnNotCompatible(String a0,
       String a1, String a2);
 
+  @BaseMessage("OVER clause is necessary for window functions")
+  ExInst<SqlValidatorException> absentOverClause();
+
   @BaseMessage("Window ''{0}'' not found")
   ExInst<SqlValidatorException> windowNotFound(String a0);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 944309a..83fe248 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
@@ -28,9 +28,13 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
  * which aggregates sets of values into a result.
  */
 public abstract class SqlAggFunction extends SqlFunction implements Context {
+  private final boolean requiresOrder;
+  private final boolean requiresOver;
+
   //~ Constructors -----------------------------------------------------------
 
   /** Creates a built-in SqlAggFunction. */
+  @Deprecated // to be removed before 2.0
   protected SqlAggFunction(
       String name,
       SqlKind kind,
@@ -40,10 +44,11 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
       SqlFunctionCategory funcType) {
     // We leave sqlIdentifier as null to indicate that this is a builtin.
     this(name, null, kind, returnTypeInference, operandTypeInference,
-        operandTypeChecker, funcType);
+        operandTypeChecker, funcType, false, false);
   }
 
   /** Creates a user-defined SqlAggFunction. */
+  @Deprecated // to be removed before 2.0
   protected SqlAggFunction(
       String name,
       SqlIdentifier sqlIdentifier,
@@ -52,8 +57,28 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
       SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker,
       SqlFunctionCategory funcType) {
+    this(name, sqlIdentifier, kind, returnTypeInference, operandTypeInference,
+        operandTypeChecker, funcType, false, false);
+  }
+
+  /** Creates a built-in or user-defined SqlAggFunction or window function.
+   *
+   * <p>A user-defined function will have a value for {@code sqlIdentifier}; for
+   * a built-in function it will be null. */
+  protected SqlAggFunction(
+      String name,
+      SqlIdentifier sqlIdentifier,
+      SqlKind kind,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeInference operandTypeInference,
+      SqlOperandTypeChecker operandTypeChecker,
+      SqlFunctionCategory funcType,
+      boolean requiresOrder,
+      boolean requireOver) {
     super(name, sqlIdentifier, kind, returnTypeInference, operandTypeInference,
         operandTypeChecker, null, funcType);
+    this.requiresOrder = requiresOrder;
+    this.requiresOver =  requireOver;
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -66,12 +91,11 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
     return true;
   }
 
-  public boolean isQuantifierAllowed() {
+  @Override public boolean isQuantifierAllowed() {
     return true;
   }
 
-  // override SqlFunction
-  public void validateCall(
+  @Override public void validateCall(
       SqlCall call,
       SqlValidator validator,
       SqlValidatorScope scope,
@@ -79,6 +103,22 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
     super.validateCall(call, validator, scope, operandScope);
     validator.validateAggregateParams(call, null, scope);
   }
+
+  @Override public final boolean requiresOrder() {
+    return requiresOrder;
+  }
+
+  /** Returns whether this is a window function that requires an OVER clause.
+   *
+   * <p>For example, {@code RANK} and {@code DENSE_RANK} require an OVER clause;
+   * {@code SUM} does not (it can be used as a non-window aggregate function).
+   *
+   * @see #allowsFraming()
+   * @see #requiresOrder()
+   */
+  public final boolean requiresOver() {
+    return requiresOver;
+  }
 }
 
 // End SqlAggFunction.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 11b5ca1..e2a2490 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlRankFunction.java
@@ -31,7 +31,6 @@ import java.util.List;
 public class SqlRankFunction extends SqlAggFunction {
   //~ Instance fields --------------------------------------------------------
 
-  private final boolean requiresOrder;
   private final RelDataType type = null;
 
   //~ Constructors -----------------------------------------------------------
@@ -39,20 +38,18 @@ public class SqlRankFunction extends SqlAggFunction {
   public SqlRankFunction(String name, boolean requiresOrder) {
     super(
         name,
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.INTEGER,
         null,
         OperandTypes.NILADIC,
-        SqlFunctionCategory.NUMERIC);
-    this.requiresOrder = requiresOrder;
+        SqlFunctionCategory.NUMERIC,
+        requiresOrder,
+        true);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public boolean requiresOrder() {
-    return requiresOrder;
-  }
-
   @Override public boolean allowsFraming() {
     return false;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 af78e8c..b86d49c 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
@@ -53,11 +53,14 @@ public class SqlAvgAggFunction extends SqlAggFunction {
       Subtype subtype) {
     super(
         subtype.name(),
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC,
-        SqlFunctionCategory.NUMERIC);
+        SqlFunctionCategory.NUMERIC,
+        false,
+        false);
     this.type = type;
     this.subtype = subtype;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 3feefc2..e91f30b 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
@@ -47,13 +47,16 @@ public class SqlCountAggFunction extends SqlAggFunction {
   public SqlCountAggFunction() {
     super(
         "COUNT",
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.BIGINT,
         null,
         SqlValidator.STRICT
             ? OperandTypes.ANY
             : OperandTypes.ONE_OR_MORE,
-        SqlFunctionCategory.NUMERIC);
+        SqlFunctionCategory.NUMERIC,
+        false,
+        false);
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 be63c1a..df36095 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
@@ -50,11 +50,14 @@ public class SqlCovarAggFunction extends SqlAggFunction {
    */
   public SqlCovarAggFunction(RelDataType type, Subtype subtype) {
     super(subtype.name(),
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC_NUMERIC,
-        SqlFunctionCategory.NUMERIC);
+        SqlFunctionCategory.NUMERIC,
+        false,
+        false);
     this.type = type;
     this.subtype = subtype;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 2ca5375..bc75424 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
@@ -40,22 +40,18 @@ public class SqlFirstLastValueAggFunction extends SqlAggFunction {
   public SqlFirstLastValueAggFunction(boolean firstFlag) {
     super(
         firstFlag ? "FIRST_VALUE" : "LAST_VALUE",
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.ANY,
-        SqlFunctionCategory.NUMERIC);
+        SqlFunctionCategory.NUMERIC,
+        false,
+        true);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public boolean requiresOrder() {
-    // Allow the user to shoot herself into the foot by using first_value
-    // and/or last_value without order by. This will result in undefined
-    // behaviour, however lots of databases allow that.
-    return false;
-  }
-
   public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
     return ImmutableList.of(
         typeFactory.createTypeWithNullability(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 9f2e461..8572338 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
@@ -45,11 +45,14 @@ public class SqlHistogramAggFunction extends SqlAggFunction {
   public SqlHistogramAggFunction(RelDataType type) {
     super(
         "$HISTOGRAM",
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.HISTOGRAM,
         null,
         OperandTypes.NUMERIC_OR_STRING,
-        SqlFunctionCategory.NUMERIC);
+        SqlFunctionCategory.NUMERIC,
+        false,
+        false);
     this.type = type;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 a6d5fec..a2b0c45 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
@@ -76,15 +76,14 @@ public class SqlLeadLagAggFunction extends SqlAggFunction {
   public SqlLeadLagAggFunction(boolean isLead) {
     super(
         isLead ? "LEAD" : "LAG",
+        null,
         SqlKind.OTHER_FUNCTION,
         RETURN_TYPE,
         null,
         OPERAND_TYPES,
-        SqlFunctionCategory.NUMERIC);
-  }
-
-  @Override public boolean requiresOrder() {
-    return true;
+        SqlFunctionCategory.NUMERIC,
+        false,
+        true);
   }
 
   @Override public boolean allowsFraming() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 2ce8d6f..27a1a8b 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
@@ -69,11 +69,14 @@ public class SqlMinMaxAggFunction extends SqlAggFunction {
       int kind) {
     super(
         isMin ? "MIN" : "MAX",
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.COMPARABLE_ORDERED,
-        SqlFunctionCategory.SYSTEM);
+        SqlFunctionCategory.SYSTEM,
+        false,
+        false);
     this.argTypes = argTypes;
     this.isMin = isMin;
     this.kind = kind;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 5aaa2c3..524e7d6 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
@@ -34,15 +34,14 @@ public class SqlNtileAggFunction extends SqlAggFunction {
   public SqlNtileAggFunction() {
     super(
         "NTILE",
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.INTEGER,
         null,
         OperandTypes.POSITIVE_INTEGER_LITERAL,
-        SqlFunctionCategory.NUMERIC);
-  }
-
-  @Override public boolean requiresOrder() {
-    return true;
+        SqlFunctionCategory.NUMERIC,
+        false,
+        true);
   }
 
   public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 f13f3d8..a27e180 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
@@ -43,11 +43,14 @@ public class SqlSingleValueAggFunction extends SqlAggFunction {
       RelDataType type) {
     super(
         "SINGLE_VALUE",
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.ARG0,
         null,
         OperandTypes.ANY,
-        SqlFunctionCategory.SYSTEM);
+        SqlFunctionCategory.SYSTEM,
+        false,
+        false);
     this.type = type;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 41b9d1d..0fc6606 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
@@ -46,11 +46,14 @@ public class SqlSumAggFunction extends SqlAggFunction {
   public SqlSumAggFunction(RelDataType type) {
     super(
         "SUM",
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.ARG0_NULLABLE_IF_EMPTY,
         null,
         OperandTypes.NUMERIC,
-        SqlFunctionCategory.NUMERIC);
+        SqlFunctionCategory.NUMERIC,
+        false,
+        false);
     this.type = type;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/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 965ae75..9cb3459 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
@@ -41,11 +41,14 @@ public class SqlSumEmptyIsZeroAggFunction extends SqlAggFunction {
 
   SqlSumEmptyIsZeroAggFunction() {
     super("$SUM0",
+        null,
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.ARG0,
         null,
         OperandTypes.NUMERIC,
-        SqlFunctionCategory.NUMERIC);
+        SqlFunctionCategory.NUMERIC,
+        false,
+        false);
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
index 6994e97..a5592f3 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
@@ -49,7 +49,7 @@ public class SqlUserDefinedAggFunction extends SqlAggFunction {
       SqlOperandTypeChecker operandTypeChecker, AggregateFunction function) {
     super(Util.last(opName.names), opName, SqlKind.OTHER_FUNCTION,
         returnTypeInference, operandTypeInference, operandTypeChecker,
-        SqlFunctionCategory.USER_DEFINED_FUNCTION);
+        SqlFunctionCategory.USER_DEFINED_FUNCTION, false, false);
     this.function = function;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index f210749..90fa0e2 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -30,6 +30,7 @@ import org.apache.calcite.sql.JoinConditionType;
 import org.apache.calcite.sql.JoinType;
 import org.apache.calcite.sql.SqlAccessEnum;
 import org.apache.calcite.sql.SqlAccessType;
+import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCallBinding;
@@ -3399,6 +3400,15 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * @param scope Scope in which expression occurs
    */
   private void validateExpr(SqlNode expr, SqlValidatorScope scope) {
+    if (expr instanceof SqlCall) {
+      final SqlCall sqlCall = (SqlCall) expr;
+      if (sqlCall.getOperator().isAggregator()
+          && ((SqlAggFunction) sqlCall.getOperator()).requiresOver()) {
+        throw newValidationError(expr,
+            RESOURCE.absentOverClause());
+      }
+    }
+
     // Call on the expression to validate itself.
     expr.validateExpr(this, scope);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 5ab24cc..b3f9cce 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -115,6 +115,7 @@ RowMustBeNonNegativeIntegral=ROWS value must be a non-negative integral constant
 OverMissingOrderBy=Window specification must contain an ORDER BY clause
 PartitionbyShouldNotContainOver=PARTITION BY expression should not contain OVER clause
 OrderbyShouldNotContainOver=ORDER BY expression should not contain OVER clause
+AbsentOverClause=OVER clause is necessary for window functions
 BadLowerBoundary=UNBOUNDED FOLLOWING cannot be specified for the lower frame boundary
 BadUpperBoundary=UNBOUNDED PRECEDING cannot be specified for the upper frame boundary
 CurrentRowPrecedingError=Upper frame boundary cannot be PRECEDING when lower boundary is CURRENT ROW

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 106ad92..78d9e1c 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -3804,6 +3804,29 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // Test specified collation, window clause syntax rule 4,5.
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-820">[CALCITE-820]
+   * Validate that window functions have OVER clause</a>. */
+  @Test public void testWindowFunctionsWithoutOver() {
+    winSql(
+        "select sum(empno) \n"
+        + "from emp \n"
+        + "group by deptno \n"
+        + "order by ^row_number()^")
+        .fails("OVER clause is necessary for window functions");
+
+    winSql(
+        "select ^rank()^ \n"
+        + "from emp")
+        .fails("OVER clause is necessary for window functions");
+
+    winSql(
+        "select cume_dist() over w , ^rank()^\n"
+        + "from emp \n"
+        + "window w as (partition by deptno order by deptno)")
+        .fails("OVER clause is necessary for window functions");
+  }
+
   @Test public void testOverInPartitionBy() {
     winSql(
         "select sum(deptno) over ^(partition by sum(deptno) \n"
@@ -3900,7 +3923,8 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     // rank function type
     if (defined.contains("DENSE_RANK")) {
-      winExp("dense_rank()").ok();
+      winExp("^dense_rank()^")
+          .fails("OVER clause is necessary for window functions");
     } else {
       checkWinFuncExpWithWinClause(
           "^dense_rank()^",

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/77674a7c/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index ff6094a..551a612 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -946,8 +946,8 @@ public class PlannerTest {
   /** User-defined aggregate function. */
   public static class MyCountAggFunction extends SqlAggFunction {
     public MyCountAggFunction() {
-      super("MY_COUNT", SqlKind.OTHER_FUNCTION, ReturnTypes.BIGINT, null,
-          OperandTypes.ANY, SqlFunctionCategory.NUMERIC);
+      super("MY_COUNT", null, SqlKind.OTHER_FUNCTION, ReturnTypes.BIGINT, null,
+          OperandTypes.ANY, SqlFunctionCategory.NUMERIC, false, false);
     }
 
     public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {