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/10 22:53:05 UTC

Change in asterixdb[master]: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

Dmitry Lychagin has uploaded a new change for review.

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................

[ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()

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

Details:
- Implement CUME_DIST() and RATIO_TO_REPORT() window functions
- Add internal WIN_PARTITION_LENGTH() window function
- Fixed typos in javadocs

Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm
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/SqlppWindowAggregationSugarVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java
M hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
31 files changed, 634 insertions(+), 203 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index 4d1d532..2a42da1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -372,7 +372,8 @@
                             expr.getExpressionTag());
                 }
                 AbstractFunctionCallExpression callExpr = (AbstractFunctionCallExpression) expr;
-                if (BuiltinFunctions.windowFunctionRequiresMaterialization(callExpr.getFunctionIdentifier())) {
+                if (BuiltinFunctions.windowFunctionHasProperty(callExpr.getFunctionIdentifier(),
+                        BuiltinFunctions.WindowFunctionProperty.MATERIALIZE_PARTITION)) {
                     partitionMaterialization = true;
                     break;
                 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index a99f9d4..d8ea74e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -785,7 +785,7 @@
             } else if (projection.hasName()) {
                 fieldBindings.add(getFieldBinding(projection, fieldNames));
             } else {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, projection.getSourceLocation());
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, projection.getSourceLocation(), "");
             }
         }
         if (!fieldBindings.isEmpty()) {
@@ -1037,9 +1037,16 @@
 
         FunctionSignature fs = winExpr.getFunctionSignature();
         FunctionIdentifier fi = getBuiltinFunctionIdentifier(fs.getName(), fs.getArity());
+        if (fi == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "");
+        }
         boolean isWin = BuiltinFunctions.isWindowFunction(fi);
-        boolean isWinAgg = isWin && BuiltinFunctions.windowFunctionWithListArg(fi);
-        boolean supportsFrameClause = isWin && BuiltinFunctions.windowFunctionSupportsFrameClause(fi);
+        boolean isWinAgg = isWin
+                && BuiltinFunctions.windowFunctionHasProperty(fi, BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG);
+        boolean prohibitOrderClause = isWin && BuiltinFunctions.windowFunctionHasProperty(fi,
+                BuiltinFunctions.WindowFunctionProperty.NO_ORDER_CLAUSE);
+        boolean prohibitFrameClause = isWin && BuiltinFunctions.windowFunctionHasProperty(fi,
+                BuiltinFunctions.WindowFunctionProperty.NO_FRAME_CLAUSE);
 
         Mutable<ILogicalOperator> currentOpRef = tupSource;
 
@@ -1065,6 +1072,9 @@
         int frameExcludeNotStartIdx = -1;
 
         if (winExpr.hasOrderByList()) {
+            if (prohibitOrderClause) {
+                throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_WINDOW_ORDERBY, sourceLoc);
+            }
             List<Expression> orderExprList = winExpr.getOrderbyList();
             List<OrderbyClause.OrderModifier> orderModifierList = winExpr.getOrderbyModifierList();
             orderExprCount = orderExprList.size();
@@ -1092,7 +1102,7 @@
         int winFrameMaxOjbects = -1;
 
         if (winExpr.hasFrameDefinition()) {
-            if (isWin && !supportsFrameClause) {
+            if (prohibitFrameClause) {
                 throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_WINDOW_FRAME, sourceLoc);
             }
             winFrameMode = winExpr.getFrameMode();
@@ -1104,17 +1114,21 @@
             if (!isValidWindowFrameDefinition(winFrameMode, winFrameStartKind, winFrameEndKind, orderExprCount)) {
                 throw new CompilationException(ErrorCode.COMPILATION_INVALID_WINDOW_FRAME, sourceLoc);
             }
-        } else if (!isWin || supportsFrameClause) {
+        } else if (!prohibitFrameClause) {
             winFrameMode = WindowExpression.FrameMode.RANGE;
             winFrameStartKind = WindowExpression.FrameBoundaryKind.UNBOUNDED_PRECEDING;
             winFrameEndKind = WindowExpression.FrameBoundaryKind.CURRENT_ROW;
             winFrameExclusionKind = WindowExpression.FrameExclusionKind.NO_OTHERS;
         }
 
-        FunctionIdentifier winAggFunc = null;
-        FunctionIdentifier winAggDefaultIfNullFunc = null;
-        Expression winAggDefaultExpr = null;
+        boolean makeRunningAgg = false, makeNestedAgg = false;
+        FunctionIdentifier runningAggFunc = null, nestedAggFunc = null, winResultFunc = null, postWinResultFunc = null;
+        Expression postWinExpr = null;
+        List<Expression> nestedAggArgs = null;
+        boolean postWinResultArgsReverse = false;
+
         if (isWinAgg) {
+            makeNestedAgg = true;
             if (BuiltinFunctions.LEAD_IMPL.equals(fi) || BuiltinFunctions.LAG_IMPL.equals(fi)) {
                 int argCount = fargs.size();
                 if (argCount < 1 || argCount > 3) {
@@ -1131,20 +1145,20 @@
                 // then use local-first-element() because it returns SYSTEM_NULL if the list is empty,
                 // otherwise (no default expression) use first-element() which returns NULL if the list is empty
                 if (argCount > 2) {
-                    winAggFunc = BuiltinFunctions.LOCAL_FIRST_ELEMENT;
-                    winAggDefaultIfNullFunc = BuiltinFunctions.IF_SYSTEM_NULL;
-                    winAggDefaultExpr = fargs.get(2);
+                    nestedAggFunc = BuiltinFunctions.SCALAR_LOCAL_FIRST_ELEMENT;
+                    postWinResultFunc = BuiltinFunctions.IF_SYSTEM_NULL;
+                    postWinExpr = fargs.get(2);
                 } else {
-                    winAggFunc = BuiltinFunctions.FIRST_ELEMENT;
+                    nestedAggFunc = BuiltinFunctions.SCALAR_FIRST_ELEMENT;
                 }
                 winFrameMaxOjbects = 1;
             } else if (BuiltinFunctions.FIRST_VALUE_IMPL.equals(fi)) {
-                winAggFunc = BuiltinFunctions.FIRST_ELEMENT;
+                nestedAggFunc = BuiltinFunctions.SCALAR_FIRST_ELEMENT;
                 winFrameMaxOjbects = 1;
             } else if (BuiltinFunctions.LAST_VALUE_IMPL.equals(fi)) {
-                winAggFunc = BuiltinFunctions.LAST_ELEMENT;
+                nestedAggFunc = BuiltinFunctions.SCALAR_LAST_ELEMENT;
             } else if (BuiltinFunctions.NTH_VALUE_IMPL.equals(fi)) {
-                winAggFunc = BuiltinFunctions.FIRST_ELEMENT;
+                nestedAggFunc = BuiltinFunctions.SCALAR_FIRST_ELEMENT;
                 winFrameMaxOjbects = 1;
                 OperatorExpr opExpr = new OperatorExpr();
                 opExpr.addOperand(fargs.get(1));
@@ -1152,9 +1166,36 @@
                 opExpr.addOperand(new LiteralExpr(new IntegerLiteral(1)));
                 opExpr.setSourceLocation(sourceLoc);
                 winFrameOffsetExpr = opExpr;
+            } else if (BuiltinFunctions.RATIO_TO_REPORT_IMPL.equals(fi)) {
+                // ratio_to_report(x) over (...) --> x / sum(x) over (...)
+                nestedAggFunc = BuiltinFunctions.SCALAR_SQL_SUM;
+                postWinResultFunc = BuiltinFunctions.NUMERIC_DIVIDE;
+                postWinExpr = fargs.get(1);
+                postWinResultArgsReverse = true;
             } else {
                 throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, fi.getName());
             }
+            nestedAggArgs = mkSingletonArrayList(fargs.get(0));
+        } else if (isWin) {
+            makeRunningAgg = true;
+            if (BuiltinFunctions.CUME_DIST_IMPL.equals(fi)) {
+                winFrameMode = WindowExpression.FrameMode.RANGE;
+                winFrameStartKind = WindowExpression.FrameBoundaryKind.UNBOUNDED_PRECEDING;
+                winFrameEndKind = WindowExpression.FrameBoundaryKind.CURRENT_ROW;
+                winFrameExclusionKind = WindowExpression.FrameExclusionKind.NO_OTHERS;
+
+                makeNestedAgg = true;
+                runningAggFunc = BuiltinFunctions.WIN_PARTITION_LENGTH_IMPL;
+                nestedAggFunc = BuiltinFunctions.SCALAR_COUNT;
+                nestedAggArgs = mkSingletonArrayList((Expression) SqlppRewriteUtil.deepCopy(winExpr.getWindowVar()));
+                winResultFunc = BuiltinFunctions.NUMERIC_DIVIDE;
+            } else {
+                runningAggFunc = fi;
+            }
+        } else { // regular aggregate
+            makeNestedAgg = true;
+            nestedAggFunc = fi;
+            nestedAggArgs = fargs;
         }
 
         if (winFrameMode != null) {
@@ -1214,54 +1255,15 @@
                 winFrameMaxOjbects);
         winOp.setSourceLocation(sourceLoc);
 
-        AbstractLogicalExpression resultExpr;
+        LogicalVariable runningAggResultVar = null, nestedAggResultVar = null;
 
-        if (isWin && !isWinAgg) {
-            CallExpr callExpr = new CallExpr(new FunctionSignature(fi), fargs);
-            Pair<ILogicalOperator, LogicalVariable> callExprResult = callExpr.accept(this, currentOpRef);
-            ILogicalOperator op = callExprResult.first;
-            if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
-            }
-            AssignOperator assignOp = (AssignOperator) op;
-            List<LogicalVariable> assignVars = assignOp.getVariables();
-            if (assignVars.size() != 1) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
-            }
-            List<Mutable<ILogicalExpression>> assignExprs = assignOp.getExpressions();
-            if (assignExprs.size() != 1) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
-            }
-            ILogicalExpression assignExpr = assignExprs.get(0).getValue();
-            if (assignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc);
-            }
-            AbstractFunctionCallExpression fcallExpr = (AbstractFunctionCallExpression) assignExpr;
-            if (fcallExpr.getKind() != AbstractFunctionCallExpression.FunctionKind.STATEFUL) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc);
-            }
-            if (BuiltinFunctions.windowFunctionRequiresOrderArgs(fi)) {
-                for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExprListOut) {
-                    fcallExpr.getArguments().add(new MutableObject<>(p.second.getValue().cloneExpression()));
-                }
-            }
-
-            winOp.getInputs().add(assignOp.getInputs().get(0));
-            winOp.getVariables().addAll(assignVars);
-            winOp.getExpressions().addAll(assignExprs);
-
-            resultExpr = new VariableReferenceExpression(assignVars.get(0));
-            resultExpr.setSourceLocation(sourceLoc);
-            currentOpRef = new MutableObject<>(winOp);
-        } else {
+        if (makeNestedAgg) {
             LogicalVariable windowRecordVar = context.newVar();
             ILogicalExpression windowRecordConstr =
                     createRecordConstructor(winExpr.getWindowFieldList(), currentOpRef, sourceLoc);
             AssignOperator assignOp = new AssignOperator(windowRecordVar, new MutableObject<>(windowRecordConstr));
             assignOp.getInputs().add(currentOpRef);
             assignOp.setSourceLocation(sourceLoc);
-
-            winOp.getInputs().add(new MutableObject<>(assignOp));
 
             NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(winOp));
             ntsOp.setSourceLocation(sourceLoc);
@@ -1280,62 +1282,93 @@
 
             context.setVar(winExpr.getWindowVar(), windowVar);
 
-            if (isWinAgg) {
-                Expression listArgExpr = fargs.get(0);
-                Pair<ILogicalOperator, LogicalVariable> listArgExprResult =
-                        listArgExpr.accept(this, new MutableObject<>(aggOp));
-                VariableReferenceExpression listArgVarRef = new VariableReferenceExpression(listArgExprResult.second);
-                listArgVarRef.setSourceLocation(sourceLoc);
+            CallExpr callExpr = new CallExpr(new FunctionSignature(nestedAggFunc), nestedAggArgs);
+            Pair<ILogicalOperator, LogicalVariable> exprResult = callExpr.accept(this, new MutableObject<>(aggOp));
+            winOp.getNestedPlans().add(new ALogicalPlanImpl(new MutableObject<>(exprResult.first)));
 
-                LogicalVariable unnestVar = context.newVar();
-                UnnestingFunctionCallExpression unnestExpr = new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
-                        mkSingletonArrayList(new MutableObject<>(listArgVarRef)));
-                unnestExpr.setSourceLocation(sourceLoc);
-                UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(unnestExpr));
-                unnestOp.setSourceLocation(sourceLoc);
-                unnestOp.getInputs().add(new MutableObject<>(listArgExprResult.first));
-
-                VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
-                unnestVarRef.setSourceLocation(sourceLoc);
-
-                AggregateFunctionCallExpression winAggCall = BuiltinFunctions.makeAggregateFunctionExpression(
-                        winAggFunc, mkSingletonArrayList(new MutableObject<>(unnestVarRef)));
-                winAggCall.setSourceLocation(sourceLoc);
-                LogicalVariable winAggVar = context.newVar();
-                AggregateOperator winAggOp = new AggregateOperator(mkSingletonArrayList(winAggVar),
-                        mkSingletonArrayList(new MutableObject<>(winAggCall)));
-                winAggOp.getInputs().add(new MutableObject<>(unnestOp));
-                winAggOp.setSourceLocation(sourceLoc);
-
-                winOp.getNestedPlans().add(new ALogicalPlanImpl(new MutableObject<>(winAggOp)));
-                currentOpRef = new MutableObject<>(winOp);
-
-                resultExpr = new VariableReferenceExpression(winAggVar);
-                resultExpr.setSourceLocation(sourceLoc);
-
-                if (winAggDefaultExpr != null) {
-                    Pair<ILogicalOperator, LogicalVariable> winAggDefaultExprResult =
-                            winAggDefaultExpr.accept(this, currentOpRef);
-                    VariableReferenceExpression winAggDefaultVarRef =
-                            new VariableReferenceExpression(winAggDefaultExprResult.second);
-                    winAggDefaultVarRef.setSourceLocation(sourceLoc);
-                    AbstractFunctionCallExpression ifNullExpr =
-                            createFunctionCallExpression(winAggDefaultIfNullFunc, sourceLoc);
-                    ifNullExpr.getArguments().add(new MutableObject<>(resultExpr));
-                    ifNullExpr.getArguments().add(new MutableObject<>(winAggDefaultVarRef));
-                    resultExpr = ifNullExpr;
-                    currentOpRef = new MutableObject<>(winAggDefaultExprResult.first);
-                }
-            } else {
-                CallExpr callExpr = new CallExpr(new FunctionSignature(fi), fargs);
-                Pair<ILogicalOperator, LogicalVariable> exprResult = callExpr.accept(this, new MutableObject<>(aggOp));
-                winOp.getNestedPlans().add(new ALogicalPlanImpl(new MutableObject<>(exprResult.first)));
-                resultExpr = new VariableReferenceExpression(exprResult.second);
-                resultExpr.setSourceLocation(sourceLoc);
-                currentOpRef = new MutableObject<>(winOp);
-            }
+            currentOpRef = new MutableObject<>(assignOp);
+            nestedAggResultVar = exprResult.second;
         }
