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/01/17 18:43:27 UTC

Change in asterixdb[master]: [NO ISSUE][COMP] Window operator compiler fixes

Dmitry Lychagin has uploaded a new change for review.

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................

[NO ISSUE][COMP] Window operator compiler fixes

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

Details:
- Make window operator non-movable
- Add optimizer rule to remove empty window operators
- Consider window operators in RemoveUnusedAssignAndAggregateRule
- Propagate source location to constant expressions

Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/pg_win/pg_win_01.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/win_opt_01/win_opt_01_8.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
A asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_8.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_opt_01/win_opt_01.8.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_opt_01/win_opt_01.8.adm
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
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/SqlppWindowAggregationSugarVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
M hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
A hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantWindowOperatorsRule.java
M hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
15 files changed, 234 insertions(+), 35 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 80469a7..31be802 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -94,6 +94,7 @@
 import org.apache.hyracks.algebricks.rewriter.rules.ComplexUnnestToProductRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
+import org.apache.hyracks.algebricks.rewriter.rules.RemoveRedundantWindowOperatorsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ReuseWindowAggregateRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateWindowOperatorsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.CopyLimitDownRule;
@@ -296,6 +297,7 @@
         // Window operator consolidation rules
         consolidation.add(new ConsolidateWindowOperatorsRule());
         consolidation.add(new ReuseWindowAggregateRule());
+        consolidation.add(new RemoveRedundantWindowOperatorsRule());
         consolidation.add(new RemoveRedundantVariablesRule());
         return consolidation;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index e3745f7..61c525e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -714,8 +714,9 @@
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME));
         fldAccess.setSourceLocation(sourceLoc);
         fldAccess.getArguments().add(new MutableObject<>(p.first));
-        ILogicalExpression faExpr =
+        ConstantExpression faExpr =
                 new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent().getValue())));
+        faExpr.setSourceLocation(sourceLoc);
         fldAccess.getArguments().add(new MutableObject<>(faExpr));
         AssignOperator a = new AssignOperator(v, new MutableObject<>(fldAccess));
         a.getInputs().add(p.second);
@@ -822,8 +823,11 @@
                     break;
                 case LITERAL_EXPRESSION:
                     LiteralExpr val = (LiteralExpr) expr;
-                    args.add(new MutableObject<>(new ConstantExpression(
-                            new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue())))));
+                    AsterixConstantValue cValue =
+                            new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()));
+                    ConstantExpression cExpr = new ConstantExpression(cValue);
+                    cExpr.setSourceLocation(expr.getSourceLocation());
+                    args.add(new MutableObject<>(cExpr));
                     break;
                 default:
                     Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(expr, topOp);
@@ -1133,10 +1137,13 @@
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+        SourceLocation sourceLoc = l.getSourceLocation();
         LogicalVariable var = context.newVar();
-        AssignOperator a = new AssignOperator(var, new MutableObject<>(
-                new ConstantExpression(new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
-        a.setSourceLocation(l.getSourceLocation());
+        AsterixConstantValue cValue = new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue()));
+        ConstantExpression cExpr = new ConstantExpression(cValue);
+        cExpr.setSourceLocation(sourceLoc);
+        AssignOperator a = new AssignOperator(var, new MutableObject<>(cExpr));
+        a.setSourceLocation(sourceLoc);
         if (tupSource != null) {
             a.getInputs().add(tupSource);
         }
@@ -1295,9 +1302,10 @@
             // condition() -> not(if-missing-or-null(condition(), false))
 
             List<Mutable<ILogicalExpression>> ifMissingOrNullArgs = new ArrayList<>(2);
+            ConstantExpression eFalse = new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
+            eFalse.setSourceLocation(sourceLoc);
             ifMissingOrNullArgs.add(new MutableObject<>(eo2.first));
-            ifMissingOrNullArgs
-                    .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE))));
+            ifMissingOrNullArgs.add(new MutableObject<>(eFalse));
 
             List<Mutable<ILogicalExpression>> notArgs = new ArrayList<>(1);
             ScalarFunctionCallExpression ifMissinOrNullExpr = new ScalarFunctionCallExpression(
@@ -1560,8 +1568,11 @@
                 return new Pair<>(varRefExpr, topOpRef);
             case LITERAL_EXPRESSION:
                 LiteralExpr val = (LiteralExpr) expr;
-                return new Pair<>(new ConstantExpression(
-                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOpRef);
+                AsterixConstantValue cValue =
+                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()));
+                ConstantExpression cExpr = new ConstantExpression(cValue);
+                cExpr.setSourceLocation(sourceLoc);
+                return new Pair<>(cExpr, topOpRef);
             default:
                 if (expressionNeedsNoNesting(expr)) {
                     Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOpRef);
