You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org> on 2019/03/08 23:14:52 UTC

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Dmitry Lychagin has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/3257

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................

[ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Fix illegal state exception raised by the compiler when
  a variable used by SQL aggregate function is not mapped
  by GROUP AS clause

Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
7 files changed, 87 insertions(+), 16 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/57/3257/1

diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp
new file mode 100644
index 0000000..7df10ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Description: Test error message when a variable used by SQL aggregate
+ *              is not mapped by GROUP AS clause
+ */
+
+from [{"f2":1, "f1":"foo"}] as t
+let x = t.f2 + 1
+group by t.f2
+group as g (t as tt)
+select t.f2, g, sum(x);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 6d90c86..23db0b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4119,6 +4119,12 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="group-by">
+      <compilation-unit name="sugar-08-negative">
+        <output-dir compare="Text">core-01</output-dir>
+        <expected-error>ASX1103: Illegal use of identifier: x</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
       <compilation-unit name="null">
         <output-dir compare="Text">null</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index ecf4eff..459773b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -189,6 +189,7 @@
     public static final int COMPILATION_UNEXPECTED_WINDOW_EXPRESSION = 1100;
     public static final int COMPILATION_UNEXPECTED_WINDOW_ORDERBY = 1101;
     public static final int COMPILATION_EXPECTED_WINDOW_FUNCTION = 1102;
+    public static final int COMPILATION_ILLEGAL_USE_OF_IDENTIFIER = 1103;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index c1b5d47..94d6942 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -176,6 +176,7 @@
 1100 = Unexpected window expression
 1101 = Unexpected ORDER BY clause in window expression
 1102 = Expected window or aggregate function, got: %1$s
+1103 = Illegal use of identifier: %1$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
index 130c8ac..c785aac 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
@@ -70,12 +70,16 @@
 
     private final Collection<VariableExpr> outerVars;
 
+    private final Set<VariableExpr> prohibitVars;
+
     Sql92AggregateFunctionVisitor(LangRewritingContext context, VariableExpr groupVar,
-            Map<Expression, Identifier> fieldVars, Collection<VariableExpr> outerVars) {
+            Map<Expression, Identifier> fieldVars, Collection<VariableExpr> outerVars,
+            Set<VariableExpr> prohibitVars) {
         this.context = context;
         this.groupVar = groupVar;
         this.fieldVars = fieldVars;
         this.outerVars = outerVars;
+        this.prohibitVars = prohibitVars;
     }
 
     @Override
@@ -86,7 +90,8 @@
         boolean rewritten = false;
         for (Expression expr : callExpr.getExprList()) {
             Expression newExpr =
-                    aggregate ? wrapAggregationArgument(expr, groupVar, fieldVars, outerVars, context) : expr;
+                    aggregate ? wrapAggregationArgument(expr, groupVar, fieldVars, outerVars, prohibitVars, context)
+                            : expr;
             rewritten |= newExpr != expr;
             newExprList.add(newExpr.accept(this, arg));
         }
@@ -100,7 +105,8 @@
     }
 
     static Expression wrapAggregationArgument(Expression expr, Expression groupVar,
-            Map<Expression, Identifier> fieldVars, Collection<VariableExpr> outerVars, LangRewritingContext context)
+            Map<Expression, Identifier> fieldVars, Collection<VariableExpr> outerVars, Collection<VariableExpr> prohibitVars,
+            LangRewritingContext context)
             throws CompilationException {
         SourceLocation sourceLoc = expr.getSourceLocation();
         Set<VariableExpr> freeVars = SqlppRewriteUtil.getFreeVariable(expr);
@@ -124,6 +130,9 @@
                 varExprMap.put(usedVar, fa);
             } else if (outerVars.contains(usedVar)) {
                 // Do nothing
+            } else if (prohibitVars != null && prohibitVars.contains(usedVar)) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_USE_OF_IDENTIFIER, sourceLoc,
+                        SqlppVariableUtil.toUserDefinedVariableName(usedVar.getVar().getValue()).getValue());
             } else {
                 // Rewrites to a reference to a single field in the group variable.
                 Identifier ident = findField(fieldVars, usedVar, context);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
index ad1268f..ba96142 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -31,10 +32,10 @@
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
-import org.apache.asterix.lang.sqlpp.clause.HavingClause;
 import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
 import org.apache.asterix.lang.sqlpp.clause.SelectClause;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.util.SqlppAstPrintUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
@@ -85,6 +86,9 @@
 
     @Override
     public Expression visit(SelectBlock selectBlock, ILangExpression arg) throws CompilationException {
+
+        Set<VariableExpr> outerScopeVars = scopeChecker.getCurrentScope().getLiveVariables();
+
         // Traverses the select block in the order of "from", "let/where"s, "group by", "let/having"s and "select".
         FromClause fromClause = selectBlock.getFromClause();
         if (selectBlock.hasFromClause()) {
@@ -105,14 +109,17 @@
             VariableExpr groupVar = groupbyClause.getGroupVar();
             Map<Expression, Identifier> groupFieldVars = getGroupFieldVariables(groupbyClause);
 
+            Set<VariableExpr> unmappedVars =
+                    getUnmappedVariables(visibleVarsPreGroupByScope, outerScopeVars, groupFieldVars);
+
             Collection<VariableExpr> freeVariables = new HashSet<>();
             Collection<VariableExpr> freeVariablesInGbyLets = new HashSet<>();
             if (selectBlock.hasLetHavingClausesAfterGroupby()) {
                 for (AbstractClause letHavingClause : selectBlock.getLetHavingListAfterGroupby()) {
                     letHavingClause.accept(this, arg);
                     // Rewrites each let/having clause after the group-by.
-                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, letHavingClause,
-                            visibleVarsPreGroupByScope);
+                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, letHavingClause, outerScopeVars,
+                            unmappedVars);
                     switch (letHavingClause.getClauseType()) {
                         case LET_CLAUSE:
                             LetClause letClause = (LetClause) letHavingClause;
@@ -138,16 +145,16 @@
                     // Rewrites the ORDER BY clause.
                     OrderbyClause orderbyClause = parentSelectExpression.getOrderbyClause();
                     orderbyClause.accept(this, arg);
-                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, orderbyClause,
-                            visibleVarsPreGroupByScope);
+                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, orderbyClause, outerScopeVars,
+                            unmappedVars);
                     freeVariables.addAll(SqlppVariableUtil.getFreeVariables(orderbyClause));
                 }
                 if (parentSelectExpression.hasLimit()) {
                     // Rewrites the LIMIT clause.
                     LimitClause limitClause = parentSelectExpression.getLimitClause();
                     limitClause.accept(this, arg);
-                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, limitClause,
-                            visibleVarsPreGroupByScope);
+                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, limitClause, outerScopeVars,
+                            unmappedVars);
                     freeVariables.addAll(SqlppVariableUtil.getFreeVariables(limitClause));
                 }
             }