+
+        if (makeRunningAgg) {
+            CallExpr callExpr = new CallExpr(new FunctionSignature(runningAggFunc), fargs);
+            Pair<ILogicalOperator, LogicalVariable> callExprResult = callExpr.accept(this, currentOpRef);
+            ILogicalOperator op = callExprResult.first;
+            if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+            }
+            AssignOperator assignOp = (AssignOperator) op;
+            List<LogicalVariable> assignVars = assignOp.getVariables();
+            if (assignVars.size() != 1) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+            }
+            List<Mutable<ILogicalExpression>> assignExprs = assignOp.getExpressions();
+            if (assignExprs.size() != 1) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+            }
+            ILogicalExpression assignExpr = assignExprs.get(0).getValue();
+            if (assignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+            }
+            AbstractFunctionCallExpression fcallExpr = (AbstractFunctionCallExpression) assignExpr;
+            if (fcallExpr.getKind() != AbstractFunctionCallExpression.FunctionKind.STATEFUL) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, fcallExpr.getKind());
+            }
+            if (BuiltinFunctions.windowFunctionHasProperty(fi,
+                    BuiltinFunctions.WindowFunctionProperty.INJECT_ORDER_ARGS)) {
+                for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExprListOut) {
+                    fcallExpr.getArguments().add(new MutableObject<>(p.second.getValue().cloneExpression()));
+                }
+            }
+
+            winOp.getVariables().addAll(assignVars);
+            winOp.getExpressions().addAll(assignExprs);
+
+            currentOpRef = new MutableObject<>(assignOp.getInputs().get(0).getValue());
+            runningAggResultVar = assignVars.get(0);
+        }
+
+        winOp.getInputs().add(currentOpRef);
+        currentOpRef = new MutableObject<>(winOp);
+
+        AbstractLogicalExpression resultExpr;
+        if (makeRunningAgg && makeNestedAgg) {
+            VariableReferenceExpression runningAggResultVarRef = new VariableReferenceExpression(runningAggResultVar);
+            runningAggResultVarRef.setSourceLocation(sourceLoc);
+            VariableReferenceExpression nestedAggResultVarRef = new VariableReferenceExpression(nestedAggResultVar);
+            nestedAggResultVarRef.setSourceLocation(sourceLoc);
+            AbstractFunctionCallExpression resultCallExpr = createFunctionCallExpression(winResultFunc, sourceLoc);
+            resultCallExpr.getArguments().add(new MutableObject<>(nestedAggResultVarRef));
+            resultCallExpr.getArguments().add(new MutableObject<>(runningAggResultVarRef));
+            resultExpr = resultCallExpr;
+        } else if (makeRunningAgg) {
+            resultExpr = new VariableReferenceExpression(runningAggResultVar);
+            resultExpr.setSourceLocation(sourceLoc);
+        } else if (makeNestedAgg) {
+            resultExpr = new VariableReferenceExpression(nestedAggResultVar);
+            resultExpr.setSourceLocation(sourceLoc);
+        } else {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+        }
+
+        if (postWinExpr != null) {
+            Pair<ILogicalOperator, LogicalVariable> postWinExprResult = postWinExpr.accept(this, currentOpRef);
+            currentOpRef = new MutableObject<>(postWinExprResult.first);
+            VariableReferenceExpression postWinVarRef = new VariableReferenceExpression(postWinExprResult.second);
+            postWinVarRef.setSourceLocation(sourceLoc);
+            AbstractFunctionCallExpression postWinResultCallExpr =
+                    createFunctionCallExpression(postWinResultFunc, sourceLoc);
+            List<Mutable<ILogicalExpression>> postWinResultCallArgs = postWinResultCallExpr.getArguments();
+            if (!postWinResultArgsReverse) {
+                postWinResultCallArgs.add(new MutableObject<>(resultExpr));
+                postWinResultCallArgs.add(new MutableObject<>(postWinVarRef));
+            } else {
+                postWinResultCallArgs.add(new MutableObject<>(postWinVarRef));
+                postWinResultCallArgs.add(new MutableObject<>(resultExpr));
+            }
+            resultExpr = postWinResultCallExpr;
+        }
+
         // must return ASSIGN
         LogicalVariable resultVar = context.newVar();
         AssignOperator resultOp = new AssignOperator(resultVar, new MutableObject<>(resultExpr));
@@ -1530,7 +1563,8 @@
             }
             AbstractFunctionCallExpression valueExpr =
                     BuiltinFunctions.makeWindowFunctionExpression(fid, new ArrayList<>());