@@ -1876,9 +1887,10 @@
         count.setSourceLocation(sourceLoc);
         AbstractFunctionCallExpression comparison = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(not ? BuiltinFunctions.EQ : BuiltinFunctions.NEQ));
+        ConstantExpression eZero = new ConstantExpression(new AsterixConstantValue(new AInt64(0L)));
+        eZero.setSourceLocation(sourceLoc);
         comparison.getArguments().add(new MutableObject<>(count));
-        comparison.getArguments()
-                .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(0L)))));
+        comparison.getArguments().add(new MutableObject<>(eZero));
         comparison.setSourceLocation(sourceLoc);
         AssignOperator a = new AssignOperator(v1, new MutableObject<>(comparison));
         a.setSourceLocation(sourceLoc);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/pg_win/pg_win_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/pg_win/pg_win_01.sqlpp
new file mode 100644
index 0000000..4d7dc92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/pg_win/pg_win_01.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 removal of unused window operator
+ * Expected Res : SUCCESS (no window operators in the optimized plan)
+ */
+
+/* pg_win.16 */
+
+FROM (
+  FROM (SELECT * FROM range(1, 10) four WHERE FALSE) s
+  SELECT count(*) OVER (PARTITION BY four) AS `count`
+) t
+SELECT count(*) AS `cnt`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/win_opt_01/win_opt_01_8.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/win_opt_01/win_opt_01_8.sqlpp
new file mode 100644
index 0000000..fff3095
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/win_opt_01/win_opt_01_8.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 removal of unused window operator
+ * Expected Res : SUCCESS (no window operators in the optimized plan)
+ */
+
+FROM (
+  FROM range(1, 10) x
+  SELECT VALUE row_number() OVER (ORDER BY x)
+) y
+SELECT VALUE COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
new file mode 100644
index 0000000..1fee6e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
@@ -0,0 +1,9 @@
+-- DISTRIBUTE_RESULT  |LOCAL|
+  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+    -- STREAM_PROJECT  |LOCAL|
+      -- ASSIGN  |LOCAL|
+        -- AGGREGATE  |LOCAL|
+          -- AGGREGATE  |LOCAL|
+            -- UNNEST  |UNPARTITIONED|
+              -- STREAM_SELECT  |UNPARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_8.plan
new file mode 100644
index 0000000..28dbe05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_8.plan
@@ -0,0 +1,6 @@
+-- DISTRIBUTE_RESULT  |LOCAL|
+  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+    -- AGGREGATE  |LOCAL|
+      -- AGGREGATE  |LOCAL|
+        -- UNNEST  |UNPARTITIONED|
+          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_opt_01/win_opt_01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_opt_01/win_opt_01.8.query.sqlpp
new file mode 100644
index 0000000..fff3095
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_opt_01/win_opt_01.8.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 removal of unused window operator
+ * Expected Res : SUCCESS (no window operators in the optimized plan)
+ */
+
+FROM (
+  FROM range(1, 10) x
+  SELECT VALUE row_number() OVER (ORDER BY x)
+) y
+SELECT VALUE COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_opt_01/win_opt_01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_opt_01/win_opt_01.8.adm
new file mode 100644
index 0000000..9a03714
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_opt_01/win_opt_01.8.adm
@@ -0,0 +1 @@
+10
\ No newline at end of file
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
index 3606b12..4441d65 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
@@ -25,7 +25,6 @@
 import java.util.List;
 
 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;
@@ -37,11 +36,10 @@
 import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Base class for visitors that extract expressions into LET clauses.
