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/06/05 22:22:48 UTC

[07/12] calcite git commit: Fix up [CALCITE-750], mainly removing mutable state

Fix up [CALCITE-750], mainly removing mutable state

Also, fix parser position of "agg(args) over windowSpec".


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

Branch: refs/heads/master
Commit: e25ceef6f31f9c0989ef695e6599e2f81f1f3556
Parents: 918e612
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Jun 3 12:49:57 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Sat Jun 4 21:35:29 2016 -0700

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       | 26 +++---
 .../org/apache/calcite/sql/SqlOverOperator.java | 13 +--
 .../apache/calcite/sql/validate/AggFinder.java  | 92 ++++----------------
 .../calcite/sql/validate/SqlValidatorImpl.java  | 66 +++++++-------
 .../calcite/sql2rel/SqlToRelConverter.java      | 70 ++++-----------
 .../apache/calcite/test/RelOptRulesTest.java    | 26 +++---
 .../calcite/test/SqlToRelConverterTest.java     | 12 +++
 .../apache/calcite/test/SqlValidatorTest.java   | 16 +++-
 .../org/apache/calcite/test/RelOptRulesTest.xml | 55 ++++++------
 .../calcite/test/SqlToRelConverterTest.xml      | 17 ++++
 10 files changed, 160 insertions(+), 233 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index cfd9f22..9b1bdd0 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -4351,8 +4351,6 @@ SqlNode NamedFunctionCall() :
     SqlParserPos starPos;
     SqlParserPos filterPos = null;
     SqlNode filter = null;
-    SqlParserPos overPos = null;
-    SqlCall function = null;
     SqlFunctionCategory funcType = SqlFunctionCategory.USER_DEFINED_FUNCTION;
     SqlNode e = null;
     SqlLiteral quantifier = null;