@@ -156,7 +163,7 @@
             SelectClause selectClause = selectBlock.getSelectClause();
             selectClause.accept(this, arg);
             // Rewrites the select clause.
-            rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, selectClause, visibleVarsPreGroupByScope);
+            rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, selectClause, outerScopeVars, unmappedVars);
             freeVariables.addAll(SqlppVariableUtil.getFreeVariables(selectClause));
             freeVariables.removeAll(visibleVarsInCurrentScope);
 
@@ -170,7 +177,8 @@
 
             // Only retains used free variables.
             if (!decorVars.containsAll(freeVariables)) {
-                throw new IllegalStateException(decorVars + ":" + freeVariables);
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, groupbyClause.getSourceLocation(),
+                        decorVars + ":" + freeVariables);
             }
             decorVars.retainAll(freeVariables);
 
@@ -201,11 +209,27 @@
                 ? SqlppVariableUtil.createFieldVariableMap(groupbyClause.getGroupFieldList()) : Collections.emptyMap();
     }
 
+    /**
+     * Returns variables of the current SELECT block that were defined before GROUP BY clause but were not mapped by
+     * GROUP AS sub-clause. These variables cannot be used by SQL aggregate functions after the GROUP BY
+     */
+    private Set<VariableExpr> getUnmappedVariables(Set<VariableExpr> preGroupByScopeVariables,
+            Set<VariableExpr> outerScopeVariables, Map<Expression, Identifier> groupFieldVariables) {
+        Set<VariableExpr> result = new HashSet<>(preGroupByScopeVariables);
+        result.removeAll(outerScopeVariables);
+        for (Expression expr : groupFieldVariables.keySet()) {
+            if (expr.getKind() == Expression.Kind.VARIABLE_EXPRESSION) {
+                result.remove(expr);
+            }
+        }
+        return result;
+    }
+
     // Applying sugar rewriting for group-by.
     private void rewriteExpressionUsingGroupVariable(VariableExpr groupVar, Map<Expression, Identifier> fieldVars,
-            ILangExpression expr, Set<VariableExpr> outerScopeVariables) throws CompilationException {
-        Sql92AggregateFunctionVisitor visitor =
-                new Sql92AggregateFunctionVisitor(context, groupVar, fieldVars, outerScopeVariables);
+            ILangExpression expr, Set<VariableExpr> outerScopeVariables, Set<VariableExpr> prohibitVars) throws CompilationException {
+        Sql92AggregateFunctionVisitor visitor = new Sql92AggregateFunctionVisitor(context, groupVar, fieldVars,
+                outerScopeVariables, prohibitVars);
         expr.accept(visitor, null);
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
index 2216901..114efc1 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
@@ -111,7 +111,8 @@
         for (int i = 0; i < n; i++) {
             Expression expr = exprList.get(i);
             Expression newExpr = i < limit
-                    ? Sql92AggregateFunctionVisitor.wrapAggregationArgument(expr, winVar, fieldMap, liveVars, context)
+                    ? Sql92AggregateFunctionVisitor.wrapAggregationArgument(expr, winVar, fieldMap, liveVars, null,
+                            context)
                     : expr;
             newExprList.add(newExpr);
         }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8091/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/297/ (16/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8093/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5220/ (5/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/683/ (4/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5276/ (11/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10796/ (5/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5528/ (15/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/298/ (16/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5310/ (8/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/479/ (10/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/131/ (9/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5527/ (2/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://goo.gl/wzXWHh : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3322/ (7/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has submitted this change and it was merged.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


[ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Fix illegal state exception raised by the compiler when
  a variable used by SQL aggregate function is not mapped
  by GROUP AS clause

Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3257
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
7 files changed, 86 insertions(+), 19 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Ali Alsuliman: Looks good to me, approved
  Jenkins: Verified; No violations found; ; Verified



diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp
new file mode 100644
index 0000000..7df10ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Description: Test error message when a variable used by SQL aggregate
+ *              is not mapped by GROUP AS clause
+ */
+
+from [{"f2":1, "f1":"foo"}] as t
+let x = t.f2 + 1
+group by t.f2
+group as g (t as tt)
+select t.f2, g, sum(x);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 6d90c86..23db0b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4119,6 +4119,12 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="group-by">
+      <compilation-unit name="sugar-08-negative">
+        <output-dir compare="Text">core-01</output-dir>
+        <expected-error>ASX1103: Illegal use of identifier: x</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
       <compilation-unit name="null">
         <output-dir compare="Text">null</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index ecf4eff..459773b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -189,6 +189,7 @@
     public static final int COMPILATION_UNEXPECTED_WINDOW_EXPRESSION = 1100;
     public static final int COMPILATION_UNEXPECTED_WINDOW_ORDERBY = 1101;
     public static final int COMPILATION_EXPECTED_WINDOW_FUNCTION = 1102;
+    public static final int COMPILATION_ILLEGAL_USE_OF_IDENTIFIER = 1103;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index c1b5d47..94d6942 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -176,6 +176,7 @@
 1100 = Unexpected window expression
 1101 = Unexpected ORDER BY clause in window expression
 1102 = Expected window or aggregate function, got: %1$s
+1103 = Illegal use of identifier: %1$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
index 130c8ac..f52acbf 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
@@ -70,12 +70,15 @@
 
     private final Collection<VariableExpr> outerVars;
 
+    private final Set<VariableExpr> prohibitVars;
+
     Sql92AggregateFunctionVisitor(LangRewritingContext context, VariableExpr groupVar,
-            Map<Expression, Identifier> fieldVars, Collection<VariableExpr> outerVars) {
+            Map<Expression, Identifier> fieldVars, Collection<VariableExpr> outerVars, Set<VariableExpr> prohibitVars) {
         this.context = context;
         this.groupVar = groupVar;
         this.fieldVars = fieldVars;
         this.outerVars = outerVars;
+        this.prohibitVars = prohibitVars;
     }
 
     @Override
@@ -85,8 +88,8 @@
         boolean aggregate = FunctionMapUtil.isSql92AggregateFunction(signature);
         boolean rewritten = false;
         for (Expression expr : callExpr.getExprList()) {
-            Expression newExpr =
-                    aggregate ? wrapAggregationArgument(expr, groupVar, fieldVars, outerVars, context) : expr;
+            Expression newExpr = aggregate
+                    ? wrapAggregationArgument(expr, groupVar, fieldVars, outerVars, prohibitVars, context) : expr;
             rewritten |= newExpr != expr;
             newExprList.add(newExpr.accept(this, arg));
         }
@@ -100,8 +103,8 @@
     }
 
     static Expression wrapAggregationArgument(Expression expr, Expression groupVar,
-            Map<Expression, Identifier> fieldVars, Collection<VariableExpr> outerVars, LangRewritingContext context)
-            throws CompilationException {
+            Map<Expression, Identifier> fieldVars, Collection<VariableExpr> outerVars,
+            Collection<VariableExpr> prohibitVars, LangRewritingContext context) throws CompilationException {
         SourceLocation sourceLoc = expr.getSourceLocation();
         Set<VariableExpr> freeVars = SqlppRewriteUtil.getFreeVariable(expr);
 
@@ -124,6 +127,9 @@
                 varExprMap.put(usedVar, fa);
             } else if (outerVars.contains(usedVar)) {
                 // Do nothing
+            } else if (prohibitVars != null && prohibitVars.contains(usedVar)) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_USE_OF_IDENTIFIER, sourceLoc,
+                        SqlppVariableUtil.toUserDefinedVariableName(usedVar.getVar().getValue()).getValue());
             } else {
                 // Rewrites to a reference to a single field in the group variable.
                 Identifier ident = findField(fieldVars, usedVar, context);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
index ad1268f..43e2838 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -31,10 +32,10 @@
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
-import org.apache.asterix.lang.sqlpp.clause.HavingClause;
 import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
 import org.apache.asterix.lang.sqlpp.clause.SelectClause;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.util.SqlppAstPrintUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
@@ -85,6 +86,9 @@
 
     @Override
     public Expression visit(SelectBlock selectBlock, ILangExpression arg) throws CompilationException {
+
+        Set<VariableExpr> outerScopeVars = scopeChecker.getCurrentScope().getLiveVariables();
+
         // Traverses the select block in the order of "from", "let/where"s, "group by", "let/having"s and "select".
         FromClause fromClause = selectBlock.getFromClause();
         if (selectBlock.hasFromClause()) {
@@ -105,14 +109,17 @@
             VariableExpr groupVar = groupbyClause.getGroupVar();
             Map<Expression, Identifier> groupFieldVars = getGroupFieldVariables(groupbyClause);
 
+            Set<VariableExpr> unmappedVars =
+                    getUnmappedVariables(visibleVarsPreGroupByScope, outerScopeVars, groupFieldVars);
+
             Collection<VariableExpr> freeVariables = new HashSet<>();
             Collection<VariableExpr> freeVariablesInGbyLets = new HashSet<>();
             if (selectBlock.hasLetHavingClausesAfterGroupby()) {
                 for (AbstractClause letHavingClause : selectBlock.getLetHavingListAfterGroupby()) {
                     letHavingClause.accept(this, arg);
                     // Rewrites each let/having clause after the group-by.
-                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, letHavingClause,
-                            visibleVarsPreGroupByScope);
+                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, letHavingClause, outerScopeVars,
+                            unmappedVars);
                     switch (letHavingClause.getClauseType()) {
                         case LET_CLAUSE:
                             LetClause letClause = (LetClause) letHavingClause;
@@ -138,16 +145,16 @@
                     // Rewrites the ORDER BY clause.
                     OrderbyClause orderbyClause = parentSelectExpression.getOrderbyClause();
                     orderbyClause.accept(this, arg);
-                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, orderbyClause,
-                            visibleVarsPreGroupByScope);
+                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, orderbyClause, outerScopeVars,
+                            unmappedVars);
                     freeVariables.addAll(SqlppVariableUtil.getFreeVariables(orderbyClause));
                 }
                 if (parentSelectExpression.hasLimit()) {
                     // Rewrites the LIMIT clause.
                     LimitClause limitClause = parentSelectExpression.getLimitClause();
                     limitClause.accept(this, arg);
-                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, limitClause,
-                            visibleVarsPreGroupByScope);
+                    rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, limitClause, outerScopeVars,
+                            unmappedVars);
                     freeVariables.addAll(SqlppVariableUtil.getFreeVariables(limitClause));
                 }
             }
@@ -156,7 +163,7 @@
             SelectClause selectClause = selectBlock.getSelectClause();
             selectClause.accept(this, arg);
             // Rewrites the select clause.
-            rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, selectClause, visibleVarsPreGroupByScope);
+            rewriteExpressionUsingGroupVariable(groupVar, groupFieldVars, selectClause, outerScopeVars, unmappedVars);
             freeVariables.addAll(SqlppVariableUtil.getFreeVariables(selectClause));
             freeVariables.removeAll(visibleVarsInCurrentScope);
 
@@ -170,7 +177,8 @@
 
             // Only retains used free variables.
             if (!decorVars.containsAll(freeVariables)) {
-                throw new IllegalStateException(decorVars + ":" + freeVariables);
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, groupbyClause.getSourceLocation(),
+                        decorVars + ":" + freeVariables);
             }
             decorVars.retainAll(freeVariables);
 
@@ -201,11 +209,28 @@
                 ? SqlppVariableUtil.createFieldVariableMap(groupbyClause.getGroupFieldList()) : Collections.emptyMap();
     }
 
+    /**
+     * Returns variables of the current SELECT block that were defined before GROUP BY clause but were not mapped by
+     * GROUP AS sub-clause. These variables cannot be used by SQL aggregate functions after the GROUP BY
+     */
+    private Set<VariableExpr> getUnmappedVariables(Set<VariableExpr> preGroupByScopeVariables,
+            Set<VariableExpr> outerScopeVariables, Map<Expression, Identifier> groupFieldVariables) {
+        Set<VariableExpr> result = new HashSet<>(preGroupByScopeVariables);
+        result.removeAll(outerScopeVariables);
+        for (Expression expr : groupFieldVariables.keySet()) {
+            if (expr.getKind() == Expression.Kind.VARIABLE_EXPRESSION) {
+                result.remove(expr);
+            }
+        }
+        return result;
+    }
+
     // Applying sugar rewriting for group-by.
     private void rewriteExpressionUsingGroupVariable(VariableExpr groupVar, Map<Expression, Identifier> fieldVars,
-            ILangExpression expr, Set<VariableExpr> outerScopeVariables) throws CompilationException {
+            ILangExpression expr, Set<VariableExpr> outerScopeVariables, Set<VariableExpr> prohibitVars)
+            throws CompilationException {
         Sql92AggregateFunctionVisitor visitor =
-                new Sql92AggregateFunctionVisitor(context, groupVar, fieldVars, outerScopeVariables);
+                new Sql92AggregateFunctionVisitor(context, groupVar, fieldVars, outerScopeVariables, prohibitVars);
         expr.accept(visitor, null);
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
index 2216901..597a1ab 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
@@ -110,9 +110,8 @@
         List<Expression> newExprList = new ArrayList<>(n);
         for (int i = 0; i < n; i++) {
             Expression expr = exprList.get(i);
-            Expression newExpr = i < limit
-                    ? Sql92AggregateFunctionVisitor.wrapAggregationArgument(expr, winVar, fieldMap, liveVars, context)
-                    : expr;
+            Expression newExpr = i < limit ? Sql92AggregateFunctionVisitor.wrapAggregationArgument(expr, winVar,
+                    fieldMap, liveVars, null, context) : expr;
             newExprList.add(newExpr);
         }
         winExpr.setExprList(newExprList);

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5218/ (8/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/685/ (14/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5315/ (4/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5891/ (9/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5889/ (14/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/3257

to look at the new patch set (#2).

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................

[ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Fix illegal state exception raised by the compiler when
  a variable used by SQL aggregate function is not mapped
  by GROUP AS clause

Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
---
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-08-negative/sugar-08-negative.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
7 files changed, 86 insertions(+), 19 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/57/3257/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/481/ (1/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://goo.gl/YT9bLX : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5532/ (13/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5274/ (15/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10798/ (6/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5680/ (11/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/622/ (7/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4044/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5682/ (13/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3324/ (3/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5313/ (6/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5534/ (12/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8093/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/GWgGLi : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/132/ (2/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4044/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/621/ (3/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8091/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Ali Alsuliman has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9264/ (12/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5308/ (1/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2528][COMP] Fix illegal state exception in the co...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2528][COMP] Fix illegal state exception in the compiler
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9266/ (10/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3257
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I12bab27ad8e25d0bd55c900e559541eff2141fb9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No