- * Subclasses should call {@link #extractExpressions(List, int, SourceLocation)} to perform the extraction.
+ * Subclasses should call {@link #extractExpressions(List, int)} to perform the extraction.
  */
 abstract class AbstractSqlppExpressionExtractionVisitor extends AbstractSqlppSimpleExpressionVisitor {
 
@@ -111,11 +109,10 @@
         fromBindingList.clear();
     }
 
-    List<Expression> extractExpressions(List<Expression> exprList, int limit, SourceLocation sourceLocation)
-            throws CompilationException {
+    List<Expression> extractExpressions(List<Expression> exprList, int limit) {
         List<Pair<Expression, VarIdentifier>> outLetList = stack.peek();
         if (outLetList == null) {
-            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLocation);
+            return null;
         }
         int n = exprList.size();
         List<Expression> newExprList = new ArrayList<>(n);
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 6e14e74..130c8ac 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
@@ -169,7 +169,7 @@
             ident = me.getValue();
         }
         if (ident == null) {
-            throw new CompilationException(ErrorCode.COMPILATION_ERROR, usedVar.getSourceLocation());
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, usedVar.getSourceLocation(), "");
         }
         return ident;
     }
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 910552a..a060d57 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
@@ -27,7 +27,6 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
index b1c2d60..3c5032a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
@@ -66,13 +66,17 @@
         if (winfi != null) {
             if (BuiltinFunctions.windowFunctionHasProperty(winfi,
                     BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
-                List<Expression> newExprList =
-                        extractExpressions(winExpr.getExprList(), 1, winExpr.getSourceLocation());
+                List<Expression> newExprList = extractExpressions(winExpr.getExprList(), 1);
+                if (newExprList == null) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, winExpr.getSourceLocation(), "");
+                }
                 winExpr.setExprList(newExprList);
             }
         } else if (FunctionMapUtil.isSql92AggregateFunction(signature)) {
-            List<Expression> newExprList = extractExpressions(winExpr.getExprList(), winExpr.getExprList().size(),
-                    winExpr.getSourceLocation());
+            List<Expression> newExprList = extractExpressions(winExpr.getExprList(), winExpr.getExprList().size());
+            if (newExprList == null) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, winExpr.getSourceLocation(), "");
+            }
             winExpr.setExprList(newExprList);
         }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
index a5cd5fa..8713106 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -300,24 +300,26 @@
      * @param op
      *            the operator to consider.
      * @return true if the operator can be moved, false if the operator cannot be moved.
-     * @throws AlgebricksException
      */
     public static boolean isMovable(ILogicalOperator op) {
         Object annotation = op.getAnnotations().get(MOVABLE);
-        if (annotation == null) {
-            // Can't move nonPures!
-            if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+        if (annotation != null) {
+            return (Boolean) annotation;
+        }
+        switch (op.getOperatorTag()) {
+            case ASSIGN:
+                // Can't move nonPures!
                 AssignOperator assign = (AssignOperator) op;
                 for (Mutable<ILogicalExpression> expr : assign.getExpressions()) {
                     if (containsNonpureCall(expr.getValue())) {
                         return false;
                     }
                 }
-            }
-            return true;
+                break;
+            case WINDOW:
+                return false;
         }
-        Boolean movable = (Boolean) annotation;
-        return movable;
+        return true;
     }
 
     private static boolean containsNonpureCall(ILogicalExpression expr) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantWindowOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantWindowOperatorsRule.java