@@ -4396,17 +4394,16 @@ SqlNode NamedFunctionCall() :
         ]
         [
             <OVER>
-            {
-                overPos = getPos();
-                pos = pos.plus(overPos);
-            }
             (
                 id = SimpleIdentifier()
-                | e = WindowSpecification()
+            |   e = WindowSpecification()
             )
+            {
+                pos = pos.plus(getPos());
+            }
         ]
         {
-            function = createCall(
+            SqlCall function = createCall(
                 qualifiedName, pos, funcType, quantifier,
                 SqlParserUtil.toNodeArray(args));
 
@@ -4414,14 +4411,11 @@ SqlNode NamedFunctionCall() :
                 function = SqlStdOperatorTable.FILTER.createCall(filterPos,
                     function, filter);
             }
-            if (overPos != null) {
-                if (id != null) {
-                    function = SqlStdOperatorTable.OVER.createCall(overPos,
-                        function, id);
-                } else {
-                    function = SqlStdOperatorTable.OVER.createCall(overPos,
-                        function, e);
-                }
+            if (id != null) {
+                return SqlStdOperatorTable.OVER.createCall(pos, function, id);
+            }
+            if (e != null) {
+                return SqlStdOperatorTable.OVER.createCall(pos, function, e);
             }
             return function;
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
index bad5995..0c9accc 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOverOperator.java
@@ -23,7 +23,6 @@ import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
-import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -68,16 +67,8 @@ public class SqlOverOperator extends SqlBinaryOperator {
     if (!aggCall.getOperator().isAggregator()) {
       throw validator.newValidationError(aggCall, RESOURCE.overNonAggregate());
     }
-    // Enable nested aggregates with window aggregates (OVER operator)
-    if (validator instanceof SqlValidatorImpl) {
-      ((SqlValidatorImpl) validator).enableNestedAggregates();
-    }
-    validator.validateWindow(call.operand(1), scope, aggCall);
-    validator.validateAggregateParams(aggCall, null, scope);
-    // Disable nested aggregates post validation
-    if (validator instanceof SqlValidatorImpl) {
-      ((SqlValidatorImpl) validator).disableNestedAggregates();
-    }
+    final SqlNode window = call.operand(1);
+    validator.validateWindow(window, scope, aggCall);
   }
 
   public RelDataType deriveType(

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java b/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
index 8ea527c..a363fca 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggFinder.java
@@ -29,8 +29,6 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.Lists;
 
-import java.util.ArrayDeque;
-import java.util.Deque;
 import java.util.List;
 
 /**
@@ -40,17 +38,15 @@ import java.util.List;
 class AggFinder extends SqlBasicVisitor<Void> {
   //~ Instance fields --------------------------------------------------------
 
-  // Maximum allowed nesting level of aggregates
-  private static final int MAX_AGG_LEVEL = 2;
   private final SqlOperatorTable opTab;
+
+  /** Whether to find windowed aggregates. */
   private final boolean over;
 
-  private boolean nestedAgg;                        // Allow nested aggregates
+  /** Whether to find regular (non-windowed) aggregates. */
+  private boolean aggregate;
 
-  // Stores aggregate nesting level while visiting the tree to keep track of
-  // nested aggregates within window aggregates. An explicit stack is used
-  // instead of recursion to obey the SqlVisitor interface
-  private Deque<Integer> aggLevelStack;
+  private final AggFinder delegate;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -58,57 +54,23 @@ class AggFinder extends SqlBasicVisitor<Void> {
    * Creates an AggFinder.
    *
    * @param opTab Operator table
-   * @param over Whether to find windowed function calls {@code Agg(x) OVER
+   * @param over Whether to find windowed function calls {@code agg(x) OVER
    *             windowSpec}
+   * @param aggregate Whether to find non-windowed aggregate calls
+   * @param delegate Finder to which to delegate when processing the arguments
+   *                  to a non-windowed aggregate
    */
-  AggFinder(SqlOperatorTable opTab, boolean over) {
+  AggFinder(SqlOperatorTable opTab, boolean over, boolean aggregate,
+      AggFinder delegate) {
     this.opTab = opTab;
     this.over = over;
-    this.nestedAgg = false;
-    this.aggLevelStack = new ArrayDeque<Integer>();
+    this.aggregate = aggregate;
+    this.delegate = delegate;
   }
 
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * Allows nested aggregates within window aggregates
-   */
-  public void enableNestedAggregates()  {
-    this.nestedAgg = true;
-    this.aggLevelStack.clear();
-  }
-
-  /**
-   * Disallows nested aggregates within window aggregates
-   */
-  public void disableNestedAggregates()  {
-    this.nestedAgg = false;
-    this.aggLevelStack.clear();
-  }
-
-  public void addAggLevel(int aggLevel) {
-    aggLevelStack.push(aggLevel);
-  }
-
-  public void removeAggLevel() {
-    if (!aggLevelStack.isEmpty()) {
-      aggLevelStack.pop();
-    }
-  }
-
-  public int getAggLevel() {
-    if (!aggLevelStack.isEmpty()) {
-      return aggLevelStack.peek();
-    } else {
-      return -1;
-    }
-  }
-
-  public boolean isEmptyAggLevel() {
-    return aggLevelStack.isEmpty();
-  }
-
-  /**
    * Finds an aggregate.
    *
    * @param node Parse tree to search
@@ -138,20 +100,13 @@ class AggFinder extends SqlBasicVisitor<Void> {
 
   public Void visit(SqlCall call) {
     final SqlOperator operator = call.getOperator();
-    final int parAggLevel = this.getAggLevel(); //parent aggregate nesting level
     // If nested aggregates disallowed or found an aggregate at invalid level
     if (operator.isAggregator()) {
-      if (!nestedAgg || (parAggLevel + 1) > MAX_AGG_LEVEL) {
-        throw new Util.FoundOne(call);
-      } else {
-        if (parAggLevel >= 0) {
-          this.addAggLevel(parAggLevel + 1);
-        }
+      if (delegate != null) {
+        return call.getOperator().acceptCall(delegate, call);
       }
-    } else {
-      // Add the parent aggregate level before visiting its children
-      if (parAggLevel >= 0) {
-        this.addAggLevel(parAggLevel);
+      if (aggregate) {
+        throw new Util.FoundOne(call);
       }
     }
     // User-defined function may not be resolved yet.
@@ -164,12 +119,8 @@ class AggFinder extends SqlBasicVisitor<Void> {
       for (SqlOperator sqlOperator : list) {
         if (sqlOperator.isAggregator()) {
           // If nested aggregates disallowed or found aggregate at invalid level
-          if (!nestedAgg || (parAggLevel + 1) > MAX_AGG_LEVEL) {
+          if (aggregate) {
             throw new Util.FoundOne(call);
-          } else {
-            if (parAggLevel >= 0) {
-              this.addAggLevel(parAggLevel + 1);
-            }
           }
         }
       }
@@ -186,12 +137,7 @@ class AggFinder extends SqlBasicVisitor<Void> {
         return null;
       }
     }
-    super.visit(call);
-    // Remove the parent aggregate level after visiting its children
-    if (parAggLevel >= 0) {
-      this.removeAggLevel();
-    }
-    return null;
+    return super.visit(call);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/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 5704f3c..10f4398 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
@@ -223,6 +223,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       new IdentityHashMap<>();
   private final AggFinder aggFinder;
   private final AggFinder aggOrOverFinder;
+  private final AggFinder overFinder;
   private final SqlConformance conformance;
   private final Map<SqlNode, SqlNode> originalExprs = new HashMap<>();
 
@@ -244,7 +245,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   // if it's OK to expand the signature of that method.
   private boolean validatingSqlMerge;
 
-  private boolean nestedAgg;                        // Allow nested aggregates
+  private boolean inWindow;                        // Allow nested aggregates
 
   //~ Constructors -----------------------------------------------------------
 
@@ -273,26 +274,13 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
     rewriteCalls = true;
     expandColumnReferences = true;
-    aggFinder = new AggFinder(opTab, false);
-    aggOrOverFinder = new AggFinder(opTab, true);
+    aggFinder = new AggFinder(opTab, false, true, null);
+    aggOrOverFinder = new AggFinder(opTab, true, true, null);
+    overFinder = new AggFinder(opTab, true, false, aggOrOverFinder);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * Allows nested aggregates within window aggregates
-   */
-  public void enableNestedAggregates() {
-    this.nestedAgg = true;
-  }
-
-  /**
-   * Disallows nested aggregates within window aggregates
-   */
-  public void disableNestedAggregates() {
-    this.nestedAgg = false;
-  }
-
   public SqlConformance getConformance() {
     return conformance;
   }
@@ -4068,6 +4056,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlNode windowOrId,
       SqlValidatorScope scope,
       SqlCall call) {
+    // Enable nested aggregates with window aggregates (OVER operator)
+    inWindow = true;
+
     final SqlWindow targetWindow;
     switch (windowOrId.getKind()) {
     case IDENTIFIER:
@@ -4087,45 +4078,48 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     targetWindow.validate(this, scope);
     targetWindow.setWindowCall(null);
     call.validate(this, scope);
+
+    validateAggregateParams(call, null, scope);
+
+    // Disable nested aggregates post validation
+    inWindow = false;
   }
 
   public void validateAggregateParams(SqlCall aggCall, SqlNode filter,
       SqlValidatorScope scope) {
-    // For agg(expr), expr cannot itself contain aggregate function
-    // invocations.  For example, SUM(2*MAX(x)) is illegal; when
+    // For "agg(expr)", expr cannot itself contain aggregate function
+    // invocations.  For example, "SUM(2 * MAX(x))" is illegal; when
     // we see it, we'll report the error for the SUM (not the MAX).
     // For more than one level of nesting, the error which results
     // depends on the traversal order for validation.
-
-    // For window function agg(expr), expr can contain an aggregate function
-    // For example, AVG(2*MAX(x)) OVER (partition by y) GROUP BY y is legal; Only
-    // one level of nesting is allowed since non-window aggregates cannot nest aggregates.
-    aggOrOverFinder.disableNestedAggregates();
+    //
+    // For a windowed aggregate "agg(expr)", expr can contain an aggregate
+    // function. For example,
+    //   SELECT AVG(2 * MAX(x)) OVER (PARTITION BY y)
+    //   FROM t
+    //   GROUP BY y
+    // is legal. Only one level of nesting is allowed since non-windowed
+    // aggregates cannot nest aggregates.
 
     // Store nesting level of each aggregate. If an aggregate is found at an invalid
     // nesting level, throw an assert.
-    if (nestedAgg) {
-      aggOrOverFinder.enableNestedAggregates();
-      aggOrOverFinder.addAggLevel(1);
+    final AggFinder a;
+    if (inWindow) {
+      a = overFinder;
+    } else {
+      a = aggOrOverFinder;
     }
 
     for (SqlNode param : aggCall.getOperandList()) {
-      if (aggOrOverFinder.findAgg(param) != null) {
+      if (a.findAgg(param) != null) {
         throw newValidationError(aggCall, RESOURCE.nestedAggIllegal());
       }
     }
     if (filter != null) {
-      if (aggOrOverFinder.findAgg(filter) != null) {
+      if (a.findAgg(filter) != null) {
         throw newValidationError(filter, RESOURCE.aggregateInFilterIllegal());
       }
     }
-    // Disallow nested aggregates post call to this function
-    if (nestedAgg) {
-      aggOrOverFinder.removeAggLevel();
-      // Assert we don't have dangling items left in the stack
-      assert aggOrOverFinder.isEmptyAggLevel();
-      aggOrOverFinder.disableNestedAggregates();
-    }
   }
 
   public void validateCall(

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index e63bb66..6b5e34a 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -2522,14 +2522,14 @@ public class SqlToRelConverter {
 
       selectList.accept(aggConverter);
       // Assert we don't have dangling items left in the stack
-      assert aggConverter.isEmptyAggLevel();
+      assert !aggConverter.inOver;
       for (SqlNode expr : orderExprList) {
         expr.accept(aggConverter);
-        assert aggConverter.isEmptyAggLevel();
+        assert !aggConverter.inOver;
       }
       if (having != null) {
         having.accept(aggConverter);
-        assert aggConverter.isEmptyAggLevel();
+        assert !aggConverter.inOver;
       }
 
       // compute inputs to the aggregator
@@ -4453,14 +4453,8 @@ public class SqlToRelConverter {
     private final Map<AggregateCall, RexNode> aggCallMapping =
         Maps.newHashMap();
 
-    // Minimum allowed nesting level for converting aggregates within the
-    // OVER operator
-    private static final int MIN_AGG_LEVEL = 1;
-
-    // Stores aggregate nesting level while visiting the tree to keep track of
-    // nested aggregates within window aggregates. An explicit stack is used
-    // instead of recursion to obey the SqlVisitor interface
-    private Deque<Integer> aggLevelStack = new ArrayDeque<Integer>();
+    /** Are we directly inside a windowed aggregate? */
+    private boolean inOver = false;
 
     /**
      * Creates an AggConverter.
@@ -4529,12 +4523,10 @@ public class SqlToRelConverter {
       convertedInputExprNames.add(name);
     }
 
-    // implement SqlVisitor
     public Void visit(SqlIdentifier id) {
       return null;
     }
 
-    // implement SqlVisitor
     public Void visit(SqlNodeList nodeList) {
       for (int i = 0; i < nodeList.size(); i++) {
         nodeList.get(i).accept(this);
@@ -4542,50 +4534,23 @@ public class SqlToRelConverter {
       return null;
     }
 
-    // implement SqlVisitor
     public Void visit(SqlLiteral lit) {
       return null;
     }
 
-    // implement SqlVisitor
     public Void visit(SqlDataTypeSpec type) {
       return null;
     }
 
-    // implement SqlVisitor
     public Void visit(SqlDynamicParam param) {
       return null;
     }
 
-    // implement SqlVisitor
     public Void visit(SqlIntervalQualifier intervalQualifier) {
       return null;
     }
 
-    public void addAggLevel(int aggLevel) {
-      aggLevelStack.push(aggLevel);
-    }
-
-    public void removeAggLevel() {
-      if (!aggLevelStack.isEmpty()) {
-        aggLevelStack.pop();
-      }
-    }
-
-    public int getAggLevel() {
-      if (!aggLevelStack.isEmpty()) {
-        return aggLevelStack.peek();
-      } else {
-        return -1;
-      }
-    }
-
-    public boolean isEmptyAggLevel() {
-      return aggLevelStack.isEmpty();
-    }
-
     public Void visit(SqlCall call) {
-      int parAggLevel;                                 //parent aggregate nesting level
       switch (call.getKind()) {
       case FILTER:
         translateAgg((SqlCall) call.operand(0), call.operand(1), call);
@@ -4595,31 +4560,28 @@ public class SqlToRelConverter {
         // for now do not detect aggregates in subqueries.
         return null;
       }
-      // ignore window aggregates and ranking functions (associated with OVER operator)
-      // However, do not ignore nested window aggregates
+      final boolean prevInOver = inOver;
+      // Ignore window aggregates and ranking functions (associated with OVER
+      // operator). However, do not ignore nested window aggregates.
       if (call.getOperator().getKind() == SqlKind.OVER) {
         if (call.operand(0).getKind() == SqlKind.RANK) {
           return null;
         }
         // Track aggregate nesting levels only within an OVER operator.
-        this.addAggLevel(0);
+        inOver = true;
       }
 
-      parAggLevel = this.getAggLevel();
       // Do not translate the top level window aggregate. Only do so for
       // nested aggregates, if present
       if (call.getOperator().isAggregator()) {
-        if (parAggLevel < 0
-            || (parAggLevel + 1) > MIN_AGG_LEVEL) {
+        if (inOver) {
+          // Add the parent aggregate level before visiting its children
+          inOver = false;
+        } else {
+          // We're beyond the one ignored level
           translateAgg(call, null, call);
           return null;
-        } else if (parAggLevel >= 0) {
-          // Add the parent aggregate level before visiting its children
-          this.addAggLevel(parAggLevel + 1);
         }
-      } else if (call.getOperator().getKind() != SqlKind.OVER
-                 && parAggLevel >= 0) {
-        this.addAggLevel(parAggLevel);
       }
       for (SqlNode operand : call.getOperandList()) {
         // Operands are occasionally null, e.g. switched CASE arg 0.
@@ -4628,9 +4590,7 @@ public class SqlToRelConverter {
         }
       }
       // Remove the parent aggregate level after visiting its children
-      if (parAggLevel >= 0) {
-        this.removeAggLevel();
-      }
+      inOver = prevInOver;
       return null;
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 7c68842..d9340fb 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -1908,6 +1908,21 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanning(tester, preProgram, hepPlanner, sql);
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-750">[CALCITE-750]
+   * Allow windowed aggregate on top of regular aggregate</a>. */
+  @Test public void testNestedAggregates() {
+    final HepProgram program = HepProgram.builder()
+        .addRuleInstance(ProjectToWindowRule.PROJECT)
+        .build();
+    final String sql = "SELECT\n"
+        + "  avg(sum(sal) + 2 * min(empno) + 3 * avg(empno))\n"
+        + "  over (partition by deptno)\n"
+        + "from emp\n"
+        + "group by deptno";
+    checkPlanning(program, sql);
+  }
+
   @Test public void testPushAggregateThroughJoin1() throws Exception {
     final HepProgram preProgram = new HepProgramBuilder()
         .addRuleInstance(AggregateProjectMergeRule.INSTANCE)
@@ -2310,17 +2325,6 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkSubQuery(sql).check();
   }
 
-  /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-750">[CALCITE-750]
-   * Support nested aggregates - allows only one level nesting of aggregates
-   * under window aggregates i.e. window_agg(standard_agg) </a>. */
-  @Test public void testNestedAggregates() {
-    final HepProgram program = HepProgram.builder()
-                    .addRuleInstance(ProjectToWindowRule.PROJECT)
-                    .build();
-    checkPlanning(program, "SELECT avg(sum(sal) + 2*min(empno) + 3*avg(empno)) "
-            + "over (partition by deptno) from emp group by deptno");
-  }
 }
 
 // End RelOptRulesTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index b39fb7e..4ab1ae7 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -1167,6 +1167,18 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-750">[CALCITE-750]
+   * Allow windowed aggregate on top of regular aggregate</a>. */
+  @Test public void testNestedAggregates() {
+    final String sql = "SELECT\n"
+        + "  avg(sum(sal) + 2 * min(empno) + 3 * avg(empno))\n"
+        + "  over (partition by deptno)\n"
+        + "from emp\n"
+        + "group by deptno";
+    sql(sql).ok();
+  }
+
   /**
    * Test one of the custom conversions which is recognized by the class of the
    * operator (in this case,

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/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 8d90a15..3b2f92d 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -6333,13 +6333,21 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkFails(
         "select sum(^max(min(empno))^) from emp",
         ERR_NESTED_AGG);
+  }
 
-    // in OVER clause - this should be OK
-    check("select ^sum(max(empno)) OVER^ (order by deptno ROWS 2 PRECEDING) from emp");
+  @Test public void testNestedAggOver() {
+    // windowed agg applied to agg is OK
+    check("select sum(max(empno))\n"
+        + "  OVER (order by deptno ROWS 2 PRECEDING)\n"
+        + "from emp");
+    check("select sum(max(empno)) OVER w\n"
+        + "from emp\n"
+        + "window w as (order by deptno ROWS 2 PRECEDING)");
 
     // in OVER clause with more than one level of nesting
-    checkFails("select ^avg(sum(min(sal))) OVER^ (partition by deptno) from emp"
-          + " group by deptno", ERR_NESTED_AGG);
+    checkFails("select ^avg(sum(min(sal))) OVER (partition by deptno)^\n"
+        + "from emp group by deptno",
+        ERR_NESTED_AGG);
 
     // OVER in clause
     checkFails(

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index d4deeb3..10cf15c 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -1131,6 +1131,34 @@ LogicalProject(EMPNO=[$0])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testNestedAggregates">
+        <Resource name="sql">
+            <![CDATA[SELECT
+  avg(sum(sal) + 2 * min(empno) + 3 * avg(empno))
+  over (partition by deptno)
+from emp
+group by deptno]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EXPR$0=[CAST(/(SUM(+(+($1, *(2, $2)), *(3, $3))) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), COUNT(+(+($1, *(2, $2)), *(3, $3))) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))):INTEGER NOT NULL])
+  LogicalAggregate(group=[{0}], agg#0=[SUM($1)], agg#1=[MIN($2)], agg#2=[AVG($2)])
+    LogicalProject(DEPTNO=[$7], SAL=[$5], EMPNO=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EXPR$0=[CAST(/($0, $1)):INTEGER NOT NULL])
+  LogicalProject($0=[$2], $1=[$3])
+    LogicalWindow(window#0=[window(partition {0} order by [] range between UNBOUNDED PRECEDING and UNBOUNDED FOLLOWING aggs [SUM($1), COUNT($1)])])
+      LogicalProject(DEPTNO=[$0], $1=[+(+($1, *(2, $2)), *(3, $3))])
+        LogicalAggregate(group=[{0}], agg#0=[SUM($1)], agg#1=[MIN($2)], agg#2=[AVG($2)])
+          LogicalProject(DEPTNO=[$7], SAL=[$5], EMPNO=[$0])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testPushSumConstantThroughUnion">
         <Resource name="sql">
             <![CDATA[select ename, sum(u) from
@@ -5218,31 +5246,4 @@ LogicalProject(EMPNO=[$0])
 ]]>
         </Resource>
     </TestCase>
-    <TestCase name="testNestedAggregates">
-        <Resource name="sql">
-            <![CDATA[SELECT avg(sum(sal) + 2*min(empno) + 3*avg(empno))) over
-    (partition by deptno)
-    from emp
-    group by deptno]]>
-        </Resource>
-        <Resource name="planBefore">
-            <![CDATA[
-LogicalProject(EXPR$0=[CAST(/(SUM(+(+($1, *(2, $2)), *(3, $3))) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), COUNT(+(+($1, *(2, $2)), *(3, $3))) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))):INTEGER NOT NULL])
-  LogicalAggregate(group=[{0}], agg#0=[SUM($1)], agg#1=[MIN($2)], agg#2=[AVG($2)])
-    LogicalProject(DEPTNO=[$7], SAL=[$5], EMPNO=[$0])
-      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-]]>
-        </Resource>
-        <Resource name="planAfter">
-            <![CDATA[
-LogicalProject(EXPR$0=[CAST(/($0, $1)):INTEGER NOT NULL])
-  LogicalProject($0=[$2], $1=[$3])
-    LogicalWindow(window#0=[window(partition {0} order by [] range between UNBOUNDED PRECEDING and UNBOUNDED FOLLOWING aggs [SUM($1), COUNT($1)])])
-      LogicalProject(DEPTNO=[$0], $1=[+(+($1, *(2, $2)), *(3, $3))])
-        LogicalAggregate(group=[{0}], agg#0=[SUM($1)], agg#1=[MIN($2)], agg#2=[AVG($2)])
-          LogicalProject(DEPTNO=[$7], SAL=[$5], EMPNO=[$0])
-            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-]]>
-          </Resource>
-    </TestCase>
 </Root>

http://git-wip-us.apache.org/repos/asf/calcite/blob/e25ceef6/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 141e72e..1ac074f 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -1668,6 +1668,23 @@ LogicalProject(DEPTNO=[$0], NAME=[$1], EXPR$0=[$2])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testNestedAggregates">
+        <Resource name="sql">
+            <![CDATA[SELECT
+  avg(sum(sal) + 2 * min(empno) + 3 * avg(empno))
+  over (partition by deptno)
+from emp
+group by deptno]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[CAST(/(SUM(+(+($1, *(2, $2)), *(3, $3))) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), COUNT(+(+($1, *(2, $2)), *(3, $3))) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))):INTEGER NOT NULL])
+  LogicalAggregate(group=[{0}], agg#0=[SUM($1)], agg#1=[MIN($2)], agg#2=[AVG($2)])
+    LogicalProject(DEPTNO=[$7], SAL=[$5], EMPNO=[$0])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testWithAlias">
         <Resource name="sql">
             <![CDATA[with w(x, y) as (select * from dept where deptno > 10)