-            if (BuiltinFunctions.windowFunctionRequiresOrderArgs(valueExpr.getFunctionIdentifier())) {
+            if (BuiltinFunctions.windowFunctionHasProperty(valueExpr.getFunctionIdentifier(),
+                    BuiltinFunctions.WindowFunctionProperty.INJECT_ORDER_ARGS)) {
                 for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExprList) {
                     valueExpr.getArguments().add(new MutableObject<>(p.second.getValue().cloneExpression()));
                 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp
new file mode 100644
index 0000000..1e114b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 window function: cume_dist()
+ * Expected Res : SUCCESS
+ */
+
+FROM range(1, 5) x, range(1, 4) y
+SELECT x, y,
+  cume_dist() over (order by x) as `cume_dist`
+ORDER BY x, y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp
new file mode 100644
index 0000000..f5c4dad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 RATIO_TO_REPORT() without PARTITION BY
+ * Expected Res : SUCCESS
+ */
+
+FROM range(1, 4) x
+SELECT x,
+  round_half_to_even(ratio_to_report(x) over (), 2) as `ratio_to_report`
+ORDER BY ratio_to_report;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp
new file mode 100644
index 0000000..29a82ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 RATIO_TO_REPORT() with PARTITION BY
+ * Expected Res : SUCCESS
+ */
+
+FROM range(1, 4) x, range(1, 2) y
+SELECT x, y,
+  round_half_to_even(ratio_to_report(x) over (partition by y), 2) as `ratio_to_report`
+ORDER BY y, ratio_to_report;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp
new file mode 100644
index 0000000..6add05c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 RATIO_TO_REPORT() handling of NULL values
+ * Expected Res : SUCCESS
+ */
+
+ FROM range(0, 4) x, range(0, 1) y
+ SELECT x, y,
+   round_half_to_even(ratio_to_report(nullif(x,y)) over (partition by y), 2) as `ratio_to_report`
+ ORDER BY y, ratio_to_report;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.query.sqlpp
new file mode 100644
index 0000000..13293ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.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 RATIO_TO_REPORT() when total = 0
+ * Expected Res : SUCCESS
+ */
+
+FROM range(1, 4) x
+SELECT x,
+  round_half_to_even(ratio_to_report(x-x) over (), 2) as `ratio_to_report`
+ORDER BY ratio_to_report;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp
new file mode 100644
index 0000000..b539629
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 RATIO_TO_REPORT()
+ * Expected Res : FAILURE (unexpected ORDER BY)
+ */
+
+FROM range(1, 4) x, range(1, 2) y
+SELECT ratio_to_report(x) over (order by y)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm
new file mode 100644
index 0000000..85969e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm
@@ -0,0 +1,20 @@
+{ "x": 1, "y": 1, "cume_dist": 0.2 }
+{ "x": 1, "y": 2, "cume_dist": 0.2 }
+{ "x": 1, "y": 3, "cume_dist": 0.2 }
+{ "x": 1, "y": 4, "cume_dist": 0.2 }
+{ "x": 2, "y": 1, "cume_dist": 0.4 }
+{ "x": 2, "y": 2, "cume_dist": 0.4 }
+{ "x": 2, "y": 3, "cume_dist": 0.4 }
+{ "x": 2, "y": 4, "cume_dist": 0.4 }
+{ "x": 3, "y": 1, "cume_dist": 0.6 }
+{ "x": 3, "y": 2, "cume_dist": 0.6 }
+{ "x": 3, "y": 3, "cume_dist": 0.6 }
+{ "x": 3, "y": 4, "cume_dist": 0.6 }
+{ "x": 4, "y": 1, "cume_dist": 0.8 }
+{ "x": 4, "y": 2, "cume_dist": 0.8 }
+{ "x": 4, "y": 3, "cume_dist": 0.8 }
+{ "x": 4, "y": 4, "cume_dist": 0.8 }
+{ "x": 5, "y": 1, "cume_dist": 1.0 }
+{ "x": 5, "y": 2, "cume_dist": 1.0 }
+{ "x": 5, "y": 3, "cume_dist": 1.0 }
+{ "x": 5, "y": 4, "cume_dist": 1.0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm
new file mode 100644
index 0000000..0a6cca0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm
@@ -0,0 +1,4 @@
+{ "x": 1, "ratio_to_report": 0.1 }
+{ "x": 2, "ratio_to_report": 0.2 }
+{ "x": 3, "ratio_to_report": 0.3 }
+{ "x": 4, "ratio_to_report": 0.4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm
new file mode 100644
index 0000000..af6e7df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm
@@ -0,0 +1,8 @@
+{ "x": 1, "y": 1, "ratio_to_report": 0.1 }
+{ "x": 2, "y": 1, "ratio_to_report": 0.2 }
+{ "x": 3, "y": 1, "ratio_to_report": 0.3 }
+{ "x": 4, "y": 1, "ratio_to_report": 0.4 }
+{ "x": 1, "y": 2, "ratio_to_report": 0.1 }
+{ "x": 2, "y": 2, "ratio_to_report": 0.2 }
+{ "x": 3, "y": 2, "ratio_to_report": 0.3 }
+{ "x": 4, "y": 2, "ratio_to_report": 0.4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm
new file mode 100644
index 0000000..a6b9df4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm
@@ -0,0 +1,10 @@
+{ "x": 0, "y": 0, "ratio_to_report": null }
+{ "x": 1, "y": 0, "ratio_to_report": 0.1 }
+{ "x": 2, "y": 0, "ratio_to_report": 0.2 }
+{ "x": 3, "y": 0, "ratio_to_report": 0.3 }
+{ "x": 4, "y": 0, "ratio_to_report": 0.4 }
+{ "x": 1, "y": 1, "ratio_to_report": null }
+{ "x": 0, "y": 1, "ratio_to_report": 0.0 }
+{ "x": 2, "y": 1, "ratio_to_report": 0.22 }
+{ "x": 3, "y": 1, "ratio_to_report": 0.33 }
+{ "x": 4, "y": 1, "ratio_to_report": 0.44 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm
new file mode 100644
index 0000000..edd48d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm
@@ -0,0 +1,4 @@
+{ "x": 1, "ratio_to_report": null }
+{ "x": 2, "ratio_to_report": null }
+{ "x": 3, "ratio_to_report": null }
+{ "x": 4, "ratio_to_report": null }
\ No newline at end of file
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 3a857c1..dbd0d29 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -9219,6 +9219,11 @@
   </test-group>
   <test-group name="window">
     <test-case FilePath="window">
+      <compilation-unit name="cume_dist_01">
+        <output-dir compare="Text">cume_dist_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
       <compilation-unit name="dense_rank_01">
         <output-dir compare="Text">dense_rank_01</output-dir>
       </compilation-unit>
@@ -9256,6 +9261,17 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="window">
+      <compilation-unit name="ratio_to_report_01">
+        <output-dir compare="Text">ratio_to_report_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="ratio_to_report_02_negative">
+        <output-dir compare="Text">ratio_to_report_01</output-dir>
+        <expected-error>ASX1101: Unexpected ORDER BY clause in window expression</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
       <compilation-unit name="row_number_01">
         <output-dir compare="Text">row_number_01</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 3d63936..d8a21f1 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
@@ -186,6 +186,8 @@
     public static final int COMPILATION_INVALID_WINDOW_FRAME = 1098;
     public static final int COMPILATION_UNEXPECTED_WINDOW_FRAME = 1099;
     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;
 
     // 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 1ff4f64..b55f537 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -173,6 +173,8 @@
 1098 = Invalid window frame definition
 1099 = Unexpected window frame definition
 1100 = Unexpected window expression
+1101 = Unexpected ORDER BY clause in window expression
+1102 = Expected window or aggregate function, got: %1$s
 
 # Feed Errors
 3001 = Illegal state.
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 fa40d78..40bd163 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,6 +27,7 @@
 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;
@@ -36,6 +37,7 @@
 import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
 import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
+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;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -86,12 +88,17 @@
         FunctionIdentifier winfi = FunctionMapUtil.getInternalWindowFunction(signature);
         if (winfi != null) {
             winExpr.setFunctionSignature(new FunctionSignature(winfi));
-            if (BuiltinFunctions.windowFunctionWithListArg(winfi)) {
+            rewriteSpecificWindowFunctions(winfi, winExpr);
+            if (BuiltinFunctions.windowFunctionHasProperty(winfi,
+                    BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
                 wrapAggregationArguments(winExpr, 1);
             }
         } else if (FunctionMapUtil.isSql92AggregateFunction(signature)) {
             winExpr.setFunctionSignature(FunctionMapUtil.sql92ToCoreAggregateFunction(signature));
             wrapAggregationArguments(winExpr, winExpr.getExprList().size());
+        } else {
+            throw new CompilationException(ErrorCode.COMPILATION_EXPECTED_WINDOW_FUNCTION, winExpr.getSourceLocation(),
+                    signature.getName());
         }
 
         return super.visit(winExpr, arg);
@@ -135,4 +142,27 @@
             }
         }
     }
+
+    /**
+     * Apply rewritings for specific window functions:
+     * <ul>
+     * <li>
+     * {@code ratio_to_report(x) -> ratio_to_report_impl(x, x)}.
+     * The first argument will then be rewritten by {@link #wrapAggregationArguments(WindowExpression, int)}.
+     * The remaining rewriting to {@code x/sum(x)} will be done by the expression to plan translator
+     * </li>
+     * </ul>
+     */
+    private void rewriteSpecificWindowFunctions(FunctionIdentifier winfi, WindowExpression winExpr)
+            throws CompilationException {
+        if (BuiltinFunctions.RATIO_TO_REPORT_IMPL.equals(winfi)) {
+            duplicateLastArgument(winExpr);
+        }
+    }
+
+    private void duplicateLastArgument(WindowExpression winExpr) throws CompilationException {
+        List<Expression> exprList = winExpr.getExprList();
+        Expression arg = exprList.get(exprList.size() - 1);
+        exprList.add((Expression) SqlppRewriteUtil.deepCopy(arg));
+    }
 }
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 cd0e151..0a99895 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
@@ -64,7 +64,8 @@
         FunctionSignature signature = winExpr.getFunctionSignature();
         FunctionIdentifier winfi = FunctionMapUtil.getInternalWindowFunction(signature);
         if (winfi != null) {
-            if (BuiltinFunctions.windowFunctionWithListArg(winfi)) {
+            if (BuiltinFunctions.windowFunctionHasProperty(winfi,
+                    BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
                 List<Expression> newExprList =
                         extractExpressions(winExpr.getExprList(), 1, winExpr.getSourceLocation());
                 winExpr.setExprList(newExprList);
@@ -73,6 +74,9 @@
             List<Expression> newExprList = extractExpressions(winExpr.getExprList(), winExpr.getExprList().size(),
                     winExpr.getSourceLocation());
             winExpr.setExprList(newExprList);
+        } else {
+            throw new CompilationException(ErrorCode.COMPILATION_EXPECTED_WINDOW_FUNCTION, winExpr.getSourceLocation(),
+                    signature.getName());
         }
 
         return winExpr;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index 7ffebc2..2eddfb5 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -219,7 +219,8 @@
         FunctionSignature fs = winExpr.getFunctionSignature();
         FunctionIdentifier winfi = FunctionMapUtil.getInternalWindowFunction(fs);
         if (winfi != null) {
-            if (BuiltinFunctions.windowFunctionWithListArg(winfi)) {
+            if (BuiltinFunctions.windowFunctionHasProperty(winfi,
+                    BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
                 visitWindowExpressionExcludingExprList(winExpr, arg);
                 List<Expression> exprList = winExpr.getExprList();
                 List<Expression> newExprList = new ArrayList<>(exprList.size());
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 54c8330..373e77b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.om.functions;
 
+import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -140,6 +142,8 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.core.algebra.properties.UnpartitionedPropertyComputer;
 
+import static org.apache.asterix.om.functions.BuiltinFunctions.WindowFunctionProperty.*;
+
 public class BuiltinFunctions {
 
     public enum SpatialFilterKind {
@@ -166,7 +170,7 @@
     private static final Map<IFunctionInfo, IFunctionInfo> distinctToRegularScalarAggregateFunctionMap =
             new HashMap<>();
     private static final Map<IFunctionInfo, IFunctionInfo> sqlToWindowFunctions = new HashMap<>();
-    private static final Set<IFunctionInfo> windowFunctions = new HashSet<>();
+    private static final Map<IFunctionInfo, Set<WindowFunctionProperty>> windowFunctions = new HashMap<>();
     private static final Set<IFunctionInfo> windowFunctionsWithListArg = new HashSet<>();
     private static final Set<IFunctionInfo> windowFunctionsWithFrameClause = new HashSet<>();
     private static final Set<IFunctionInfo> windowFunctionsWithOrderArgs = new HashSet<>();
@@ -851,44 +855,56 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-var_pop-distinct", 1);
 
     // window functions
-    public static final FunctionIdentifier ROW_NUMBER =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "row_number", 0);
-    public static final FunctionIdentifier ROW_NUMBER_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "row-number-impl", 0);
-    public static final FunctionIdentifier RANK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rank", 0);
-    public static final FunctionIdentifier RANK_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rank-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier CUME_DIST =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cume_dist", 0);
+    public static final FunctionIdentifier CUME_DIST_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cume-dist-impl", 0);
     public static final FunctionIdentifier DENSE_RANK =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dense_rank", 0);
     public static final FunctionIdentifier DENSE_RANK_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dense-rank-impl", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier PERCENT_RANK =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent_rank", 0);
-    public static final FunctionIdentifier PERCENT_RANK_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent-rank-impl", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier NTILE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ntile", 1);
-    public static final FunctionIdentifier NTILE_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ntile-impl", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier LEAD =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lead", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier LEAD_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lead-impl", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier LAG =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lag", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier LAG_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lag-impl", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier FIRST_VALUE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "first_value", 1);
     public static final FunctionIdentifier FIRST_VALUE_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "first-value-impl", 1);
+    public static final FunctionIdentifier LAG =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lag", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier LAG_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lag-impl", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier LAST_VALUE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "last_value", 1);
     public static final FunctionIdentifier LAST_VALUE_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "last-value-impl", 1);
+    public static final FunctionIdentifier LEAD =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lead", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier LEAD_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lead-impl", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier NTH_VALUE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "nth_value", 2);
     public static final FunctionIdentifier NTH_VALUE_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "nth-value-impl", 2);
+    public static final FunctionIdentifier NTILE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ntile", 1);
+    public static final FunctionIdentifier NTILE_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ntile-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier RANK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rank", 0);
+    public static final FunctionIdentifier RANK_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rank-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier RATIO_TO_REPORT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ratio_to_report", 1);
+    public static final FunctionIdentifier RATIO_TO_REPORT_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ratio-to-report-impl", 2);
+    public static final FunctionIdentifier ROW_NUMBER =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "row_number", 0);
+    public static final FunctionIdentifier ROW_NUMBER_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "row-number-impl", 0);
+    public static final FunctionIdentifier PERCENT_RANK =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent_rank", 0);
+    public static final FunctionIdentifier PERCENT_RANK_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent-rank-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier WIN_PARTITION_LENGTH =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "win_partition_length", 0);
+    public static final FunctionIdentifier WIN_PARTITION_LENGTH_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "win-partition-length-impl", 0);
 
     // unnesting functions
     public static final FunctionIdentifier SCAN_COLLECTION =
@@ -1831,26 +1847,32 @@
 
         // Window functions
 