new file mode 100644
index 0000000..9ad071f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantWindowOperatorsRule.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Removes redundant window operators
+ * <ol>
+ * <li>
+*  Empty window operator: {@code window [] <- []} with nested plan that does not produce any variables
+ * </li>
+ * </ol>
+ */
+public class RemoveRedundantWindowOperatorsRule implements IAlgebraicRewriteRule {
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.WINDOW) {
+            return false;
+        }
+        // fail fast to avoid traversing nested plans
+        WindowOperator winOp = (WindowOperator) op;
+        if (!winOp.getVariables().isEmpty()) {
+            return false;
+        }
+        Set<LogicalVariable> producedVars = new HashSet<>();
+        VariableUtilities.getProducedVariables(op, producedVars);
+        if (!producedVars.isEmpty()) {
+            return false;
+        }
+        opRef.setValue(op.getInputs().get(0).getValue());
+        return true;
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
index 062bb2f..eba3c91 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
@@ -45,6 +45,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
@@ -53,12 +54,12 @@
  */
 public class RemoveUnusedAssignAndAggregateRule implements IAlgebraicRewriteRule {
 
-    // Keep the variables that are produced by ASSIGN, UNNEST, AGGREGATE, UNION,
+    // Keep the variables that are produced by ASSIGN, UNNEST, AGGREGATE, UNION, WINDOW,
     // and GROUP operators.
     private Map<Mutable<ILogicalOperator>, Set<LogicalVariable>> assignedVarMap = new LinkedHashMap<>();
     private Set<LogicalVariable> assignedVarSet = new HashSet<>();
 
-    // Keep the variables that are used after ASSIGN, UNNEST, AGGREGATE, UNION,
+    // Keep the variables that are used after ASSIGN, UNNEST, AGGREGATE, UNION, WINDOW,
     // and GROUP operators.
     private Map<Mutable<ILogicalOperator>, Set<LogicalVariable>> accumulatedUsedVarFromRootMap = new LinkedHashMap<>();
 
@@ -236,6 +237,13 @@
                 }
                 return groupByOp.getGroupByList().size() + groupByOp.getNestedPlans().size()
                         + groupByOp.getDecorList().size();
+            case WINDOW:
+                WindowOperator winOp = (WindowOperator) op;
+                if (removeUnusedVarsAndExprs(toRemove, winOp.getVariables(), winOp.getExpressions())) {
+                    context.computeAndSetTypeEnvironmentForOperator(winOp);
+                    isTransformed = true;
+                }
+                return winOp.getVariables().size() + winOp.getNestedPlans().size();
             default:
                 break;
         }
@@ -371,7 +379,10 @@
                     }
                 }
                 break;
-            default:
+            case WINDOW:
+                WindowOperator winOp = (WindowOperator) op;
+                assignVarsSetInThisOp.addAll(winOp.getVariables());
+                targetOpFound = true;
                 break;
         }
 

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

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

Change in asterixdb[master]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/122/ (4/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10438/ (11/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4965/ (13/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


[NO ISSUE][COMP] Window operator compiler fixes

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

Details:
- Make window operator non-movable
- Add optimizer rule to remove empty window operators
- Consider window operators in RemoveUnusedAssignAndAggregateRule
- Propagate source location to constant expressions

Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3120
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/pg_win/pg_win_01.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/win_opt_01/win_opt_01_8.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
A asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_8.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_opt_01/win_opt_01.8.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_opt_01/win_opt_01.8.adm
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
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/SqlppWindowAggregationSugarVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
M hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
A hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantWindowOperatorsRule.java
M hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
15 files changed, 234 insertions(+), 35 deletions(-)

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

Objections:
  Jenkins: Violations found



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 80469a7..31be802 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -94,6 +94,7 @@
 import org.apache.hyracks.algebricks.rewriter.rules.ComplexUnnestToProductRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
+import org.apache.hyracks.algebricks.rewriter.rules.RemoveRedundantWindowOperatorsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ReuseWindowAggregateRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateWindowOperatorsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.CopyLimitDownRule;
@@ -296,6 +297,7 @@
         // Window operator consolidation rules
         consolidation.add(new ConsolidateWindowOperatorsRule());
         consolidation.add(new ReuseWindowAggregateRule());
+        consolidation.add(new RemoveRedundantWindowOperatorsRule());
         consolidation.add(new RemoveRedundantVariablesRule());
         return consolidation;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index e3745f7..61c525e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -714,8 +714,9 @@
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME));
         fldAccess.setSourceLocation(sourceLoc);
         fldAccess.getArguments().add(new MutableObject<>(p.first));
-        ILogicalExpression faExpr =
+        ConstantExpression faExpr =
                 new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent().getValue())));
+        faExpr.setSourceLocation(sourceLoc);
         fldAccess.getArguments().add(new MutableObject<>(faExpr));
         AssignOperator a = new AssignOperator(v, new MutableObject<>(fldAccess));
         a.getInputs().add(p.second);
@@ -822,8 +823,11 @@
                     break;
                 case LITERAL_EXPRESSION:
                     LiteralExpr val = (LiteralExpr) expr;
-                    args.add(new MutableObject<>(new ConstantExpression(
-                            new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue())))));
+                    AsterixConstantValue cValue =
+                            new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()));
+                    ConstantExpression cExpr = new ConstantExpression(cValue);
+                    cExpr.setSourceLocation(expr.getSourceLocation());
+                    args.add(new MutableObject<>(cExpr));
                     break;
                 default:
                     Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(expr, topOp);
@@ -1133,10 +1137,13 @@
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+        SourceLocation sourceLoc = l.getSourceLocation();
         LogicalVariable var = context.newVar();
-        AssignOperator a = new AssignOperator(var, new MutableObject<>(
-                new ConstantExpression(new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
-        a.setSourceLocation(l.getSourceLocation());
+        AsterixConstantValue cValue = new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue()));
+        ConstantExpression cExpr = new ConstantExpression(cValue);
+        cExpr.setSourceLocation(sourceLoc);
+        AssignOperator a = new AssignOperator(var, new MutableObject<>(cExpr));
+        a.setSourceLocation(sourceLoc);
         if (tupSource != null) {
             a.getInputs().add(tupSource);
         }
@@ -1295,9 +1302,10 @@
             // condition() -> not(if-missing-or-null(condition(), false))
 
             List<Mutable<ILogicalExpression>> ifMissingOrNullArgs = new ArrayList<>(2);
+            ConstantExpression eFalse = new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
+            eFalse.setSourceLocation(sourceLoc);
             ifMissingOrNullArgs.add(new MutableObject<>(eo2.first));
-            ifMissingOrNullArgs
-                    .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE))));
+            ifMissingOrNullArgs.add(new MutableObject<>(eFalse));
 
             List<Mutable<ILogicalExpression>> notArgs = new ArrayList<>(1);
             ScalarFunctionCallExpression ifMissinOrNullExpr = new ScalarFunctionCallExpression(
@@ -1560,8 +1568,11 @@
                 return new Pair<>(varRefExpr, topOpRef);
             case LITERAL_EXPRESSION:
                 LiteralExpr val = (LiteralExpr) expr;
-                return new Pair<>(new ConstantExpression(
-                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOpRef);
+                AsterixConstantValue cValue =
+                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()));
+                ConstantExpression cExpr = new ConstantExpression(cValue);
+                cExpr.setSourceLocation(sourceLoc);
+                return new Pair<>(cExpr, topOpRef);
             default:
                 if (expressionNeedsNoNesting(expr)) {
                     Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOpRef);
@@ -1876,9 +1887,10 @@
         count.setSourceLocation(sourceLoc);
         AbstractFunctionCallExpression comparison = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(not ? BuiltinFunctions.EQ : BuiltinFunctions.NEQ));
+        ConstantExpression eZero = new ConstantExpression(new AsterixConstantValue(new AInt64(0L)));
+        eZero.setSourceLocation(sourceLoc);
         comparison.getArguments().add(new MutableObject<>(count));
-        comparison.getArguments()
-                .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(0L)))));
+        comparison.getArguments().add(new MutableObject<>(eZero));
         comparison.setSourceLocation(sourceLoc);
         AssignOperator a = new AssignOperator(v1, new MutableObject<>(comparison));
         a.setSourceLocation(sourceLoc);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/pg_win/pg_win_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/pg_win/pg_win_01.sqlpp