-        addFunction(ROW_NUMBER, AInt64TypeComputer.INSTANCE, false);
-        addFunction(ROW_NUMBER_IMPL, AInt64TypeComputer.INSTANCE, false);
-        addFunction(RANK, AInt64TypeComputer.INSTANCE, false);
-        addFunction(RANK_IMPL, AInt64TypeComputer.INSTANCE, false);
+        addFunction(CUME_DIST, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(CUME_DIST_IMPL, ADoubleTypeComputer.INSTANCE, false);
         addFunction(DENSE_RANK, AInt64TypeComputer.INSTANCE, false);
         addFunction(DENSE_RANK_IMPL, AInt64TypeComputer.INSTANCE, false);
-        addFunction(PERCENT_RANK, ADoubleTypeComputer.INSTANCE, false);
-        addFunction(PERCENT_RANK_IMPL, ADoubleTypeComputer.INSTANCE, false);
-        addFunction(NTILE, AInt64TypeComputer.INSTANCE_NULLABLE, false);
-        addFunction(NTILE_IMPL, AInt64TypeComputer.INSTANCE_NULLABLE, false);
-        addFunction(LEAD, AnyTypeComputer.INSTANCE, false);
-        addFunction(LEAD_IMPL, AnyTypeComputer.INSTANCE, false);
-        addFunction(LAG, AnyTypeComputer.INSTANCE, false);
-        addFunction(LAG_IMPL, AnyTypeComputer.INSTANCE, false);
         addFunction(FIRST_VALUE, CollectionMemberResultType.INSTANCE_NULLABLE, false);
         addFunction(FIRST_VALUE_IMPL, CollectionMemberResultType.INSTANCE_NULLABLE, false);
+        addFunction(LAG, AnyTypeComputer.INSTANCE, false);
+        addFunction(LAG_IMPL, AnyTypeComputer.INSTANCE, false);
         addFunction(LAST_VALUE, CollectionMemberResultType.INSTANCE_NULLABLE, false);
         addFunction(LAST_VALUE_IMPL, CollectionMemberResultType.INSTANCE_NULLABLE, false);
+        addFunction(LEAD, AnyTypeComputer.INSTANCE, false);
+        addFunction(LEAD_IMPL, AnyTypeComputer.INSTANCE, false);
         addFunction(NTH_VALUE, CollectionMemberResultType.INSTANCE_NULLABLE, false);
         addFunction(NTH_VALUE_IMPL, CollectionMemberResultType.INSTANCE_NULLABLE, false);
+        addFunction(NTILE, AInt64TypeComputer.INSTANCE_NULLABLE, false);
+        addFunction(NTILE_IMPL, AInt64TypeComputer.INSTANCE_NULLABLE, false);
+        addFunction(RANK, AInt64TypeComputer.INSTANCE, false);
+        addFunction(RANK_IMPL, AInt64TypeComputer.INSTANCE, false);
+        addFunction(RATIO_TO_REPORT, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(RATIO_TO_REPORT_IMPL, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(ROW_NUMBER, AInt64TypeComputer.INSTANCE, false);
+        addFunction(ROW_NUMBER_IMPL, AInt64TypeComputer.INSTANCE, false);
+        addFunction(PERCENT_RANK, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(PERCENT_RANK_IMPL, ADoubleTypeComputer.INSTANCE, false);
+        addPrivateFunction(WIN_PARTITION_LENGTH, AInt64TypeComputer.INSTANCE, false);
+        addPrivateFunction(WIN_PARTITION_LENGTH_IMPL, AInt64TypeComputer.INSTANCE, false);
 
         // Similarity functions
         addFunction(EDIT_DISTANCE_CONTAINS, OrderedListOfAnyTypeComputer.INSTANCE, true);
@@ -2623,18 +2645,34 @@
         addGlobalAgg(ST_UNION_AGG, ST_UNION_AGG);
     }
 
+    public enum WindowFunctionProperty {
+        /** Whether the order clause is prohibited */
+        NO_ORDER_CLAUSE,
+        /** Whether the frame clause is prohibited */
+        NO_FRAME_CLAUSE,
+        /** Whether the first argument is a list */
+        HAS_LIST_ARG,
+        /** Whether order by expressions must be injected as arguments */
+        INJECT_ORDER_ARGS,
+        /** Whether a running aggregate requires partition materialization runtime */
+        MATERIALIZE_PARTITION
+    }
+
     static {
         // Window functions
-        addWindowFunction(ROW_NUMBER, ROW_NUMBER_IMPL, false, false, false, false);
-        addWindowFunction(RANK, RANK_IMPL, false, false, true, false);
-        addWindowFunction(DENSE_RANK, DENSE_RANK_IMPL, false, false, true, false);
-        addWindowFunction(PERCENT_RANK, PERCENT_RANK_IMPL, false, false, true, true);
-        addWindowFunction(NTILE, NTILE_IMPL, false, false, false, true);
-        addWindowFunction(LEAD, LEAD_IMPL, false, true, false, false);
-        addWindowFunction(LAG, LAG_IMPL, false, true, false, false);
-        addWindowFunction(FIRST_VALUE, FIRST_VALUE_IMPL, true, true, false, false);
-        addWindowFunction(LAST_VALUE, LAST_VALUE_IMPL, true, true, false, false);
-        addWindowFunction(NTH_VALUE, NTH_VALUE_IMPL, true, true, false, false);
+        addWindowFunction(CUME_DIST, CUME_DIST_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
+        addWindowFunction(DENSE_RANK, DENSE_RANK_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
+        addWindowFunction(FIRST_VALUE, FIRST_VALUE_IMPL, HAS_LIST_ARG);
+        addWindowFunction(LAG, LAG_IMPL, NO_FRAME_CLAUSE, HAS_LIST_ARG);
+        addWindowFunction(LAST_VALUE, LAST_VALUE_IMPL, HAS_LIST_ARG);
+        addWindowFunction(LEAD, LEAD_IMPL, NO_FRAME_CLAUSE, HAS_LIST_ARG);
+        addWindowFunction(NTH_VALUE, NTH_VALUE_IMPL, HAS_LIST_ARG);
+        addWindowFunction(NTILE, NTILE_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
+        addWindowFunction(PERCENT_RANK, PERCENT_RANK_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS, MATERIALIZE_PARTITION);
+        addWindowFunction(RANK, RANK_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
+        addWindowFunction(RATIO_TO_REPORT, RATIO_TO_REPORT_IMPL, NO_ORDER_CLAUSE, NO_FRAME_CLAUSE, HAS_LIST_ARG);
+        addWindowFunction(ROW_NUMBER, ROW_NUMBER_IMPL, NO_FRAME_CLAUSE);
+        addWindowFunction(WIN_PARTITION_LENGTH, WIN_PARTITION_LENGTH_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
     }
 
     static {
@@ -2819,24 +2857,13 @@
     }
 
     public static void addWindowFunction(FunctionIdentifier sqlfi, FunctionIdentifier winfi,
-            boolean supportsFrameClause, boolean hasListArg, boolean requiresOrderArgs,
-            boolean requiresMaterialization) {
+            WindowFunctionProperty... properties) {
         IFunctionInfo sqlinfo = getAsterixFunctionInfo(sqlfi);
         IFunctionInfo wininfo = getAsterixFunctionInfo(winfi);
+        Set<WindowFunctionProperty> propertiesSet = EnumSet.noneOf(WindowFunctionProperty.class);
+        Collections.addAll(propertiesSet, properties);
         sqlToWindowFunctions.put(sqlinfo, wininfo);
-        windowFunctions.add(wininfo);
-        if (supportsFrameClause) {
-            windowFunctionsWithFrameClause.add(wininfo);
-        }
-        if (hasListArg) {
-            windowFunctionsWithListArg.add(wininfo);
-        }
-        if (requiresOrderArgs) {
-            windowFunctionsWithOrderArgs.add(wininfo);
-        }
-        if (requiresMaterialization) {
-            windowFunctionsWithMaterialization.add(wininfo);
-        }
+        windowFunctions.put(wininfo, propertiesSet);
     }
 
     public static FunctionIdentifier getWindowFunction(FunctionIdentifier sqlfi) {
@@ -2845,23 +2872,12 @@
     }
 
     public static boolean isWindowFunction(FunctionIdentifier winfi) {
-        return windowFunctions.contains(getAsterixFunctionInfo(winfi));
+        return windowFunctions.containsKey(getAsterixFunctionInfo(winfi));
     }
 
-    public static boolean windowFunctionSupportsFrameClause(FunctionIdentifier winfi) {
-        return windowFunctionsWithFrameClause.contains(getAsterixFunctionInfo(winfi));
-    }
-
-    public static boolean windowFunctionWithListArg(FunctionIdentifier winfi) {
-        return windowFunctionsWithListArg.contains(getAsterixFunctionInfo(winfi));
-    }
-
-    public static boolean windowFunctionRequiresOrderArgs(FunctionIdentifier winfi) {
-        return windowFunctionsWithOrderArgs.contains(getAsterixFunctionInfo(winfi));
-    }
-
-    public static boolean windowFunctionRequiresMaterialization(FunctionIdentifier winfi) {
-        return windowFunctionsWithMaterialization.contains(getAsterixFunctionInfo(winfi));
+    public static boolean windowFunctionHasProperty(FunctionIdentifier winfi, WindowFunctionProperty property) {
+        Set<WindowFunctionProperty> propertySet = windowFunctions.get(getAsterixFunctionInfo(winfi));
+        return propertySet != null && propertySet.contains(property);
     }
 
     public static AbstractFunctionCallExpression makeWindowFunctionExpression(FunctionIdentifier winfi,
@@ -2896,5 +2912,4 @@
     public static boolean isSimilarityFunction(FunctionIdentifier fi) {
         return similarityFunctions.contains(getAsterixFunctionInfo(fi));
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 0487385..ebde275 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -460,6 +460,7 @@
 import org.apache.asterix.runtime.runningaggregates.std.RankRunningAggregateDescriptor;
 import org.apache.asterix.runtime.runningaggregates.std.RowNumberRunningAggregateDescriptor;
 import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.WinPartitionLenRunningAggregateDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
@@ -664,11 +665,12 @@
         fc.add(ScalarSqlVarPopAggregateDescriptor.FACTORY);
 
         // window functions
-        fc.add(RowNumberRunningAggregateDescriptor.FACTORY);
-        fc.add(RankRunningAggregateDescriptor.FACTORY);
         fc.add(DenseRankRunningAggregateDescriptor.FACTORY);
-        fc.add(PercentRankRunningAggregateDescriptor.FACTORY);
         fc.add(NtileRunningAggregateDescriptor.FACTORY);
+        fc.add(RankRunningAggregateDescriptor.FACTORY);
+        fc.add(RowNumberRunningAggregateDescriptor.FACTORY);
+        fc.add(PercentRankRunningAggregateDescriptor.FACTORY);
+        fc.add(WinPartitionLenRunningAggregateDescriptor.FACTORY);
 
         // boolean functions
         fc.add(AndDescriptor.FACTORY);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
index 6e51559..bccfb45 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Descriptor {@code dense_rank()} window function
+ * Descriptor for {@code dense_rank()} window function
  */
 public class DenseRankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
index 5157451..e3b940d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
@@ -30,7 +30,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Descriptor {@code ntile()} window function
+ * Descriptor for {@code ntile()} window function
  */
 public class NtileRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
index aebef6a..8e4f1f7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 /**
- * Evaluator {@code ntile()} window function
+ * Evaluator for {@code ntile()} window function
  */
 public class NtileRunningAggregateEvaluator implements IWindowAggregateEvaluator {
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
index 180ca99..32282ab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Descriptor {@code percent_rank()} window function
+ * Descriptor for {@code percent_rank()} window function
  */
 public class PercentRankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
index edad37f..6cff7e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
- * Evaluator {@code percent_rank()} window function
+ * Evaluator for {@code percent_rank()} window function
  */
 class PercentRankRunningAggregateEvaluator extends AbstractRankRunningAggregateEvaluator {
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
index d5db134..671393a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Descriptor {@code rank()} window function
+ * Descriptor for {@code rank()} window function
  */
 public class RankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
index 7464751..2500336 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
@@ -29,7 +29,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
 /**
- * Descriptor {@code row_number()} window function
+ * Descriptor for {@code row_number()} window function
  */
 public class RowNumberRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java
new file mode 100644
index 0000000..234e76d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.asterix.runtime.runningaggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Descriptor for internal {@code win_partition_length()} window function
+ */
+public class WinPartitionLenRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = WinPartitionLenRunningAggregateDescriptor::new;
+
+    @Override
+    public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IRunningAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx) {
+                return new WinPartitionLenRunningAggregateEvaluator();
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.WIN_PARTITION_LENGTH_IMPL;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java
new file mode 100644
index 0000000..d5e0b47
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.asterix.runtime.runningaggregates.std;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.runtime.base.IWindowAggregateEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * Evaluator for internal {@code win_partition_length()} window function
+ */
+public class WinPartitionLenRunningAggregateEvaluator implements IWindowAggregateEvaluator {
+
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<AInt64> serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+
+    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+    @Override
+    public void init() {
+        // nothing to do
+    }
+
+    @Override
+    public void initPartition(long partitionLength) throws HyracksDataException {
+        resultStorage.reset();
+        aInt64.setValue(partitionLength);
+        serde.serialize(aInt64, resultStorage.getDataOutput());
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        result.set(resultStorage);
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
index 61880a2..7fa903c 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
@@ -43,8 +43,8 @@
 /**
  * Merges two adjacent window operators into one if their window specifications are compatible.
  * <pre>
- * window [$x] <- [f()] with nested plan (aggergate [$a] <- [agg_1()] - ... - nts )
- * window [$y] <- [g()] with nesedd plan (aggregate [$b] <- [agg_2()] - ... - nts )
+ * window [$x] <- [f()] with nested plan (aggregate [$a] <- [agg_1()] - ... - nts )
+ * window [$y] <- [g()] with nested plan (aggregate [$b] <- [agg_2()] - ... - nts )
  * -->
  * window [$x, $y] <- [f(), g()] with nested plan ( aggregate [$a, $b] <- [agg_1(), agg_2()] - ... - nts )
  * </pre>

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

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

Change in asterixdb[master]: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3113/2/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
File asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java:

PS2, Line 30: import org.apache.asterix.common.exceptions.ErrorCode;
Not used I believe.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


[ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()

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

Details:
- Implement CUME_DIST() and RATIO_TO_REPORT() window functions
- Add internal WIN_PARTITION_LENGTH() window function
- Fixed typos in javadocs

Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3113
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
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/rules/SetAsterixPhysicalOperatorsRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm
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/SqlppWindowAggregationSugarVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java
M hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
31 files changed, 629 insertions(+), 207 deletions(-)

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



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index 4d1d532..2a42da1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -372,7 +372,8 @@
                             expr.getExpressionTag());
                 }
                 AbstractFunctionCallExpression callExpr = (AbstractFunctionCallExpression) expr;
-                if (BuiltinFunctions.windowFunctionRequiresMaterialization(callExpr.getFunctionIdentifier())) {
+                if (BuiltinFunctions.windowFunctionHasProperty(callExpr.getFunctionIdentifier(),
+                        BuiltinFunctions.WindowFunctionProperty.MATERIALIZE_PARTITION)) {
                     partitionMaterialization = true;
                     break;
                 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index a99f9d4..fef765c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -785,7 +785,7 @@
             } else if (projection.hasName()) {
                 fieldBindings.add(getFieldBinding(projection, fieldNames));
             } else {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, projection.getSourceLocation());
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, projection.getSourceLocation(), "");
             }
         }
         if (!fieldBindings.isEmpty()) {
@@ -1037,9 +1037,17 @@
 
         FunctionSignature fs = winExpr.getFunctionSignature();
         FunctionIdentifier fi = getBuiltinFunctionIdentifier(fs.getName(), fs.getArity());
+        if (fi == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_EXPECTED_WINDOW_FUNCTION, winExpr.getSourceLocation(),
+                    fs.getName());
+        }
         boolean isWin = BuiltinFunctions.isWindowFunction(fi);
-        boolean isWinAgg = isWin && BuiltinFunctions.windowFunctionWithListArg(fi);
-        boolean supportsFrameClause = isWin && BuiltinFunctions.windowFunctionSupportsFrameClause(fi);
+        boolean isWinAgg = isWin
+                && BuiltinFunctions.windowFunctionHasProperty(fi, BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG);
+        boolean prohibitOrderClause = isWin && BuiltinFunctions.windowFunctionHasProperty(fi,
+                BuiltinFunctions.WindowFunctionProperty.NO_ORDER_CLAUSE);
+        boolean prohibitFrameClause = isWin && BuiltinFunctions.windowFunctionHasProperty(fi,
+                BuiltinFunctions.WindowFunctionProperty.NO_FRAME_CLAUSE);
 
         Mutable<ILogicalOperator> currentOpRef = tupSource;
 
@@ -1065,6 +1073,9 @@
         int frameExcludeNotStartIdx = -1;
 
         if (winExpr.hasOrderByList()) {
+            if (prohibitOrderClause) {
+                throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_WINDOW_ORDERBY, sourceLoc);
+            }
             List<Expression> orderExprList = winExpr.getOrderbyList();
             List<OrderbyClause.OrderModifier> orderModifierList = winExpr.getOrderbyModifierList();
             orderExprCount = orderExprList.size();
@@ -1092,7 +1103,7 @@
         int winFrameMaxOjbects = -1;
 
         if (winExpr.hasFrameDefinition()) {
-            if (isWin && !supportsFrameClause) {
+            if (prohibitFrameClause) {
                 throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_WINDOW_FRAME, sourceLoc);
             }
             winFrameMode = winExpr.getFrameMode();
@@ -1104,17 +1115,21 @@
             if (!isValidWindowFrameDefinition(winFrameMode, winFrameStartKind, winFrameEndKind, orderExprCount)) {
                 throw new CompilationException(ErrorCode.COMPILATION_INVALID_WINDOW_FRAME, sourceLoc);
             }
-        } else if (!isWin || supportsFrameClause) {
+        } else if (!prohibitFrameClause) {
             winFrameMode = WindowExpression.FrameMode.RANGE;
             winFrameStartKind = WindowExpression.FrameBoundaryKind.UNBOUNDED_PRECEDING;
             winFrameEndKind = WindowExpression.FrameBoundaryKind.CURRENT_ROW;
             winFrameExclusionKind = WindowExpression.FrameExclusionKind.NO_OTHERS;
         }
 
-        FunctionIdentifier winAggFunc = null;
-        FunctionIdentifier winAggDefaultIfNullFunc = null;
-        Expression winAggDefaultExpr = null;
+        boolean makeRunningAgg = false, makeNestedAgg = false;
+        FunctionIdentifier runningAggFunc = null, nestedAggFunc = null, winResultFunc = null, postWinResultFunc = null;
+        Expression postWinExpr = null;
+        List<Expression> nestedAggArgs = null;
+        boolean postWinResultArgsReverse = false;
+
         if (isWinAgg) {
+            makeNestedAgg = true;
             if (BuiltinFunctions.LEAD_IMPL.equals(fi) || BuiltinFunctions.LAG_IMPL.equals(fi)) {
                 int argCount = fargs.size();
                 if (argCount < 1 || argCount > 3) {
@@ -1131,20 +1146,20 @@
                 // then use local-first-element() because it returns SYSTEM_NULL if the list is empty,
                 // otherwise (no default expression) use first-element() which returns NULL if the list is empty
                 if (argCount > 2) {
-                    winAggFunc = BuiltinFunctions.LOCAL_FIRST_ELEMENT;
-                    winAggDefaultIfNullFunc = BuiltinFunctions.IF_SYSTEM_NULL;
-                    winAggDefaultExpr = fargs.get(2);
+                    nestedAggFunc = BuiltinFunctions.SCALAR_LOCAL_FIRST_ELEMENT;
+                    postWinResultFunc = BuiltinFunctions.IF_SYSTEM_NULL;
+                    postWinExpr = fargs.get(2);
                 } else {
-                    winAggFunc = BuiltinFunctions.FIRST_ELEMENT;
+                    nestedAggFunc = BuiltinFunctions.SCALAR_FIRST_ELEMENT;
                 }
                 winFrameMaxOjbects = 1;
             } else if (BuiltinFunctions.FIRST_VALUE_IMPL.equals(fi)) {
-                winAggFunc = BuiltinFunctions.FIRST_ELEMENT;
+                nestedAggFunc = BuiltinFunctions.SCALAR_FIRST_ELEMENT;
                 winFrameMaxOjbects = 1;
             } else if (BuiltinFunctions.LAST_VALUE_IMPL.equals(fi)) {
-                winAggFunc = BuiltinFunctions.LAST_ELEMENT;
+                nestedAggFunc = BuiltinFunctions.SCALAR_LAST_ELEMENT;
             } else if (BuiltinFunctions.NTH_VALUE_IMPL.equals(fi)) {
-                winAggFunc = BuiltinFunctions.FIRST_ELEMENT;
+                nestedAggFunc = BuiltinFunctions.SCALAR_FIRST_ELEMENT;
                 winFrameMaxOjbects = 1;
                 OperatorExpr opExpr = new OperatorExpr();
                 opExpr.addOperand(fargs.get(1));
@@ -1152,9 +1167,36 @@
                 opExpr.addOperand(new LiteralExpr(new IntegerLiteral(1)));
                 opExpr.setSourceLocation(sourceLoc);
                 winFrameOffsetExpr = opExpr;
+            } else if (BuiltinFunctions.RATIO_TO_REPORT_IMPL.equals(fi)) {
+                // ratio_to_report(x) over (...) --> x / sum(x) over (...)
+                nestedAggFunc = BuiltinFunctions.SCALAR_SQL_SUM;
+                postWinResultFunc = BuiltinFunctions.NUMERIC_DIVIDE;
+                postWinExpr = fargs.get(1);
+                postWinResultArgsReverse = true;
             } else {
                 throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, fi.getName());
             }
+            nestedAggArgs = mkSingletonArrayList(fargs.get(0));
+        } else if (isWin) {
+            makeRunningAgg = true;
+            if (BuiltinFunctions.CUME_DIST_IMPL.equals(fi)) {
+                winFrameMode = WindowExpression.FrameMode.RANGE;
+                winFrameStartKind = WindowExpression.FrameBoundaryKind.UNBOUNDED_PRECEDING;
+                winFrameEndKind = WindowExpression.FrameBoundaryKind.CURRENT_ROW;
+                winFrameExclusionKind = WindowExpression.FrameExclusionKind.NO_OTHERS;
+
+                makeNestedAgg = true;
+                runningAggFunc = BuiltinFunctions.WIN_PARTITION_LENGTH_IMPL;
+                nestedAggFunc = BuiltinFunctions.SCALAR_COUNT;
+                nestedAggArgs = mkSingletonArrayList((Expression) SqlppRewriteUtil.deepCopy(winExpr.getWindowVar()));
+                winResultFunc = BuiltinFunctions.NUMERIC_DIVIDE;
+            } else {
+                runningAggFunc = fi;
+            }
+        } else { // regular aggregate
+            makeNestedAgg = true;
+            nestedAggFunc = fi;
+            nestedAggArgs = fargs;
         }
 
         if (winFrameMode != null) {
@@ -1214,54 +1256,15 @@
                 winFrameMaxOjbects);
         winOp.setSourceLocation(sourceLoc);
 
-        AbstractLogicalExpression resultExpr;
+        LogicalVariable runningAggResultVar = null, nestedAggResultVar = null;
 
-        if (isWin && !isWinAgg) {
-            CallExpr callExpr = new CallExpr(new FunctionSignature(fi), fargs);
-            Pair<ILogicalOperator, LogicalVariable> callExprResult = callExpr.accept(this, currentOpRef);
-            ILogicalOperator op = callExprResult.first;
-            if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
-            }
-            AssignOperator assignOp = (AssignOperator) op;
-            List<LogicalVariable> assignVars = assignOp.getVariables();
-            if (assignVars.size() != 1) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
-            }
-            List<Mutable<ILogicalExpression>> assignExprs = assignOp.getExpressions();
-            if (assignExprs.size() != 1) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
-            }
-            ILogicalExpression assignExpr = assignExprs.get(0).getValue();
-            if (assignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc);
-            }
-            AbstractFunctionCallExpression fcallExpr = (AbstractFunctionCallExpression) assignExpr;
-            if (fcallExpr.getKind() != AbstractFunctionCallExpression.FunctionKind.STATEFUL) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc);
-            }
-            if (BuiltinFunctions.windowFunctionRequiresOrderArgs(fi)) {
-                for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExprListOut) {
-                    fcallExpr.getArguments().add(new MutableObject<>(p.second.getValue().cloneExpression()));
-                }
-            }
-
-            winOp.getInputs().add(assignOp.getInputs().get(0));
-            winOp.getVariables().addAll(assignVars);
-            winOp.getExpressions().addAll(assignExprs);
-
-            resultExpr = new VariableReferenceExpression(assignVars.get(0));
-            resultExpr.setSourceLocation(sourceLoc);
-            currentOpRef = new MutableObject<>(winOp);
-        } else {
+        if (makeNestedAgg) {
             LogicalVariable windowRecordVar = context.newVar();
             ILogicalExpression windowRecordConstr =
                     createRecordConstructor(winExpr.getWindowFieldList(), currentOpRef, sourceLoc);
             AssignOperator assignOp = new AssignOperator(windowRecordVar, new MutableObject<>(windowRecordConstr));
             assignOp.getInputs().add(currentOpRef);
             assignOp.setSourceLocation(sourceLoc);
-
-            winOp.getInputs().add(new MutableObject<>(assignOp));
 
             NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(winOp));
             ntsOp.setSourceLocation(sourceLoc);
@@ -1280,62 +1283,93 @@
 
             context.setVar(winExpr.getWindowVar(), windowVar);
 
-            if (isWinAgg) {
-                Expression listArgExpr = fargs.get(0);
-                Pair<ILogicalOperator, LogicalVariable> listArgExprResult =
-                        listArgExpr.accept(this, new MutableObject<>(aggOp));
-                VariableReferenceExpression listArgVarRef = new VariableReferenceExpression(listArgExprResult.second);
-                listArgVarRef.setSourceLocation(sourceLoc);
+            CallExpr callExpr = new CallExpr(new FunctionSignature(nestedAggFunc), nestedAggArgs);
+            Pair<ILogicalOperator, LogicalVariable> exprResult = callExpr.accept(this, new MutableObject<>(aggOp));
+            winOp.getNestedPlans().add(new ALogicalPlanImpl(new MutableObject<>(exprResult.first)));
 
-                LogicalVariable unnestVar = context.newVar();
-                UnnestingFunctionCallExpression unnestExpr = new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
-                        mkSingletonArrayList(new MutableObject<>(listArgVarRef)));
-                unnestExpr.setSourceLocation(sourceLoc);
-                UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(unnestExpr));
-                unnestOp.setSourceLocation(sourceLoc);
-                unnestOp.getInputs().add(new MutableObject<>(listArgExprResult.first));
-
-                VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
-                unnestVarRef.setSourceLocation(sourceLoc);
-
-                AggregateFunctionCallExpression winAggCall = BuiltinFunctions.makeAggregateFunctionExpression(
-                        winAggFunc, mkSingletonArrayList(new MutableObject<>(unnestVarRef)));
-                winAggCall.setSourceLocation(sourceLoc);
-                LogicalVariable winAggVar = context.newVar();
-                AggregateOperator winAggOp = new AggregateOperator(mkSingletonArrayList(winAggVar),
-                        mkSingletonArrayList(new MutableObject<>(winAggCall)));
-                winAggOp.getInputs().add(new MutableObject<>(unnestOp));
-                winAggOp.setSourceLocation(sourceLoc);
-
-                winOp.getNestedPlans().add(new ALogicalPlanImpl(new MutableObject<>(winAggOp)));
-                currentOpRef = new MutableObject<>(winOp);
-
-                resultExpr = new VariableReferenceExpression(winAggVar);
-                resultExpr.setSourceLocation(sourceLoc);
-
-                if (winAggDefaultExpr != null) {
-                    Pair<ILogicalOperator, LogicalVariable> winAggDefaultExprResult =
-                            winAggDefaultExpr.accept(this, currentOpRef);
-                    VariableReferenceExpression winAggDefaultVarRef =
-                            new VariableReferenceExpression(winAggDefaultExprResult.second);
-                    winAggDefaultVarRef.setSourceLocation(sourceLoc);
-                    AbstractFunctionCallExpression ifNullExpr =
-                            createFunctionCallExpression(winAggDefaultIfNullFunc, sourceLoc);
-                    ifNullExpr.getArguments().add(new MutableObject<>(resultExpr));
-                    ifNullExpr.getArguments().add(new MutableObject<>(winAggDefaultVarRef));
-                    resultExpr = ifNullExpr;
-                    currentOpRef = new MutableObject<>(winAggDefaultExprResult.first);
-                }
-            } else {
-                CallExpr callExpr = new CallExpr(new FunctionSignature(fi), fargs);
-                Pair<ILogicalOperator, LogicalVariable> exprResult = callExpr.accept(this, new MutableObject<>(aggOp));
-                winOp.getNestedPlans().add(new ALogicalPlanImpl(new MutableObject<>(exprResult.first)));
-                resultExpr = new VariableReferenceExpression(exprResult.second);
-                resultExpr.setSourceLocation(sourceLoc);
-                currentOpRef = new MutableObject<>(winOp);
-            }
+            currentOpRef = new MutableObject<>(assignOp);
+            nestedAggResultVar = exprResult.second;
         }
+
+        if (makeRunningAgg) {
+            CallExpr callExpr = new CallExpr(new FunctionSignature(runningAggFunc), fargs);
+            Pair<ILogicalOperator, LogicalVariable> callExprResult = callExpr.accept(this, currentOpRef);
+            ILogicalOperator op = callExprResult.first;
+            if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+            }
+            AssignOperator assignOp = (AssignOperator) op;
+            List<LogicalVariable> assignVars = assignOp.getVariables();
+            if (assignVars.size() != 1) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+            }
+            List<Mutable<ILogicalExpression>> assignExprs = assignOp.getExpressions();
+            if (assignExprs.size() != 1) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+            }
+            ILogicalExpression assignExpr = assignExprs.get(0).getValue();
+            if (assignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+            }
+            AbstractFunctionCallExpression fcallExpr = (AbstractFunctionCallExpression) assignExpr;
+            if (fcallExpr.getKind() != AbstractFunctionCallExpression.FunctionKind.STATEFUL) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, fcallExpr.getKind());
+            }
+            if (BuiltinFunctions.windowFunctionHasProperty(fi,
+                    BuiltinFunctions.WindowFunctionProperty.INJECT_ORDER_ARGS)) {
+                for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExprListOut) {
+                    fcallExpr.getArguments().add(new MutableObject<>(p.second.getValue().cloneExpression()));
+                }
+            }
+
+            winOp.getVariables().addAll(assignVars);
+            winOp.getExpressions().addAll(assignExprs);
+
+            currentOpRef = new MutableObject<>(assignOp.getInputs().get(0).getValue());
+            runningAggResultVar = assignVars.get(0);
+        }
+
+        winOp.getInputs().add(currentOpRef);
+        currentOpRef = new MutableObject<>(winOp);
+
+        AbstractLogicalExpression resultExpr;
+        if (makeRunningAgg && makeNestedAgg) {
+            VariableReferenceExpression runningAggResultVarRef = new VariableReferenceExpression(runningAggResultVar);
+            runningAggResultVarRef.setSourceLocation(sourceLoc);
+            VariableReferenceExpression nestedAggResultVarRef = new VariableReferenceExpression(nestedAggResultVar);
+            nestedAggResultVarRef.setSourceLocation(sourceLoc);
+            AbstractFunctionCallExpression resultCallExpr = createFunctionCallExpression(winResultFunc, sourceLoc);
+            resultCallExpr.getArguments().add(new MutableObject<>(nestedAggResultVarRef));
+            resultCallExpr.getArguments().add(new MutableObject<>(runningAggResultVarRef));
+            resultExpr = resultCallExpr;
+        } else if (makeRunningAgg) {
+            resultExpr = new VariableReferenceExpression(runningAggResultVar);
+            resultExpr.setSourceLocation(sourceLoc);
+        } else if (makeNestedAgg) {
+            resultExpr = new VariableReferenceExpression(nestedAggResultVar);
+            resultExpr.setSourceLocation(sourceLoc);
+        } else {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+        }
+
+        if (postWinExpr != null) {
+            Pair<ILogicalOperator, LogicalVariable> postWinExprResult = postWinExpr.accept(this, currentOpRef);
+            currentOpRef = new MutableObject<>(postWinExprResult.first);
+            VariableReferenceExpression postWinVarRef = new VariableReferenceExpression(postWinExprResult.second);
+            postWinVarRef.setSourceLocation(sourceLoc);
+            AbstractFunctionCallExpression postWinResultCallExpr =
+                    createFunctionCallExpression(postWinResultFunc, sourceLoc);
+            List<Mutable<ILogicalExpression>> postWinResultCallArgs = postWinResultCallExpr.getArguments();
+            if (!postWinResultArgsReverse) {
+                postWinResultCallArgs.add(new MutableObject<>(resultExpr));
+                postWinResultCallArgs.add(new MutableObject<>(postWinVarRef));
+            } else {
+                postWinResultCallArgs.add(new MutableObject<>(postWinVarRef));
+                postWinResultCallArgs.add(new MutableObject<>(resultExpr));
+            }
+            resultExpr = postWinResultCallExpr;
+        }
+
         // must return ASSIGN
         LogicalVariable resultVar = context.newVar();
         AssignOperator resultOp = new AssignOperator(resultVar, new MutableObject<>(resultExpr));
@@ -1530,7 +1564,8 @@
             }
             AbstractFunctionCallExpression valueExpr =
                     BuiltinFunctions.makeWindowFunctionExpression(fid, new ArrayList<>());
-            if (BuiltinFunctions.windowFunctionRequiresOrderArgs(valueExpr.getFunctionIdentifier())) {
+            if (BuiltinFunctions.windowFunctionHasProperty(valueExpr.getFunctionIdentifier(),
+                    BuiltinFunctions.WindowFunctionProperty.INJECT_ORDER_ARGS)) {
                 for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExprList) {
                     valueExpr.getArguments().add(new MutableObject<>(p.second.getValue().cloneExpression()));
                 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp
new file mode 100644
index 0000000..1e114b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 window function: cume_dist()
+ * Expected Res : SUCCESS
+ */
+
+FROM range(1, 5) x, range(1, 4) y
+SELECT x, y,
+  cume_dist() over (order by x) as `cume_dist`
+ORDER BY x, y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp
new file mode 100644
index 0000000..f5c4dad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 RATIO_TO_REPORT() without PARTITION BY
+ * Expected Res : SUCCESS
+ */
+
+FROM range(1, 4) x
+SELECT x,
+  round_half_to_even(ratio_to_report(x) over (), 2) as `ratio_to_report`
+ORDER BY ratio_to_report;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp
new file mode 100644
index 0000000..29a82ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 RATIO_TO_REPORT() with PARTITION BY
+ * Expected Res : SUCCESS
+ */
+
+FROM range(1, 4) x, range(1, 2) y
+SELECT x, y,
+  round_half_to_even(ratio_to_report(x) over (partition by y), 2) as `ratio_to_report`
+ORDER BY y, ratio_to_report;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp
new file mode 100644
index 0000000..6add05c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 RATIO_TO_REPORT() handling of NULL values
+ * Expected Res : SUCCESS
+ */
+
+ FROM range(0, 4) x, range(0, 1) y
+ SELECT x, y,
+   round_half_to_even(ratio_to_report(nullif(x,y)) over (partition by y), 2) as `ratio_to_report`
+ ORDER BY y, ratio_to_report;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.query.sqlpp
new file mode 100644
index 0000000..13293ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.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 RATIO_TO_REPORT() when total = 0
+ * Expected Res : SUCCESS
+ */
+
+FROM range(1, 4) x
+SELECT x,
+  round_half_to_even(ratio_to_report(x-x) over (), 2) as `ratio_to_report`
+ORDER BY ratio_to_report;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp
new file mode 100644
index 0000000..b539629
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 RATIO_TO_REPORT()
+ * Expected Res : FAILURE (unexpected ORDER BY)
+ */
+
+FROM range(1, 4) x, range(1, 2) y
+SELECT ratio_to_report(x) over (order by y)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm
new file mode 100644
index 0000000..85969e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm
@@ -0,0 +1,20 @@
+{ "x": 1, "y": 1, "cume_dist": 0.2 }
+{ "x": 1, "y": 2, "cume_dist": 0.2 }
+{ "x": 1, "y": 3, "cume_dist": 0.2 }
+{ "x": 1, "y": 4, "cume_dist": 0.2 }
+{ "x": 2, "y": 1, "cume_dist": 0.4 }
+{ "x": 2, "y": 2, "cume_dist": 0.4 }
+{ "x": 2, "y": 3, "cume_dist": 0.4 }
+{ "x": 2, "y": 4, "cume_dist": 0.4 }
+{ "x": 3, "y": 1, "cume_dist": 0.6 }
+{ "x": 3, "y": 2, "cume_dist": 0.6 }
+{ "x": 3, "y": 3, "cume_dist": 0.6 }
+{ "x": 3, "y": 4, "cume_dist": 0.6 }
+{ "x": 4, "y": 1, "cume_dist": 0.8 }
+{ "x": 4, "y": 2, "cume_dist": 0.8 }
+{ "x": 4, "y": 3, "cume_dist": 0.8 }
+{ "x": 4, "y": 4, "cume_dist": 0.8 }
+{ "x": 5, "y": 1, "cume_dist": 1.0 }
+{ "x": 5, "y": 2, "cume_dist": 1.0 }
+{ "x": 5, "y": 3, "cume_dist": 1.0 }
+{ "x": 5, "y": 4, "cume_dist": 1.0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm
new file mode 100644
index 0000000..0a6cca0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm
@@ -0,0 +1,4 @@
+{ "x": 1, "ratio_to_report": 0.1 }
+{ "x": 2, "ratio_to_report": 0.2 }
+{ "x": 3, "ratio_to_report": 0.3 }
+{ "x": 4, "ratio_to_report": 0.4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm
new file mode 100644
index 0000000..af6e7df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm
@@ -0,0 +1,8 @@
+{ "x": 1, "y": 1, "ratio_to_report": 0.1 }
+{ "x": 2, "y": 1, "ratio_to_report": 0.2 }
+{ "x": 3, "y": 1, "ratio_to_report": 0.3 }
+{ "x": 4, "y": 1, "ratio_to_report": 0.4 }
+{ "x": 1, "y": 2, "ratio_to_report": 0.1 }
+{ "x": 2, "y": 2, "ratio_to_report": 0.2 }
+{ "x": 3, "y": 2, "ratio_to_report": 0.3 }
+{ "x": 4, "y": 2, "ratio_to_report": 0.4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm
new file mode 100644
index 0000000..a6b9df4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm
@@ -0,0 +1,10 @@
+{ "x": 0, "y": 0, "ratio_to_report": null }
+{ "x": 1, "y": 0, "ratio_to_report": 0.1 }
+{ "x": 2, "y": 0, "ratio_to_report": 0.2 }
+{ "x": 3, "y": 0, "ratio_to_report": 0.3 }
+{ "x": 4, "y": 0, "ratio_to_report": 0.4 }
+{ "x": 1, "y": 1, "ratio_to_report": null }
+{ "x": 0, "y": 1, "ratio_to_report": 0.0 }
+{ "x": 2, "y": 1, "ratio_to_report": 0.22 }
+{ "x": 3, "y": 1, "ratio_to_report": 0.33 }
+{ "x": 4, "y": 1, "ratio_to_report": 0.44 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm
new file mode 100644
index 0000000..edd48d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm
@@ -0,0 +1,4 @@
+{ "x": 1, "ratio_to_report": null }
+{ "x": 2, "ratio_to_report": null }
+{ "x": 3, "ratio_to_report": null }
+{ "x": 4, "ratio_to_report": null }
\ No newline at end of file
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 3a857c1..dbd0d29 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -9219,6 +9219,11 @@
   </test-group>
   <test-group name="window">
     <test-case FilePath="window">
+      <compilation-unit name="cume_dist_01">
+        <output-dir compare="Text">cume_dist_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
       <compilation-unit name="dense_rank_01">
         <output-dir compare="Text">dense_rank_01</output-dir>
       </compilation-unit>
@@ -9256,6 +9261,17 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="window">
+      <compilation-unit name="ratio_to_report_01">
+        <output-dir compare="Text">ratio_to_report_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="ratio_to_report_02_negative">
+        <output-dir compare="Text">ratio_to_report_01</output-dir>
+        <expected-error>ASX1101: Unexpected ORDER BY clause in window expression</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
       <compilation-unit name="row_number_01">
         <output-dir compare="Text">row_number_01</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 3d63936..d8a21f1 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
@@ -186,6 +186,8 @@
     public static final int COMPILATION_INVALID_WINDOW_FRAME = 1098;
     public static final int COMPILATION_UNEXPECTED_WINDOW_FRAME = 1099;
     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;
 
     // 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 1ff4f64..b55f537 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -173,6 +173,8 @@
 1098 = Invalid window frame definition
 1099 = Unexpected window frame definition
 1100 = Unexpected window expression
+1101 = Unexpected ORDER BY clause in window expression
+1102 = Expected window or aggregate function, got: %1$s
 
 # Feed Errors
 3001 = Illegal state.
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 fa40d78..0b2d2cf 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,6 +27,7 @@
 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;
@@ -36,6 +37,7 @@
 import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
 import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
+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;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -86,7 +88,9 @@
         FunctionIdentifier winfi = FunctionMapUtil.getInternalWindowFunction(signature);
         if (winfi != null) {
             winExpr.setFunctionSignature(new FunctionSignature(winfi));
-            if (BuiltinFunctions.windowFunctionWithListArg(winfi)) {
+            rewriteSpecificWindowFunctions(winfi, winExpr);
+            if (BuiltinFunctions.windowFunctionHasProperty(winfi,
+                    BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
                 wrapAggregationArguments(winExpr, 1);
             }
         } else if (FunctionMapUtil.isSql92AggregateFunction(signature)) {
@@ -135,4 +139,27 @@
             }
         }
     }
+
+    /**
+     * Apply rewritings for specific window functions:
+     * <ul>
+     * <li>
+     * {@code ratio_to_report(x) -> ratio_to_report_impl(x, x)}.
+     * The first argument will then be rewritten by {@link #wrapAggregationArguments(WindowExpression, int)}.
+     * The remaining rewriting to {@code x/sum(x)} will be done by the expression to plan translator
+     * </li>
+     * </ul>
+     */
+    private void rewriteSpecificWindowFunctions(FunctionIdentifier winfi, WindowExpression winExpr)
+            throws CompilationException {
+        if (BuiltinFunctions.RATIO_TO_REPORT_IMPL.equals(winfi)) {
+            duplicateLastArgument(winExpr);
+        }
+    }
+
+    private void duplicateLastArgument(WindowExpression winExpr) throws CompilationException {
+        List<Expression> exprList = winExpr.getExprList();
+        Expression arg = exprList.get(exprList.size() - 1);
+        exprList.add((Expression) SqlppRewriteUtil.deepCopy(arg));
+    }
 }
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 cd0e151..b1c2d60 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
@@ -64,7 +64,8 @@
         FunctionSignature signature = winExpr.getFunctionSignature();
         FunctionIdentifier winfi = FunctionMapUtil.getInternalWindowFunction(signature);
         if (winfi != null) {
-            if (BuiltinFunctions.windowFunctionWithListArg(winfi)) {
+            if (BuiltinFunctions.windowFunctionHasProperty(winfi,
+                    BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
                 List<Expression> newExprList =
                         extractExpressions(winExpr.getExprList(), 1, winExpr.getSourceLocation());
                 winExpr.setExprList(newExprList);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index 7ffebc2..2eddfb5 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -219,7 +219,8 @@
         FunctionSignature fs = winExpr.getFunctionSignature();
         FunctionIdentifier winfi = FunctionMapUtil.getInternalWindowFunction(fs);
         if (winfi != null) {
-            if (BuiltinFunctions.windowFunctionWithListArg(winfi)) {
+            if (BuiltinFunctions.windowFunctionHasProperty(winfi,
+                    BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
                 visitWindowExpressionExcludingExprList(winExpr, arg);
                 List<Expression> exprList = winExpr.getExprList();
                 List<Expression> newExprList = new ArrayList<>(exprList.size());
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 54c8330..20fbf63 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.om.functions;
 
+import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -140,6 +142,8 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.core.algebra.properties.UnpartitionedPropertyComputer;
 
+import static org.apache.asterix.om.functions.BuiltinFunctions.WindowFunctionProperty.*;
+
 public class BuiltinFunctions {
 
     public enum SpatialFilterKind {
@@ -166,11 +170,7 @@
     private static final Map<IFunctionInfo, IFunctionInfo> distinctToRegularScalarAggregateFunctionMap =
             new HashMap<>();
     private static final Map<IFunctionInfo, IFunctionInfo> sqlToWindowFunctions = new HashMap<>();
-    private static final Set<IFunctionInfo> windowFunctions = new HashSet<>();
-    private static final Set<IFunctionInfo> windowFunctionsWithListArg = new HashSet<>();
-    private static final Set<IFunctionInfo> windowFunctionsWithFrameClause = new HashSet<>();
-    private static final Set<IFunctionInfo> windowFunctionsWithOrderArgs = new HashSet<>();
-    private static final Set<IFunctionInfo> windowFunctionsWithMaterialization = new HashSet<>();
+    private static final Map<IFunctionInfo, Set<WindowFunctionProperty>> windowFunctions = new HashMap<>();
 
     private static final Map<IFunctionInfo, SpatialFilterKind> spatialFilterFunctions = new HashMap<>();
 
@@ -851,44 +851,56 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-var_pop-distinct", 1);
 
     // window functions
-    public static final FunctionIdentifier ROW_NUMBER =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "row_number", 0);
-    public static final FunctionIdentifier ROW_NUMBER_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "row-number-impl", 0);
-    public static final FunctionIdentifier RANK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rank", 0);
-    public static final FunctionIdentifier RANK_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rank-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier CUME_DIST =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cume_dist", 0);
+    public static final FunctionIdentifier CUME_DIST_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cume-dist-impl", 0);
     public static final FunctionIdentifier DENSE_RANK =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dense_rank", 0);
     public static final FunctionIdentifier DENSE_RANK_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dense-rank-impl", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier PERCENT_RANK =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent_rank", 0);
-    public static final FunctionIdentifier PERCENT_RANK_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent-rank-impl", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier NTILE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ntile", 1);
-    public static final FunctionIdentifier NTILE_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ntile-impl", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier LEAD =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lead", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier LEAD_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lead-impl", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier LAG =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lag", FunctionIdentifier.VARARGS);
-    public static final FunctionIdentifier LAG_IMPL =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lag-impl", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier FIRST_VALUE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "first_value", 1);
     public static final FunctionIdentifier FIRST_VALUE_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "first-value-impl", 1);
+    public static final FunctionIdentifier LAG =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lag", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier LAG_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lag-impl", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier LAST_VALUE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "last_value", 1);
     public static final FunctionIdentifier LAST_VALUE_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "last-value-impl", 1);
+    public static final FunctionIdentifier LEAD =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lead", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier LEAD_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "lead-impl", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier NTH_VALUE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "nth_value", 2);
     public static final FunctionIdentifier NTH_VALUE_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "nth-value-impl", 2);
+    public static final FunctionIdentifier NTILE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ntile", 1);
+    public static final FunctionIdentifier NTILE_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ntile-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier RANK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rank", 0);
+    public static final FunctionIdentifier RANK_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "rank-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier RATIO_TO_REPORT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ratio_to_report", 1);
+    public static final FunctionIdentifier RATIO_TO_REPORT_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ratio-to-report-impl", 2);
+    public static final FunctionIdentifier ROW_NUMBER =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "row_number", 0);
+    public static final FunctionIdentifier ROW_NUMBER_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "row-number-impl", 0);
+    public static final FunctionIdentifier PERCENT_RANK =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent_rank", 0);
+    public static final FunctionIdentifier PERCENT_RANK_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent-rank-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier WIN_PARTITION_LENGTH =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "win_partition_length", 0);
+    public static final FunctionIdentifier WIN_PARTITION_LENGTH_IMPL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "win-partition-length-impl", 0);
 
     // unnesting functions
     public static final FunctionIdentifier SCAN_COLLECTION =
@@ -1831,26 +1843,32 @@
 
         // Window functions
 
-        addFunction(ROW_NUMBER, AInt64TypeComputer.INSTANCE, false);
-        addFunction(ROW_NUMBER_IMPL, AInt64TypeComputer.INSTANCE, false);
-        addFunction(RANK, AInt64TypeComputer.INSTANCE, false);
-        addFunction(RANK_IMPL, AInt64TypeComputer.INSTANCE, false);
+        addFunction(CUME_DIST, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(CUME_DIST_IMPL, ADoubleTypeComputer.INSTANCE, false);
         addFunction(DENSE_RANK, AInt64TypeComputer.INSTANCE, false);
         addFunction(DENSE_RANK_IMPL, AInt64TypeComputer.INSTANCE, false);
-        addFunction(PERCENT_RANK, ADoubleTypeComputer.INSTANCE, false);
-        addFunction(PERCENT_RANK_IMPL, ADoubleTypeComputer.INSTANCE, false);
-        addFunction(NTILE, AInt64TypeComputer.INSTANCE_NULLABLE, false);
-        addFunction(NTILE_IMPL, AInt64TypeComputer.INSTANCE_NULLABLE, false);
-        addFunction(LEAD, AnyTypeComputer.INSTANCE, false);
-        addFunction(LEAD_IMPL, AnyTypeComputer.INSTANCE, false);
-        addFunction(LAG, AnyTypeComputer.INSTANCE, false);
-        addFunction(LAG_IMPL, AnyTypeComputer.INSTANCE, false);
         addFunction(FIRST_VALUE, CollectionMemberResultType.INSTANCE_NULLABLE, false);
         addFunction(FIRST_VALUE_IMPL, CollectionMemberResultType.INSTANCE_NULLABLE, false);
+        addFunction(LAG, AnyTypeComputer.INSTANCE, false);
+        addFunction(LAG_IMPL, AnyTypeComputer.INSTANCE, false);
         addFunction(LAST_VALUE, CollectionMemberResultType.INSTANCE_NULLABLE, false);
         addFunction(LAST_VALUE_IMPL, CollectionMemberResultType.INSTANCE_NULLABLE, false);
+        addFunction(LEAD, AnyTypeComputer.INSTANCE, false);
+        addFunction(LEAD_IMPL, AnyTypeComputer.INSTANCE, false);
         addFunction(NTH_VALUE, CollectionMemberResultType.INSTANCE_NULLABLE, false);
         addFunction(NTH_VALUE_IMPL, CollectionMemberResultType.INSTANCE_NULLABLE, false);
+        addFunction(NTILE, AInt64TypeComputer.INSTANCE_NULLABLE, false);
+        addFunction(NTILE_IMPL, AInt64TypeComputer.INSTANCE_NULLABLE, false);
+        addFunction(RANK, AInt64TypeComputer.INSTANCE, false);
+        addFunction(RANK_IMPL, AInt64TypeComputer.INSTANCE, false);
+        addFunction(RATIO_TO_REPORT, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(RATIO_TO_REPORT_IMPL, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(ROW_NUMBER, AInt64TypeComputer.INSTANCE, false);
+        addFunction(ROW_NUMBER_IMPL, AInt64TypeComputer.INSTANCE, false);
+        addFunction(PERCENT_RANK, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(PERCENT_RANK_IMPL, ADoubleTypeComputer.INSTANCE, false);
+        addPrivateFunction(WIN_PARTITION_LENGTH, AInt64TypeComputer.INSTANCE, false);
+        addPrivateFunction(WIN_PARTITION_LENGTH_IMPL, AInt64TypeComputer.INSTANCE, false);
 
         // Similarity functions
         addFunction(EDIT_DISTANCE_CONTAINS, OrderedListOfAnyTypeComputer.INSTANCE, true);
@@ -2623,18 +2641,34 @@
         addGlobalAgg(ST_UNION_AGG, ST_UNION_AGG);
     }
 
+    public enum WindowFunctionProperty {
+        /** Whether the order clause is prohibited */
+        NO_ORDER_CLAUSE,
+        /** Whether the frame clause is prohibited */
+        NO_FRAME_CLAUSE,
+        /** Whether the first argument is a list */
+        HAS_LIST_ARG,
+        /** Whether order by expressions must be injected as arguments */
+        INJECT_ORDER_ARGS,
+        /** Whether a running aggregate requires partition materialization runtime */
+        MATERIALIZE_PARTITION
+    }
+
     static {
         // Window functions
-        addWindowFunction(ROW_NUMBER, ROW_NUMBER_IMPL, false, false, false, false);
-        addWindowFunction(RANK, RANK_IMPL, false, false, true, false);
-        addWindowFunction(DENSE_RANK, DENSE_RANK_IMPL, false, false, true, false);
-        addWindowFunction(PERCENT_RANK, PERCENT_RANK_IMPL, false, false, true, true);
-        addWindowFunction(NTILE, NTILE_IMPL, false, false, false, true);
-        addWindowFunction(LEAD, LEAD_IMPL, false, true, false, false);
-        addWindowFunction(LAG, LAG_IMPL, false, true, false, false);
-        addWindowFunction(FIRST_VALUE, FIRST_VALUE_IMPL, true, true, false, false);
-        addWindowFunction(LAST_VALUE, LAST_VALUE_IMPL, true, true, false, false);
-        addWindowFunction(NTH_VALUE, NTH_VALUE_IMPL, true, true, false, false);
+        addWindowFunction(CUME_DIST, CUME_DIST_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
+        addWindowFunction(DENSE_RANK, DENSE_RANK_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
+        addWindowFunction(FIRST_VALUE, FIRST_VALUE_IMPL, HAS_LIST_ARG);
+        addWindowFunction(LAG, LAG_IMPL, NO_FRAME_CLAUSE, HAS_LIST_ARG);
+        addWindowFunction(LAST_VALUE, LAST_VALUE_IMPL, HAS_LIST_ARG);
+        addWindowFunction(LEAD, LEAD_IMPL, NO_FRAME_CLAUSE, HAS_LIST_ARG);
+        addWindowFunction(NTH_VALUE, NTH_VALUE_IMPL, HAS_LIST_ARG);
+        addWindowFunction(NTILE, NTILE_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
+        addWindowFunction(PERCENT_RANK, PERCENT_RANK_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS, MATERIALIZE_PARTITION);
+        addWindowFunction(RANK, RANK_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
+        addWindowFunction(RATIO_TO_REPORT, RATIO_TO_REPORT_IMPL, NO_ORDER_CLAUSE, NO_FRAME_CLAUSE, HAS_LIST_ARG);
+        addWindowFunction(ROW_NUMBER, ROW_NUMBER_IMPL, NO_FRAME_CLAUSE);
+        addWindowFunction(WIN_PARTITION_LENGTH, WIN_PARTITION_LENGTH_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
     }
 
     static {
@@ -2819,24 +2853,13 @@
     }
 
     public static void addWindowFunction(FunctionIdentifier sqlfi, FunctionIdentifier winfi,
-            boolean supportsFrameClause, boolean hasListArg, boolean requiresOrderArgs,
-            boolean requiresMaterialization) {
+            WindowFunctionProperty... properties) {
         IFunctionInfo sqlinfo = getAsterixFunctionInfo(sqlfi);
         IFunctionInfo wininfo = getAsterixFunctionInfo(winfi);
+        Set<WindowFunctionProperty> propertiesSet = EnumSet.noneOf(WindowFunctionProperty.class);
+        Collections.addAll(propertiesSet, properties);
         sqlToWindowFunctions.put(sqlinfo, wininfo);
-        windowFunctions.add(wininfo);
-        if (supportsFrameClause) {
-            windowFunctionsWithFrameClause.add(wininfo);
-        }
-        if (hasListArg) {
-            windowFunctionsWithListArg.add(wininfo);
-        }
-        if (requiresOrderArgs) {
-            windowFunctionsWithOrderArgs.add(wininfo);
-        }
-        if (requiresMaterialization) {
-            windowFunctionsWithMaterialization.add(wininfo);
-        }
+        windowFunctions.put(wininfo, propertiesSet);
     }
 
     public static FunctionIdentifier getWindowFunction(FunctionIdentifier sqlfi) {
@@ -2845,23 +2868,12 @@
     }
 
     public static boolean isWindowFunction(FunctionIdentifier winfi) {
-        return windowFunctions.contains(getAsterixFunctionInfo(winfi));
+        return windowFunctions.containsKey(getAsterixFunctionInfo(winfi));
     }
 
-    public static boolean windowFunctionSupportsFrameClause(FunctionIdentifier winfi) {
-        return windowFunctionsWithFrameClause.contains(getAsterixFunctionInfo(winfi));
-    }
-
-    public static boolean windowFunctionWithListArg(FunctionIdentifier winfi) {
-        return windowFunctionsWithListArg.contains(getAsterixFunctionInfo(winfi));
-    }
-
-    public static boolean windowFunctionRequiresOrderArgs(FunctionIdentifier winfi) {
-        return windowFunctionsWithOrderArgs.contains(getAsterixFunctionInfo(winfi));
-    }
-
-    public static boolean windowFunctionRequiresMaterialization(FunctionIdentifier winfi) {
-        return windowFunctionsWithMaterialization.contains(getAsterixFunctionInfo(winfi));
+    public static boolean windowFunctionHasProperty(FunctionIdentifier winfi, WindowFunctionProperty property) {
+        Set<WindowFunctionProperty> propertySet = windowFunctions.get(getAsterixFunctionInfo(winfi));
+        return propertySet != null && propertySet.contains(property);
     }
 
     public static AbstractFunctionCallExpression makeWindowFunctionExpression(FunctionIdentifier winfi,
@@ -2896,5 +2908,4 @@
     public static boolean isSimilarityFunction(FunctionIdentifier fi) {
         return similarityFunctions.contains(getAsterixFunctionInfo(fi));
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 0487385..ebde275 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -460,6 +460,7 @@
 import org.apache.asterix.runtime.runningaggregates.std.RankRunningAggregateDescriptor;
 import org.apache.asterix.runtime.runningaggregates.std.RowNumberRunningAggregateDescriptor;
 import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.WinPartitionLenRunningAggregateDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
@@ -664,11 +665,12 @@
         fc.add(ScalarSqlVarPopAggregateDescriptor.FACTORY);
 
         // window functions
-        fc.add(RowNumberRunningAggregateDescriptor.FACTORY);
-        fc.add(RankRunningAggregateDescriptor.FACTORY);
         fc.add(DenseRankRunningAggregateDescriptor.FACTORY);
-        fc.add(PercentRankRunningAggregateDescriptor.FACTORY);
         fc.add(NtileRunningAggregateDescriptor.FACTORY);
+        fc.add(RankRunningAggregateDescriptor.FACTORY);
+        fc.add(RowNumberRunningAggregateDescriptor.FACTORY);
+        fc.add(PercentRankRunningAggregateDescriptor.FACTORY);
+        fc.add(WinPartitionLenRunningAggregateDescriptor.FACTORY);
 
         // boolean functions
         fc.add(AndDescriptor.FACTORY);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
index 6e51559..bccfb45 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Descriptor {@code dense_rank()} window function
+ * Descriptor for {@code dense_rank()} window function
  */
 public class DenseRankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
index 5157451..e3b940d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
@@ -30,7 +30,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Descriptor {@code ntile()} window function
+ * Descriptor for {@code ntile()} window function
  */
 public class NtileRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
index aebef6a..8e4f1f7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 /**
- * Evaluator {@code ntile()} window function
+ * Evaluator for {@code ntile()} window function
  */
 public class NtileRunningAggregateEvaluator implements IWindowAggregateEvaluator {
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
index 180ca99..32282ab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Descriptor {@code percent_rank()} window function
+ * Descriptor for {@code percent_rank()} window function
  */
 public class PercentRankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
index edad37f..6cff7e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
- * Evaluator {@code percent_rank()} window function
+ * Evaluator for {@code percent_rank()} window function
  */
 class PercentRankRunningAggregateEvaluator extends AbstractRankRunningAggregateEvaluator {
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
index d5db134..671393a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
@@ -31,7 +31,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Descriptor {@code rank()} window function
+ * Descriptor for {@code rank()} window function
  */
 public class RankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
index 7464751..2500336 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
@@ -29,7 +29,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
 /**
- * Descriptor {@code row_number()} window function
+ * Descriptor for {@code row_number()} window function
  */
 public class RowNumberRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java
new file mode 100644
index 0000000..234e76d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.asterix.runtime.runningaggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Descriptor for internal {@code win_partition_length()} window function
+ */
+public class WinPartitionLenRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = WinPartitionLenRunningAggregateDescriptor::new;
+
+    @Override
+    public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IRunningAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx) {
+                return new WinPartitionLenRunningAggregateEvaluator();
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.WIN_PARTITION_LENGTH_IMPL;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java
new file mode 100644
index 0000000..d5e0b47
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.asterix.runtime.runningaggregates.std;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.runtime.base.IWindowAggregateEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * Evaluator for internal {@code win_partition_length()} window function
+ */
+public class WinPartitionLenRunningAggregateEvaluator implements IWindowAggregateEvaluator {
+
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<AInt64> serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+
+    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+    @Override
+    public void init() {
+        // nothing to do
+    }
+
+    @Override
+    public void initPartition(long partitionLength) throws HyracksDataException {
+        resultStorage.reset();
+        aInt64.setValue(partitionLength);
+        serde.serialize(aInt64, resultStorage.getDataOutput());
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        result.set(resultStorage);
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
index 61880a2..7fa903c 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
@@ -43,8 +43,8 @@
 /**
  * Merges two adjacent window operators into one if their window specifications are compatible.
  * <pre>
- * window [$x] <- [f()] with nested plan (aggergate [$a] <- [agg_1()] - ... - nts )
- * window [$y] <- [g()] with nesedd plan (aggregate [$b] <- [agg_2()] - ... - nts )
+ * window [$x] <- [f()] with nested plan (aggregate [$a] <- [agg_1()] - ... - nts )
+ * window [$y] <- [g()] with nested plan (aggregate [$b] <- [agg_2()] - ... - nts )
  * -->
  * window [$x, $y] <- [f(), g()] with nested plan ( aggregate [$a, $b] <- [agg_1(), agg_2()] - ... - nts )
  * </pre>

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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>

Change in asterixdb[master]: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/s9WyYo : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1: Integration-Tests-1

Integration Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7760/ : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


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/3113
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3113/2/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
File asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java:

PS2, Line 30: import org.apache.asterix.common.exceptions.ErrorCode;
> Not used I believe.
right. Will fix in the next change


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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/3113

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................

[ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()

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

Details:
- Implement CUME_DIST() and RATIO_TO_REPORT() window functions
- Add internal WIN_PARTITION_LENGTH() window function
- Fixed typos in javadocs

Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/cume_dist_01/cume_dist_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_01/ratio_to_report_01.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ratio_to_report_02_negative/ratio_to_report_02_negative.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/cume_dist_01/cume_dist_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.2.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.3.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/window/ratio_to_report_01/ratio_to_report_01.4.adm
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/SqlppWindowAggregationSugarVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateDescriptor.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinPartitionLenRunningAggregateEvaluator.java
M hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateWindowOperatorsRule.java
31 files changed, 629 insertions(+), 207 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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/3113
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2: Contrib+1

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I921d3e0488b3ca277cffed3b99a9a8e770ac9a24
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-2509][FUN] Implement CUME_DIST() and RATIO_TO_REP...

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

Change subject: [ASTERIXDB-2509][FUN] Implement CUME_DIST() and RATIO_TO_REPORT()
......................................................................


Patch Set 1:

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

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

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