new file mode 100644
index 0000000..4d7dc92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/pg_win/pg_win_01.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 removal of unused window operator
+ * Expected Res : SUCCESS (no window operators in the optimized plan)
+ */
+
+/* pg_win.16 */
+
+FROM (
+  FROM (SELECT * FROM range(1, 10) four WHERE FALSE) s
+  SELECT count(*) OVER (PARTITION BY four) AS `count`
+) t
+SELECT count(*) AS `cnt`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/win_opt_01/win_opt_01_8.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/win_opt_01/win_opt_01_8.sqlpp
new file mode 100644
index 0000000..fff3095
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/win_opt_01/win_opt_01_8.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 removal of unused window operator
+ * Expected Res : SUCCESS (no window operators in the optimized plan)
+ */
+
+FROM (
+  FROM range(1, 10) x
+  SELECT VALUE row_number() OVER (ORDER BY x)
+) y
+SELECT VALUE COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
new file mode 100644
index 0000000..1fee6e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/pg_win/pg_win_01.plan
@@ -0,0 +1,9 @@
+-- DISTRIBUTE_RESULT  |LOCAL|
+  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+    -- STREAM_PROJECT  |LOCAL|
+      -- ASSIGN  |LOCAL|
+        -- AGGREGATE  |LOCAL|
+          -- AGGREGATE  |LOCAL|
+            -- UNNEST  |UNPARTITIONED|
+              -- STREAM_SELECT  |UNPARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_8.plan
new file mode 100644
index 0000000..28dbe05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/win_opt_01/win_opt_01_8.plan
@@ -0,0 +1,6 @@
+-- DISTRIBUTE_RESULT  |LOCAL|
+  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+    -- AGGREGATE  |LOCAL|
+      -- AGGREGATE  |LOCAL|
+        -- UNNEST  |UNPARTITIONED|
+          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_opt_01/win_opt_01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_opt_01/win_opt_01.8.query.sqlpp
new file mode 100644
index 0000000..fff3095
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_opt_01/win_opt_01.8.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 removal of unused window operator
+ * Expected Res : SUCCESS (no window operators in the optimized plan)
+ */
+
+FROM (
+  FROM range(1, 10) x
+  SELECT VALUE row_number() OVER (ORDER BY x)
+) y
+SELECT VALUE COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_opt_01/win_opt_01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_opt_01/win_opt_01.8.adm
new file mode 100644
index 0000000..9a03714
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_opt_01/win_opt_01.8.adm
@@ -0,0 +1 @@
+10
\ No newline at end of file
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
index 3606b12..4441d65 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
@@ -25,7 +25,6 @@
 import java.util.List;
 
 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;
@@ -37,11 +36,10 @@
 import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Base class for visitors that extract expressions into LET clauses.
- * Subclasses should call {@link #extractExpressions(List, int, SourceLocation)} to perform the extraction.
+ * Subclasses should call {@link #extractExpressions(List, int)} to perform the extraction.
  */
 abstract class AbstractSqlppExpressionExtractionVisitor extends AbstractSqlppSimpleExpressionVisitor {
 
@@ -111,11 +109,10 @@
         fromBindingList.clear();
     }
 
-    List<Expression> extractExpressions(List<Expression> exprList, int limit, SourceLocation sourceLocation)
-            throws CompilationException {
+    List<Expression> extractExpressions(List<Expression> exprList, int limit) {
         List<Pair<Expression, VarIdentifier>> outLetList = stack.peek();
         if (outLetList == null) {
-            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLocation);
+            return null;
         }
         int n = exprList.size();
         List<Expression> newExprList = new ArrayList<>(n);
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 6e14e74..130c8ac 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
@@ -169,7 +169,7 @@
             ident = me.getValue();
         }
         if (ident == null) {
-            throw new CompilationException(ErrorCode.COMPILATION_ERROR, usedVar.getSourceLocation());
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, usedVar.getSourceLocation(), "");
         }
         return ident;
     }
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 910552a..a060d57 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
@@ -27,7 +27,6 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
index b1c2d60..3c5032a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
@@ -66,13 +66,17 @@
         if (winfi != null) {
             if (BuiltinFunctions.windowFunctionHasProperty(winfi,
                     BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
-                List<Expression> newExprList =
-                        extractExpressions(winExpr.getExprList(), 1, winExpr.getSourceLocation());
+                List<Expression> newExprList = extractExpressions(winExpr.getExprList(), 1);
+                if (newExprList == null) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, winExpr.getSourceLocation(), "");
+                }
                 winExpr.setExprList(newExprList);
             }
         } else if (FunctionMapUtil.isSql92AggregateFunction(signature)) {
-            List<Expression> newExprList = extractExpressions(winExpr.getExprList(), winExpr.getExprList().size(),
-                    winExpr.getSourceLocation());
+            List<Expression> newExprList = extractExpressions(winExpr.getExprList(), winExpr.getExprList().size());
+            if (newExprList == null) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, winExpr.getSourceLocation(), "");
+            }
             winExpr.setExprList(newExprList);
         }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
index a5cd5fa..8713106 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -300,24 +300,26 @@
      * @param op
      *            the operator to consider.
      * @return true if the operator can be moved, false if the operator cannot be moved.
-     * @throws AlgebricksException
      */
     public static boolean isMovable(ILogicalOperator op) {
         Object annotation = op.getAnnotations().get(MOVABLE);
-        if (annotation == null) {
-            // Can't move nonPures!
-            if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+        if (annotation != null) {
+            return (Boolean) annotation;
+        }
+        switch (op.getOperatorTag()) {
+            case ASSIGN:
+                // Can't move nonPures!
                 AssignOperator assign = (AssignOperator) op;
                 for (Mutable<ILogicalExpression> expr : assign.getExpressions()) {
                     if (containsNonpureCall(expr.getValue())) {
                         return false;
                     }
                 }
-            }
-            return true;
+                break;
+            case WINDOW:
+                return false;
         }
-        Boolean movable = (Boolean) annotation;
-        return movable;
+        return true;
     }
 
     private static boolean containsNonpureCall(ILogicalExpression expr) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantWindowOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantWindowOperatorsRule.java
new file mode 100644
index 0000000..9ad071f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantWindowOperatorsRule.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.rewriter.rules;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Removes redundant window operators
+ * <ol>
+ * <li>
+*  Empty window operator: {@code window [] <- []} with nested plan that does not produce any variables
+ * </li>
+ * </ol>
+ */
+public class RemoveRedundantWindowOperatorsRule implements IAlgebraicRewriteRule {
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.WINDOW) {
+            return false;
+        }
+        // fail fast to avoid traversing nested plans
+        WindowOperator winOp = (WindowOperator) op;
+        if (!winOp.getVariables().isEmpty()) {
+            return false;
+        }
+        Set<LogicalVariable> producedVars = new HashSet<>();
+        VariableUtilities.getProducedVariables(op, producedVars);
+        if (!producedVars.isEmpty()) {
+            return false;
+        }
+        opRef.setValue(op.getInputs().get(0).getValue());
+        return true;
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
index 062bb2f..eba3c91 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveUnusedAssignAndAggregateRule.java
@@ -45,6 +45,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
@@ -53,12 +54,12 @@
  */
 public class RemoveUnusedAssignAndAggregateRule implements IAlgebraicRewriteRule {
 
-    // Keep the variables that are produced by ASSIGN, UNNEST, AGGREGATE, UNION,
+    // Keep the variables that are produced by ASSIGN, UNNEST, AGGREGATE, UNION, WINDOW,
     // and GROUP operators.
     private Map<Mutable<ILogicalOperator>, Set<LogicalVariable>> assignedVarMap = new LinkedHashMap<>();
     private Set<LogicalVariable> assignedVarSet = new HashSet<>();
 
-    // Keep the variables that are used after ASSIGN, UNNEST, AGGREGATE, UNION,
+    // Keep the variables that are used after ASSIGN, UNNEST, AGGREGATE, UNION, WINDOW,
     // and GROUP operators.
     private Map<Mutable<ILogicalOperator>, Set<LogicalVariable>> accumulatedUsedVarFromRootMap = new LinkedHashMap<>();
 
@@ -236,6 +237,13 @@
                 }
                 return groupByOp.getGroupByList().size() + groupByOp.getNestedPlans().size()
                         + groupByOp.getDecorList().size();
+            case WINDOW:
+                WindowOperator winOp = (WindowOperator) op;
+                if (removeUnusedVarsAndExprs(toRemove, winOp.getVariables(), winOp.getExpressions())) {
+                    context.computeAndSetTypeEnvironmentForOperator(winOp);
+                    isTransformed = true;
+                }
+                return winOp.getVariables().size() + winOp.getNestedPlans().size();
             default:
                 break;
         }
@@ -371,7 +379,10 @@
                     }
                 }
                 break;
-            default:
+            case WINDOW:
+                WindowOperator winOp = (WindowOperator) op;
+                assignVarsSetInThisOp.addAll(winOp.getVariables());
+                targetOpFound = true;
                 break;
         }
 

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4956/ (12/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4868/ (1/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5340/ (2/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5179/ (8/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
Gerrit-PatchSet: 1
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
Gerrit-PatchSet: 1
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4922/ (10/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8905/ (3/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1: Contrib+1

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
Gerrit-PatchSet: 1
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2966/ (6/14)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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]: [NO ISSUE][COMP] Window operator compiler fixes

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

Change subject: [NO ISSUE][COMP] Window operator compiler fixes
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8e15620220844a07857656256a89c23d243ff2b2
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