You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2018/11/01 03:54:39 UTC

[1/5] asterixdb git commit: [ASTERIXDB-2466][FUN] Implement window functions

Repository: asterixdb
Updated Branches:
  refs/heads/master b9d55c40e -> fdedf6263


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/AbstractWindowPushRuntime.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/AbstractWindowPushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/AbstractWindowPushRuntime.java
new file mode 100644
index 0000000..f0bd204
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/AbstractWindowPushRuntime.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.runtime.operators.aggrun;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IWindowAggregateEvaluator;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
+
+public abstract class AbstractWindowPushRuntime extends AbstractRunningAggregatePushRuntime<IWindowAggregateEvaluator> {
+
+    private final int[] partitionColumnList;
+    private final IBinaryComparatorFactory[] partitionComparatorFactories;
+    private IBinaryComparator[] partitionComparators;
+    private final IBinaryComparatorFactory[] orderComparatorFactories;
+    private IFrame copyFrame;
+    private FrameTupleAccessor copyFrameAccessor;
+    private FrameTupleAccessor frameAccessor;
+    private long frameId;
+    private boolean inPartition;
+
+    public AbstractWindowPushRuntime(int[] outColumns, IRunningAggregateEvaluatorFactory[] aggFactories,
+            int[] projectionList, int[] partitionColumnList, IBinaryComparatorFactory[] partitionComparatorFactories,
+            IBinaryComparatorFactory[] orderComparatorFactories, IHyracksTaskContext ctx) {
+        super(outColumns, aggFactories, projectionList, ctx, IWindowAggregateEvaluator.class);
+        this.partitionColumnList = partitionColumnList;
+        this.partitionComparatorFactories = partitionComparatorFactories;
+        this.orderComparatorFactories = orderComparatorFactories;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        super.open();
+        frameId = 0;
+        inPartition = false;
+    }
+
+    @Override
+    protected void init() throws HyracksDataException {
+        super.init();
+        partitionComparators = createBinaryComparators(partitionComparatorFactories);
+        frameAccessor = new FrameTupleAccessor(inputRecordDesc);
+        copyFrame = new VSizeFrame(ctx);
+        copyFrameAccessor = new FrameTupleAccessor(inputRecordDesc);
+        copyFrameAccessor.reset(copyFrame.getBuffer());
+        IBinaryComparator[] orderComparators = createBinaryComparators(orderComparatorFactories);
+        for (IWindowAggregateEvaluator aggEval : aggEvals) {
+            aggEval.configure(orderComparators);
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (inPartition) {
+            endPartition();
+        }
+        super.close();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        frameAccessor.reset(buffer);
+        int nTuple = frameAccessor.getTupleCount();
+        if (nTuple == 0) {
+            return;
+        }
+
+        if (frameId == 0) {
+            beginPartition();
+        } else {
+            boolean samePartition = PreclusteredGroupWriter.sameGroup(copyFrameAccessor,
+                    copyFrameAccessor.getTupleCount() - 1, frameAccessor, 0, partitionColumnList, partitionComparators);
+            if (!samePartition) {
+                endPartition();
+                beginPartition();
+            }
+        }
+        if (nTuple == 1) {
+            partitionChunk(frameId, buffer, 0, 0);
+        } else {
+            int tBeginIndex = 0;
+            int tLastIndex = nTuple - 1;
+            for (int tIndex = 1; tIndex <= tLastIndex; tIndex++) {
+                boolean samePartition = PreclusteredGroupWriter.sameGroup(frameAccessor, tIndex - 1, frameAccessor,
+                        tIndex, partitionColumnList, partitionComparators);
+                if (!samePartition) {
+                    partitionChunk(frameId, buffer, tBeginIndex, tIndex - 1);
+                    endPartition();
+                    beginPartition();
+                    tBeginIndex = tIndex;
+                }
+            }
+            partitionChunk(frameId, buffer, tBeginIndex, tLastIndex);
+        }
+
+        copyFrame.resize(buffer.capacity());
+        FrameUtils.copyAndFlip(buffer, copyFrame.getBuffer());
+        copyFrameAccessor.reset(copyFrame.getBuffer());
+        frameId++;
+    }
+
+    private void beginPartition() throws HyracksDataException {
+        if (inPartition) {
+            throw new IllegalStateException();
+        }
+        inPartition = true;
+        beginPartitionImpl();
+    }
+
+    private void partitionChunk(long frameId, ByteBuffer frameBuffer, int beginTupleIdx, int endTupleIdx)
+            throws HyracksDataException {
+        if (!inPartition || frameId < 0) {
+            throw new IllegalStateException();
+        }
+        partitionChunkImpl(frameId, frameBuffer, beginTupleIdx, endTupleIdx);
+    }
+
+    private void endPartition() throws HyracksDataException {
+        if (!inPartition) {
+            throw new IllegalStateException();
+        }
+        endPartitionImpl();
+        inPartition = false;
+    }
+
+    void aggInitPartition(long partitionLength) throws HyracksDataException {
+        for (IWindowAggregateEvaluator aggEval : aggEvals) {
+            aggEval.initPartition(partitionLength);
+        }
+    }
+
+    private static IBinaryComparator[] createBinaryComparators(IBinaryComparatorFactory[] factories) {
+        IBinaryComparator[] comparators = new IBinaryComparator[factories.length];
+        for (int i = 0; i < factories.length; i++) {
+            comparators[i] = factories[i].createBinaryComparator();
+        }
+        return comparators;
+    }
+
+    protected abstract void beginPartitionImpl() throws HyracksDataException;
+
+    protected abstract void partitionChunkImpl(long frameId, ByteBuffer frameBuffer, int tBeginIdx, int tEndIdx)
+            throws HyracksDataException;
+
+    protected abstract void endPartitionImpl() throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/MaterializingWindowPushRuntime.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/MaterializingWindowPushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/MaterializingWindowPushRuntime.java
new file mode 100644
index 0000000..850430f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/MaterializingWindowPushRuntime.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.runtime.operators.aggrun;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.api.comm.FrameHelper;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.storage.common.arraylist.IntArrayList;
+
+class MaterializingWindowPushRuntime extends AbstractWindowPushRuntime {
+
+    private RunFileWriter run;
+
+    private IntArrayList runInfo;
+
+    private long partitionLength;
+
+    private IFrame curFrame;
+
+    private long curFrameId;
+
+    private long runLastFrameId;
+
+    MaterializingWindowPushRuntime(int[] outColumns, IRunningAggregateEvaluatorFactory[] aggFactories,
+            int[] projectionList, int[] partitionColumnList, IBinaryComparatorFactory[] partitionComparatorFactories,
+            IBinaryComparatorFactory[] orderComparatorFactories, IHyracksTaskContext ctx) {
+        super(outColumns, aggFactories, projectionList, partitionColumnList, partitionComparatorFactories,
+                orderComparatorFactories, ctx);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        super.open();
+        run = null;
+        curFrameId = -1;
+    }
+
+    @Override
+    protected void init() throws HyracksDataException {
+        super.init();
+        curFrame = new VSizeFrame(ctx);
+        runInfo = new IntArrayList(128, 128);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        super.close();
+        if (run != null) {
+            run.erase();
+        }
+    }
+
+    @Override
+    protected void beginPartitionImpl() {
+        runInfo.clear();
+        partitionLength = 0;
+        if (run != null) {
+            run.rewind();
+        }
+    }
+
+    @Override
+    protected void partitionChunkImpl(long frameId, ByteBuffer frameBuffer, int tBeginIdx, int tEndIdx)
+            throws HyracksDataException {
+        boolean firstChunk = runInfo.isEmpty();
+        runInfo.add(tBeginIdx);
+        runInfo.add(tEndIdx);
+
+        // save frame. first one to memory, remaining ones to the run file
+        if (firstChunk || tBeginIdx == 0) {
+            int pos = frameBuffer.position();
+            frameBuffer.position(0);
+
+            if (firstChunk) {
+                if (frameId != curFrameId) {
+                    curFrame.resize(curFrame.getMinSize() * FrameHelper.deserializeNumOfMinFrame(frameBuffer));
+                    curFrame.getBuffer().clear();
+                    curFrame.getBuffer().put(frameBuffer);
+                    curFrameId = frameId;
+                }
+            } else {
+                if (run == null) {
+                    FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(getClass().getSimpleName());
+                    run = new RunFileWriter(file, ctx.getIoManager());
+                    run.open();
+                }
+                run.nextFrame(frameBuffer);
+                runLastFrameId = frameId;
+            }
+
+            frameBuffer.position(pos);
+        }
+
+        partitionLength += tEndIdx - tBeginIdx + 1;
+    }
+
+    @Override
+    protected void endPartitionImpl() throws HyracksDataException {
+        aggInitPartition(partitionLength);
+        GeneratedRunFileReader reader = null;
+        try {
+            boolean runRead = false;
+            for (int idx = 0, ln = runInfo.size(); idx < ln; idx += 2) {
+                int tBeginIdx = runInfo.get(idx);
+                int tEndIdx = runInfo.get(idx + 1);
+                if (tBeginIdx == 0 && idx > 0) {
+                    if (reader == null) {
+                        reader = run.createReader();
+                        reader.open();
+                    }
+                    reader.nextFrame(curFrame);
+                    runRead = true;
+                }
+                tAccess.reset(curFrame.getBuffer());
+                produceTuples(tAccess, tBeginIdx, tEndIdx);
+            }
+            if (runRead) {
+                curFrameId = runLastFrameId;
+            }
+        } finally {
+            if (reader != null) {
+                reader.close();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/RunningAggregatePushRuntime.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/RunningAggregatePushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/RunningAggregatePushRuntime.java
new file mode 100644
index 0000000..7c39a21
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/RunningAggregatePushRuntime.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.runtime.operators.aggrun;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+class RunningAggregatePushRuntime extends AbstractRunningAggregatePushRuntime<IRunningAggregateEvaluator> {
+
+    RunningAggregatePushRuntime(int[] outColumns, IRunningAggregateEvaluatorFactory[] aggFactories,
+            int[] projectionList, IHyracksTaskContext ctx) {
+        super(outColumns, aggFactories, projectionList, ctx, IRunningAggregateEvaluator.class);
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        tAccess.reset(buffer);
+        int nTuple = tAccess.getTupleCount();
+        produceTuples(tAccess, 0, nTuple - 1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/RunningAggregateRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/RunningAggregateRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/RunningAggregateRuntimeFactory.java
new file mode 100644
index 0000000..e97ea7d
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/RunningAggregateRuntimeFactory.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.runtime.operators.aggrun;
+
+import java.util.Arrays;
+
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class RunningAggregateRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final int[] outColumns;
+
+    protected final IRunningAggregateEvaluatorFactory[] aggFactories;
+
+    /**
+     * @param outColumns
+     *            a sorted array of columns into which the result is written to
+     * @param aggFactories
+     * @param projectionList
+     *            an array of columns to be projected
+     */
+    public RunningAggregateRuntimeFactory(int[] outColumns, IRunningAggregateEvaluatorFactory[] aggFactories,
+            int[] projectionList) {
+        super(projectionList);
+        this.outColumns = outColumns;
+        this.aggFactories = aggFactories;
+    }
+
+    @Override
+    public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(IHyracksTaskContext ctx) {
+        return new RunningAggregatePushRuntime(outColumns, aggFactories, projectionList, ctx);
+    }
+
+    @Override
+    public String toString() {
+        return "running-aggregate " + Arrays.toString(outColumns) + " := " + Arrays.toString(aggFactories);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/SimpleWindowPushRuntime.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/SimpleWindowPushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/SimpleWindowPushRuntime.java
new file mode 100644
index 0000000..61b06bf
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/SimpleWindowPushRuntime.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.runtime.operators.aggrun;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+class SimpleWindowPushRuntime extends AbstractWindowPushRuntime {
+
+    SimpleWindowPushRuntime(int[] outColumns, IRunningAggregateEvaluatorFactory[] aggFactories, int[] projectionList,
+            int[] partitionColumnList, IBinaryComparatorFactory[] partitionComparatorFactories,
+            IBinaryComparatorFactory[] orderComparatorFactories, IHyracksTaskContext ctx) {
+        super(outColumns, aggFactories, projectionList, partitionColumnList, partitionComparatorFactories,
+                orderComparatorFactories, ctx);
+    }
+
+    @Override
+    protected void beginPartitionImpl() throws HyracksDataException {
+        aggInitPartition(-1);
+    }
+
+    @Override
+    protected void partitionChunkImpl(long frameId, ByteBuffer frameBuffer, int tBeginIdx, int tEndIdx)
+            throws HyracksDataException {
+        tAccess.reset(frameBuffer);
+        produceTuples(tAccess, tBeginIdx, tEndIdx);
+    }
+
+    @Override
+    protected void endPartitionImpl() {
+        // nothing to do
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/WindowRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/WindowRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/WindowRuntimeFactory.java
new file mode 100644
index 0000000..fe7f554
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/WindowRuntimeFactory.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.runtime.operators.aggrun;
+
+import java.util.Arrays;
+
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class WindowRuntimeFactory extends RunningAggregateRuntimeFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int[] partitionColumnList;
+
+    private final IBinaryComparatorFactory[] partitionComparatorFactories;
+
+    private final boolean partitionMaterialization;
+
+    private final IBinaryComparatorFactory[] orderComparatorFactories;
+
+    public WindowRuntimeFactory(int[] outColumns, IRunningAggregateEvaluatorFactory[] aggFactories,
+            int[] projectionList, int[] partitionColumnList, IBinaryComparatorFactory[] partitionComparatorFactories,
+            boolean partitionMaterialization, IBinaryComparatorFactory[] orderComparatorFactories) {
+        super(outColumns, aggFactories, projectionList);
+        this.partitionColumnList = partitionColumnList;
+        this.partitionComparatorFactories = partitionComparatorFactories;
+        this.partitionMaterialization = partitionMaterialization;
+        this.orderComparatorFactories = orderComparatorFactories;
+    }
+
+    @Override
+    public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(IHyracksTaskContext ctx) {
+        return partitionMaterialization
+                ? new MaterializingWindowPushRuntime(outColumns, aggFactories, projectionList, partitionColumnList,
+                        partitionComparatorFactories, orderComparatorFactories, ctx)
+                : new SimpleWindowPushRuntime(outColumns, aggFactories, projectionList, partitionColumnList,
+                        partitionComparatorFactories, orderComparatorFactories, ctx);
+    }
+
+    @Override
+    public String toString() {
+        return "window (" + Arrays.toString(partitionColumnList) + ") " + Arrays.toString(outColumns) + " := "
+                + Arrays.toString(aggFactories);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
deleted file mode 100644
index ca58d4d..0000000
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.algebricks.runtime.operators.std;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
-import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
-import org.apache.hyracks.api.comm.IFrameTupleAccessor;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-
-public class RunningAggregateRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private final int[] outColumns;
-    private final IRunningAggregateEvaluatorFactory[] runningAggregates;
-
-    /**
-     * @param outColumns
-     *            a sorted array of columns into which the result is written to
-     * @param runningAggregates
-     * @param projectionList
-     *            an array of columns to be projected
-     */
-
-    public RunningAggregateRuntimeFactory(int[] outColumns, IRunningAggregateEvaluatorFactory[] runningAggregates,
-            int[] projectionList) {
-        super(projectionList);
-        this.outColumns = outColumns;
-        this.runningAggregates = runningAggregates;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("running-aggregate [");
-        for (int i = 0; i < outColumns.length; i++) {
-            if (i > 0) {
-                sb.append(", ");
-            }
-            sb.append(outColumns[i]);
-        }
-        sb.append("] := [");
-        for (int i = 0; i < runningAggregates.length; i++) {
-            if (i > 0) {
-                sb.append(", ");
-            }
-            sb.append(runningAggregates[i]);
-        }
-        sb.append("]");
-        return sb.toString();
-    }
-
-    @Override
-    public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
-            throws HyracksDataException {
-        final int[] projectionToOutColumns = new int[projectionList.length];
-        for (int j = 0; j < projectionList.length; j++) {
-            projectionToOutColumns[j] = Arrays.binarySearch(outColumns, projectionList[j]);
-        }
-
-        return new AbstractOneInputOneOutputOneFramePushRuntime() {
-            private final IPointable p = VoidPointable.FACTORY.createPointable();
-            private final IRunningAggregateEvaluator[] raggs = new IRunningAggregateEvaluator[runningAggregates.length];
-            private final ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(projectionList.length);
-            private boolean first = true;
-
-            @Override
-            public void open() throws HyracksDataException {
-                initAccessAppendRef(ctx);
-                if (first) {
-                    first = false;
-                    int n = runningAggregates.length;
-                    for (int i = 0; i < n; i++) {
-                        raggs[i] = runningAggregates[i].createRunningAggregateEvaluator(ctx);
-                    }
-                }
-                for (int i = 0; i < runningAggregates.length; i++) {
-                    raggs[i].init();
-                }
-                super.open();
-            }
-
-            @Override
-            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                tAccess.reset(buffer);
-                int nTuple = tAccess.getTupleCount();
-                for (int t = 0; t < nTuple; t++) {
-                    tRef.reset(tAccess, t);
-                    produceTuple(tupleBuilder, tAccess, t, tRef);
-                    appendToFrameFromTupleBuilder(tupleBuilder);
-                }
-            }
-
-            private void produceTuple(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
-                    FrameTupleReference tupleRef) throws HyracksDataException {
-                tb.reset();
-                for (int f = 0; f < projectionList.length; f++) {
-                    int k = projectionToOutColumns[f];
-                    if (k >= 0) {
-                        raggs[k].step(tupleRef, p);
-                        tb.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
-                    } else {
-                        tb.addField(accessor, tIndex, projectionList[f]);
-                    }
-                }
-            }
-
-            @Override
-            public void flush() throws HyracksDataException {
-                appender.flush(writer);
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index d4944af..d235d6d 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -51,7 +51,7 @@ import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.PrinterRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.std.RunningAggregateRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggrun.RunningAggregateRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.SplitOperatorDescriptor;
 import org.apache.hyracks.algebricks.runtime.operators.std.StreamLimitRuntimeFactory;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 7d126ac..690870c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -152,6 +152,7 @@ public class ErrorCode {
     public static final int ONE_TUPLE_RANGEMAP_EXPECTED = 116;
     public static final int NO_RANGEMAP_PRODUCED = 117;
     public static final int RANGEMAP_NOT_FOUND = 118;
+    public static final int UNSUPPORTED_WINDOW_SPEC = 119;
 
     // Compilation error codes.
     public static final int RULECOLLECTION_NOT_INSTANCE_OF_LIST = 10000;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index 50e92b3..413bbee 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -135,6 +135,7 @@
 116 = One tuple rangemap is expected
 117 = No range map produced for parallel sort
 118 = Range map was not found for parallel sort
+119 = Unsupported window specification: PARTITION BY %1$s, ORDER BY %2$s
 
 10000 = The given rule collection %1$s is not an instance of the List class.
 10001 = Cannot compose partition constraint %1$s with %2$s

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
index 915c63d..dabdd4f 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
@@ -49,6 +49,11 @@ public class RunFileWriter implements IFrameWriter {
         maxOutputFrameSize = 0;
     }
 
+    public void rewind() {
+        size = 0;
+        maxOutputFrameSize = 0;
+    }
+
     @Override
     public void fail() throws HyracksDataException {
         ioManager.close(handle);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
index ca78046..c018e9d 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
@@ -21,6 +21,7 @@ package org.apache.hyracks.dataflow.std.group.preclustered;
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -138,9 +139,10 @@ public class PreclusteredGroupWriter implements IFrameWriter {
         }
     }
 
-    private void switchGroupIfRequired(FrameTupleAccessor prevTupleAccessor, int prevTupleIndex,
-            FrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException {
-        if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor, currTupleIndex)) {
+    private void switchGroupIfRequired(IFrameTupleAccessor prevTupleAccessor, int prevTupleIndex,
+            IFrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException {
+        if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor, currTupleIndex, groupFields,
+                comparators)) {
             writeOutput(prevTupleAccessor, prevTupleIndex);
 
             tupleBuilder.reset();
@@ -153,7 +155,7 @@ public class PreclusteredGroupWriter implements IFrameWriter {
         }
     }
 
-    private void writeOutput(final FrameTupleAccessor lastTupleAccessor, int lastTupleIndex)
+    private void writeOutput(final IFrameTupleAccessor lastTupleAccessor, int lastTupleIndex)
             throws HyracksDataException {
 
         tupleBuilder.reset();
@@ -171,8 +173,8 @@ public class PreclusteredGroupWriter implements IFrameWriter {
 
     }
 
-    private boolean sameGroup(FrameTupleAccessor a1, int t1Idx, FrameTupleAccessor a2, int t2Idx)
-            throws HyracksDataException {
+    public static boolean sameGroup(IFrameTupleAccessor a1, int t1Idx, IFrameTupleAccessor a2, int t2Idx,
+            int[] groupFields, IBinaryComparator[] comparators) throws HyracksDataException {
         for (int i = 0; i < comparators.length; ++i) {
             int fIdx = groupFields[i];
             int s1 = a1.getAbsoluteFieldStartOffset(t1Idx, fIdx);


[2/5] asterixdb git commit: [ASTERIXDB-2466][FUN] Implement window functions

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 2b5e569..d91a255 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -69,6 +69,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
@@ -609,7 +610,32 @@ public class IsomorphismOperatorVisitor implements ILogicalOperatorVisitor<Boole
 
     @Override
     public Boolean visitSinkOperator(SinkOperator op, ILogicalOperator arg) throws AlgebricksException {
-        return true;
+        AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+        if (aop.getOperatorTag() != LogicalOperatorTag.SINK) {
+            return Boolean.FALSE;
+        }
+        return Boolean.TRUE;
+    }
+
+    @Override
+    public Boolean visitWindowOperator(WindowOperator op, ILogicalOperator arg) throws AlgebricksException {
+        AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+        if (aop.getOperatorTag() != LogicalOperatorTag.WINDOW) {
+            return Boolean.FALSE;
+        }
+        WindowOperator windowOpArg = (WindowOperator) copyAndSubstituteVar(op, arg);
+        if (!VariableUtilities.varListEqualUnordered(op.getPartitionExpressions(),
+                windowOpArg.getPartitionExpressions())) {
+            return Boolean.FALSE;
+        }
+        if (!compareIOrderAndExpressions(op.getOrderExpressions(), windowOpArg.getOrderExpressions())) {
+            return Boolean.FALSE;
+        }
+        if (!VariableUtilities.varListEqualUnordered(getPairList(op.getVariables(), op.getExpressions()),
+                getPairList(windowOpArg.getVariables(), windowOpArg.getExpressions()))) {
+            return Boolean.FALSE;
+        }
+        return Boolean.TRUE;
     }
 
     private Boolean compareExpressions(List<Mutable<ILogicalExpression>> opExprs,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 742d485..d0aec16 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -68,6 +68,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
@@ -96,6 +97,13 @@ public class IsomorphismVariableMappingVisitor implements ILogicalOperatorVisito
     }
 
     @Override
+    public Void visitWindowOperator(WindowOperator op, ILogicalOperator arg) throws AlgebricksException {
+        mapChildren(op, arg);
+        mapVariablesForAbstractAssign(op, arg);
+        return null;
+    }
+
+    @Override
     public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, ILogicalOperator arg)
             throws AlgebricksException {
         mapVariablesStandard(op, arg);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index 0196db6..198ffdc 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -66,6 +66,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
@@ -611,6 +612,20 @@ public class LogicalOperatorDeepCopyWithNewVariablesVisitor
         return opCopy;
     }
 
+    @Override
+    public ILogicalOperator visitWindowOperator(WindowOperator op, ILogicalOperator arg) throws AlgebricksException {
+        List<Mutable<ILogicalExpression>> partitionExprCopy =
+                exprDeepCopyVisitor.deepCopyExpressionReferenceList(op.getPartitionExpressions());
+        List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExprCopy =
+                deepCopyOrderExpressionReferencePairList(op.getOrderExpressions());
+        List<LogicalVariable> varCopy = deepCopyVariableList(op.getVariables());
+        List<Mutable<ILogicalExpression>> exprCopy =
+                exprDeepCopyVisitor.deepCopyExpressionReferenceList(op.getExpressions());
+        WindowOperator opCopy = new WindowOperator(partitionExprCopy, orderExprCopy, varCopy, exprCopy);
+        deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
+        return opCopy;
+    }
+
     public LinkedHashMap<LogicalVariable, LogicalVariable> getInputToOutputVariableMapping() {
         return inputVarToOutputVarMapping;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 7d3d676..0aaa529 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -60,6 +60,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
@@ -205,6 +206,12 @@ public class LogicalPropertiesVisitor implements ILogicalOperatorVisitor<Void, I
     }
 
     @Override
+    public Void visitWindowOperator(WindowOperator op, IOptimizationContext context) throws AlgebricksException {
+        visitAssignment(op, context);
+        return null;
+    }
+
+    @Override
     public Void visitScriptOperator(ScriptOperator op, IOptimizationContext arg) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
@@ -367,5 +374,4 @@ public class LogicalPropertiesVisitor implements ILogicalOperatorVisitor<Void, I
             throws AlgebricksException {
         return null;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index c6f0c14..e5ca646 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -66,6 +66,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
@@ -406,4 +407,19 @@ public class OperatorDeepCopyVisitor implements ILogicalOperatorVisitor<ILogical
         return new LeftOuterUnnestOperator(op.getVariable(), deepCopyExpressionRef(op.getExpressionRef()),
                 op.getPositionalVariable(), op.getPositionalVariableType(), op.getPositionWriter());
     }
+
+    @Override
+    public ILogicalOperator visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        List<Mutable<ILogicalExpression>> newPartitionExprs = new ArrayList<>();
+        deepCopyExpressionRefs(op.getPartitionExpressions(), newPartitionExprs);
+        List<Pair<IOrder, Mutable<ILogicalExpression>>> newOrderExprs =
+                deepCopyOrderAndExpression(op.getOrderExpressions());
+
+        ArrayList<LogicalVariable> newList = new ArrayList<>();
+        ArrayList<Mutable<ILogicalExpression>> newExpressions = new ArrayList<>();
+        newList.addAll(op.getVariables());
+        deepCopyExpressionRefs(newExpressions, op.getExpressions());
+
+        return new WindowOperator(newPartitionExprs, newOrderExprs, newList, newExpressions);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
index f36f604..eb90288 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
@@ -62,6 +62,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
@@ -289,4 +290,8 @@ public class PrimaryKeyVariablesVisitor implements ILogicalOperatorVisitor<Void,
         return null;
     }
 
+    @Override
+    public Void visitWindowOperator(WindowOperator op, IOptimizationContext arg) throws AlgebricksException {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index 5d0ef6a..43b7c80 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -66,6 +66,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
@@ -91,6 +92,12 @@ public class ProducedVariableVisitor implements ILogicalOperatorVisitor<Void, Vo
     }
 
     @Override
+    public Void visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        producedVariables.addAll(op.getVariables());
+        return null;
+    }
+
+    @Override
     public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
         producedVariables.addAll(op.getVariables());
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 70ccf6d..69b17ed 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -64,6 +64,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
@@ -343,4 +344,9 @@ public class SchemaVariableVisitor implements ILogicalOperatorVisitor<Void, Void
         return null;
     }
 
+    @Override
+    public Void visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index c62f555..99d3488 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -65,6 +65,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
@@ -85,15 +86,7 @@ public class SubstituteVariableVisitor
     @Override
     public Void visitAggregateOperator(AggregateOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
-        List<LogicalVariable> variables = op.getVariables();
-        int n = variables.size();
-        for (int i = 0; i < n; i++) {
-            if (variables.get(i).equals(pair.first)) {
-                variables.set(i, pair.second);
-            } else {
-                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);
-            }
-        }
+        substAssignVariables(op.getVariables(), op.getExpressions(), pair);
         substVarTypes(op, pair);
         return null;
     }
@@ -101,15 +94,7 @@ public class SubstituteVariableVisitor
     @Override
     public Void visitAssignOperator(AssignOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
-        List<LogicalVariable> variables = op.getVariables();
-        int n = variables.size();
-        for (int i = 0; i < n; i++) {
-            if (variables.get(i).equals(pair.first)) {
-                variables.set(i, pair.second);
-            } else {
-                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);
-            }
-        }
+        substAssignVariables(op.getVariables(), op.getExpressions(), pair);
         // Substitute variables stored in ordering property
         if (op.getExplicitOrderingProperty() != null) {
             List<OrderColumn> orderColumns = op.getExplicitOrderingProperty().getOrderColumns();
@@ -134,10 +119,10 @@ public class SubstituteVariableVisitor
                 return null;
             }
         }
-        substVarTypes(op, pair);
         if (op.getSelectCondition() != null) {
             op.getSelectCondition().getValue().substituteVar(pair.first, pair.second);
         }
+        substVarTypes(op, pair);
         return null;
     }
 
@@ -240,15 +225,7 @@ public class SubstituteVariableVisitor
     @Override
     public Void visitRunningAggregateOperator(RunningAggregateOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
-        List<LogicalVariable> variables = op.getVariables();
-        int n = variables.size();
-        for (int i = 0; i < n; i++) {
-            if (variables.get(i).equals(pair.first)) {
-                variables.set(i, pair.second);
-            } else {
-                op.getExpressions().get(i).getValue().substituteVar(pair.first, pair.second);
-            }
-        }
+        substAssignVariables(op.getVariables(), op.getExpressions(), pair);
         substVarTypes(op, pair);
         return null;
     }
@@ -403,6 +380,18 @@ public class SubstituteVariableVisitor
         }
     }
 
+    private void substAssignVariables(List<LogicalVariable> variables, List<Mutable<ILogicalExpression>> expressions,
+            Pair<LogicalVariable, LogicalVariable> pair) {
+        int n = variables.size();
+        for (int i = 0; i < n; i++) {
+            if (variables.get(i).equals(pair.first)) {
+                variables.set(i, pair.second);
+            } else {
+                expressions.get(i).getValue().substituteVar(pair.first, pair.second);
+            }
+        }
+    }
+
     @Override
     public Void visitReplicateOperator(ReplicateOperator op, Pair<LogicalVariable, LogicalVariable> arg)
             throws AlgebricksException {
@@ -510,4 +499,18 @@ public class SubstituteVariableVisitor
         substVarTypes(op, pair);
         return null;
     }
+
+    @Override
+    public Void visitWindowOperator(WindowOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        for (Mutable<ILogicalExpression> pe : op.getPartitionExpressions()) {
+            pe.getValue().substituteVar(pair.first, pair.second);
+        }
+        for (Pair<IOrder, Mutable<ILogicalExpression>> oe : op.getOrderExpressions()) {
+            oe.second.getValue().substituteVar(pair.first, pair.second);
+        }
+        substAssignVariables(op.getVariables(), op.getExpressions(), pair);
+        substVarTypes(op, pair);
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 2c68697..b4bea84 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -67,6 +67,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
@@ -471,4 +472,17 @@ public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void>
         return null;
     }
 
+    @Override
+    public Void visitWindowOperator(WindowOperator op, Void arg) {
+        for (Mutable<ILogicalExpression> exprRef : op.getPartitionExpressions()) {
+            exprRef.getValue().getUsedVariables(usedVariables);
+        }
+        for (Pair<IOrder, Mutable<ILogicalExpression>> oe : op.getOrderExpressions()) {
+            oe.second.getValue().getUsedVariables(usedVariables);
+        }
+        for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {
+            exprRef.getValue().getUsedVariables(usedVariables);
+        }
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
index d68be20..09ee358 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
@@ -40,7 +40,7 @@ import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertie
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.operators.std.RunningAggregateRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggrun.RunningAggregateRuntimeFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 
 public class RunningAggregatePOperator extends AbstractPhysicalOperator {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WindowPOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WindowPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WindowPOperator.java
new file mode 100644
index 0000000..7853524
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WindowPOperator.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.ListSet;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
+import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggrun.WindowRuntimeFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+
+public class WindowPOperator extends AbstractPhysicalOperator {
+
+    private final List<LogicalVariable> partitionColumns;
+
+    private final boolean partitionMaterialization;
+
+    private final List<OrderColumn> orderColumns;
+
+    public WindowPOperator(List<LogicalVariable> partitionColumns, boolean partitionMaterialization,
+            List<OrderColumn> orderColumns) {
+        this.partitionColumns = partitionColumns;
+        this.partitionMaterialization = partitionMaterialization;
+        this.orderColumns = orderColumns;
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.WINDOW;
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) throws AlgebricksException {
+        IPartitioningProperty pp;
+        switch (op.getExecutionMode()) {
+            case PARTITIONED:
+                pp = new UnorderedPartitionedProperty(new ListSet<>(partitionColumns),
+                        context.getComputationNodeDomain());
+                break;
+            case UNPARTITIONED:
+                pp = IPartitioningProperty.UNPARTITIONED;
+                break;
+            case LOCAL:
+                pp = null;
+                break;
+            default:
+                throw new IllegalStateException(op.getExecutionMode().name());
+        }
+
+        // require local order property [pc1, ... pcN, oc1, ... ocN]
+        // accounting for cases where there's an overlap between order and partition columns
+        // TODO replace with required local grouping on partition columns + local order on order columns
+        List<OrderColumn> lopColumns = new ArrayList<>();
+        ListSet<LogicalVariable> pcVars = new ListSet<>();
+        pcVars.addAll(partitionColumns);
+        for (int oIdx = 0, ln = orderColumns.size(); oIdx < ln; oIdx++) {
+            OrderColumn oc = orderColumns.get(oIdx);
+            LogicalVariable ocVar = oc.getColumn();
+            if (!pcVars.remove(ocVar) && containsAny(orderColumns, oIdx + 1, pcVars)) {
+                throw new AlgebricksException(ErrorCode.HYRACKS, ErrorCode.UNSUPPORTED_WINDOW_SPEC,
+                        op.getSourceLocation(), String.valueOf(partitionColumns), String.valueOf(orderColumns));
+            }
+            lopColumns.add(new OrderColumn(oc.getColumn(), oc.getOrder()));
+        }
+        int pIdx = 0;
+        for (LogicalVariable pColumn : pcVars) {
+            lopColumns.add(pIdx++, new OrderColumn(pColumn, OrderOperator.IOrder.OrderKind.ASC));
+        }
+        List<ILocalStructuralProperty> localProps = Collections.singletonList(new LocalOrderProperty(lopColumns));
+
+        return new PhysicalRequirements(
+                new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp, localProps) },
+                IPartitioningRequirementsCoordinator.NO_COORDINATION);
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        WindowOperator winOp = (WindowOperator) op;
+        int[] outColumns = JobGenHelper.projectVariables(opSchema, winOp.getVariables());
+        List<Mutable<ILogicalExpression>> expressions = winOp.getExpressions();
+        IRunningAggregateEvaluatorFactory[] winFuncs = new IRunningAggregateEvaluatorFactory[expressions.size()];
+        IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
+        for (int i = 0; i < winFuncs.length; i++) {
+            StatefulFunctionCallExpression expr = (StatefulFunctionCallExpression) expressions.get(i).getValue();
+            winFuncs[i] = expressionRuntimeProvider.createRunningAggregateFunctionFactory(expr,
+                    context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas, context);
+        }
+
+        // TODO push projections into the operator
+        int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
+
+        int[] partitionColumnList = JobGenHelper.projectVariables(inputSchemas[0], partitionColumns);
+
+        IBinaryComparatorFactory[] partitionComparatorFactories = JobGenHelper
+                .variablesToAscBinaryComparatorFactories(partitionColumns, context.getTypeEnvironment(op), context);
+
+        //TODO not all functions need order comparators
+        IBinaryComparatorFactory[] orderComparatorFactories = JobGenHelper
+                .variablesToBinaryComparatorFactories(orderColumns, context.getTypeEnvironment(op), context);
+
+        WindowRuntimeFactory runtime = new WindowRuntimeFactory(outColumns, winFuncs, projectionList,
+                partitionColumnList, partitionComparatorFactories, partitionMaterialization, orderComparatorFactories);
+        runtime.setSourceLocation(winOp.getSourceLocation());
+
+        // contribute one Asterix framewriter
+        RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
+        builder.contributeMicroOperator(winOp, runtime, recDesc);
+        // and contribute one edge from its child
+        ILogicalOperator src = winOp.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, winOp, 0);
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return true;
+    }
+
+    @Override
+    public boolean expensiveThanMaterialization() {
+        return true;
+    }
+
+    public boolean isPartitionMaterialization() {
+        return partitionMaterialization;
+    }
+
+    private boolean containsAny(List<OrderColumn> ocList, int startIdx, Set<LogicalVariable> varSet) {
+        for (int i = startIdx, ln = ocList.size(); i < ln; i++) {
+            if (varSet.contains(ocList.get(i).getColumn())) {
+                return true;
+            }
+        }
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 77f052e..ad45614 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -66,6 +66,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
@@ -170,13 +171,10 @@ public class LogicalOperatorPrettyPrintVisitor extends AbstractLogicalOperatorPr
     @Override
     public Void visitOrderOperator(OrderOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("order ");
-        for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : op.getOrderExpressions()) {
-            if (op.getTopK() != -1) {
-                buffer.append("(topK: " + op.getTopK() + ") ");
-            }
-            String fst = getOrderString(p.first);
-            buffer.append("(" + fst + ", " + p.second.getValue().accept(exprVisitor, indent) + ") ");
+        if (op.getTopK() != -1) {
+            buffer.append("(topK: " + op.getTopK() + ") ");
         }
+        pprintOrderList(op.getOrderExpressions(), indent);
         return null;
     }
 
@@ -484,6 +482,17 @@ public class LogicalOperatorPrettyPrintVisitor extends AbstractLogicalOperatorPr
         return null;
     }
 
+    @Override
+    public Void visitWindowOperator(WindowOperator op, Integer indent) throws AlgebricksException {
+        addIndent(indent).append("window ").append(str(op.getVariables())).append(" <- ");
+        pprintExprList(op.getExpressions(), indent);
+        buffer.append(" partition ");
+        pprintExprList(op.getPartitionExpressions(), indent);
+        buffer.append(" order ");
+        pprintOrderList(op.getOrderExpressions(), indent);
+        return null;
+    }
+
     protected void printNestedPlans(AbstractOperatorWithNestedPlans op, Integer indent) throws AlgebricksException {
         boolean first = true;
         if (op.getNestedPlans().isEmpty()) {
@@ -537,4 +546,12 @@ public class LogicalOperatorPrettyPrintVisitor extends AbstractLogicalOperatorPr
         }
         buffer.append("]");
     }
+
+    protected void pprintOrderList(List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderList,
+            Integer indent) throws AlgebricksException {
+        for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderList) {
+            String fst = getOrderString(p.first);
+            buffer.append("(" + fst + ", " + p.second.getValue().accept(exprVisitor, indent) + ") ");
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 4a17cc6..4c810ab 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -70,6 +70,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 
@@ -251,30 +252,12 @@ public class LogicalOperatorPrettyPrintVisitorJson extends AbstractLogicalOperat
     @Override
     public Void visitOrderOperator(OrderOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("\"operator\": \"order\"");
-        for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : op.getOrderExpressions()) {
-            buffer.append(",\n");
-            if (op.getTopK() != -1) {
-                addIndent(indent).append("\"topK\": \"" + op.getTopK() + "\",\n");
-            }
-            String fst = getOrderString(p.first);
-            addIndent(indent).append("\"first\": " + fst + ",\n");
-            addIndent(indent).append(
-                    "\"second\": \"" + p.second.getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
-        }
+        int topK = op.getTopK();
+        List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExpressions = op.getOrderExpressions();
+        pprintOrderExprList(orderExpressions, topK, indent);
         return null;
     }
 
-    private String getOrderString(OrderOperator.IOrder first) {
-        switch (first.getKind()) {
-            case ASC:
-                return "\"ASC\"";
-            case DESC:
-                return "\"DESC\"";
-            default:
-                return first.getExpressionRef().toString();
-        }
-    }
-
     @Override
     public Void visitAssignOperator(AssignOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("\"operator\": \"assign\"");
@@ -667,6 +650,23 @@ public class LogicalOperatorPrettyPrintVisitorJson extends AbstractLogicalOperat
         return null;
     }
 
+    @Override
+    public Void visitWindowOperator(WindowOperator op, Integer indent) throws AlgebricksException {
+        addIndent(indent).append("\"operator\": \"window\"");
+        variablePrintHelper(op.getVariables(), indent);
+        addIndent(0).append(",\n");
+        pprintExprList(op.getExpressions(), indent);
+        if (!op.getPartitionExpressions().isEmpty()) {
+            buffer.append(",\n");
+            addIndent(indent).append("\"partition by\": ");
+            pprintExprList(op.getPartitionExpressions(), indent);
+        }
+        buffer.append(",\n");
+        addIndent(indent).append("\"order by\": ");
+        pprintOrderExprList(op.getOrderExpressions(), -1, indent);
+        return null;
+    }
+
     protected void printNestedPlans(AbstractOperatorWithNestedPlans op, Integer indent) throws AlgebricksException {
         idCounter.nextPrefix();
         buffer.append("[\n");
@@ -718,4 +718,29 @@ public class LogicalOperatorPrettyPrintVisitorJson extends AbstractLogicalOperat
         }
         buffer.append("]");
     }
+
+    private void pprintOrderExprList(List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExpressions,
+            int topK, Integer indent) throws AlgebricksException {
+        for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExpressions) {
+            buffer.append(",\n");
+            if (topK != -1) {
+                addIndent(indent).append("\"topK\": \"" + topK + "\",\n");
+            }
+            String fst = getOrderString(p.first);
+            addIndent(indent).append("\"first\": " + fst + ",\n");
+            addIndent(indent).append(
+                    "\"second\": \"" + p.second.getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
+        }
+    }
+
+    private String getOrderString(OrderOperator.IOrder first) {
+        switch (first.getKind()) {
+            case ASC:
+                return "\"ASC\"";
+            case DESC:
+                return "\"DESC\"";
+            default:
+                return first.getExpressionRef().toString();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
index 67199b9..ea914fa 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
@@ -43,6 +43,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOpera
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.LogicalOperatorDeepCopyWithNewVariablesVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.OperatorDeepCopyVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
@@ -134,6 +135,13 @@ public class OperatorManipulationUtil {
                         forceUnpartitioned = true;
                     }
                 }
+                if (op.getOperatorTag() == LogicalOperatorTag.WINDOW) {
+                    WindowOperator winOp = (WindowOperator) op;
+                    if (winOp.getPartitionExpressions().isEmpty()) {
+                        op.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+                        forceUnpartitioned = true;
+                    }
+                }
 
                 for (Mutable<ILogicalOperator> i : op.getInputs()) {
                     boolean exit = false;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index 548a29f..9d5cdeb 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -52,6 +52,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 
@@ -126,4 +127,6 @@ public interface ILogicalOperatorVisitor<R, T> {
     public R visitTokenizeOperator(TokenizeOperator op, T arg) throws AlgebricksException;
 
     public R visitForwardOperator(ForwardOperator op, T arg) throws AlgebricksException;
+
+    public R visitWindowOperator(WindowOperator op, T arg) throws AlgebricksException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
index b204bcb..5142ce7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
@@ -25,6 +25,8 @@ import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
 import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFamilyProvider;
@@ -144,6 +146,20 @@ public final class JobGenHelper {
         return compFactories;
     }
 
+    public static IBinaryComparatorFactory[] variablesToBinaryComparatorFactories(Collection<OrderColumn> orderColumns,
+            IVariableTypeEnvironment env, JobGenContext context) throws AlgebricksException {
+        IBinaryComparatorFactory[] compFactories = new IBinaryComparatorFactory[orderColumns.size()];
+        IBinaryComparatorFactoryProvider bcfProvider = context.getBinaryComparatorFactoryProvider();
+        int i = 0;
+        for (OrderColumn oc : orderColumns) {
+            LogicalVariable v = oc.getColumn();
+            boolean ascending = oc.getOrder() == OrderOperator.IOrder.OrderKind.ASC;
+            Object type = env.getVarType(v);
+            compFactories[i++] = bcfProvider.getBinaryComparatorFactory(type, ascending);
+        }
+        return compFactories;
+    }
+
     public static INormalizedKeyComputerFactory variablesToAscNormalizedKeyComputerFactory(
             Collection<LogicalVariable> varLogical, IVariableTypeEnvironment env, JobGenContext context)
             throws AlgebricksException {
@@ -181,12 +197,20 @@ public final class JobGenHelper {
     }
 
     public static int[] projectAllVariables(IOperatorSchema opSchema) {
-        int[] projectionList = new int[opSchema.getSize()];
+        return projectVariablesImpl(opSchema, opSchema, opSchema.getSize());
+    }
+
+    public static int[] projectVariables(IOperatorSchema opSchema, List<LogicalVariable> variables) {
+        return projectVariablesImpl(opSchema, variables, variables.size());
+    }
+
+    private static int[] projectVariablesImpl(IOperatorSchema opSchema, Iterable<LogicalVariable> variables,
+            int variableCount) {
+        int[] projectionList = new int[variableCount];
         int k = 0;
-        for (LogicalVariable v : opSchema) {
+        for (LogicalVariable v : variables) {
             projectionList[k++] = opSchema.findVariable(v);
         }
         return projectionList;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 113d205..f10c3a4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -67,6 +67,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
@@ -192,17 +193,7 @@ public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String
                 stringBuilder.append("(topK: ").append(op.getTopK()).append(") ");
             }
             stringBuilder.append("(");
-            switch (p.first.getKind()) {
-                case ASC:
-                    stringBuilder.append("ASC");
-                    break;
-                case DESC:
-                    stringBuilder.append("DESC");
-                    break;
-                default:
-                    final Mutable<ILogicalExpression> expressionRef = p.first.getExpressionRef();
-                    stringBuilder.append(expressionRef == null ? "null" : expressionRef.toString());
-            }
+            appendOrder(p.first);
             stringBuilder.append(", ").append(p.second.getValue().toString()).append(") ");
         }
         appendSchema(op, showDetails);
@@ -211,6 +202,20 @@ public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String
         return stringBuilder.toString();
     }
 
+    private void appendOrder(OrderOperator.IOrder order) {
+        switch (order.getKind()) {
+            case ASC:
+                stringBuilder.append("ASC");
+                break;
+            case DESC:
+                stringBuilder.append("DESC");
+                break;
+            default:
+                final Mutable<ILogicalExpression> expressionRef = order.getExpressionRef();
+                stringBuilder.append(expressionRef == null ? "null" : expressionRef.toString());
+        }
+    }
+
     @Override
     public String visitAssignOperator(AssignOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
@@ -595,6 +600,26 @@ public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String
         return stringBuilder.toString();
     }
 
+    @Override
+    public String visitWindowOperator(WindowOperator op, Boolean showDetails) throws AlgebricksException {
+        stringBuilder.setLength(0);
+        stringBuilder.append("window (").append(str(op.getVariables())).append(" <- ");
+        printExprList(op.getExpressions());
+        stringBuilder.append(") partition by (");
+        printExprList(op.getPartitionExpressions());
+        stringBuilder.append(") order by (");
+        for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : op.getOrderExpressions()) {
+            stringBuilder.append("(");
+            appendOrder(p.first);
+            stringBuilder.append(", ").append(p.second.getValue().toString()).append(") ");
+        }
+        stringBuilder.append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
+        return stringBuilder.toString();
+    }
+
     private void printExprList(List<Mutable<ILogicalExpression>> expressions) {
         stringBuilder.append("[");
         expressions.forEach(exprRef -> stringBuilder.append(exprRef.getValue().toString()).append(", "));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
index 036b3e1..7adc732 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
@@ -18,26 +18,30 @@
  */
 package org.apache.hyracks.algebricks.rewriter.rules;
 
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 public abstract class AbstractExtractExprRule implements IAlgebraicRewriteRule {
 
-    protected LogicalVariable extractExprIntoAssignOpRef(ILogicalExpression gExpr, Mutable<ILogicalOperator> opRef2,
-            IOptimizationContext context) throws AlgebricksException {
+    protected static LogicalVariable extractExprIntoAssignOpRef(ILogicalExpression gExpr,
+            Mutable<ILogicalOperator> opRef2, IOptimizationContext context) throws AlgebricksException {
         LogicalVariable v = context.newVar();
-        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(gExpr));
+        AssignOperator a = new AssignOperator(v, new MutableObject<>(gExpr));
         a.setSourceLocation(gExpr.getSourceLocation());
-        a.getInputs().add(new MutableObject<ILogicalOperator>(opRef2.getValue()));
+        a.getInputs().add(new MutableObject<>(opRef2.getValue()));
         opRef2.setValue(a);
         if (gExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
             context.addNotToBeInlinedVar(v);
@@ -46,4 +50,37 @@ public abstract class AbstractExtractExprRule implements IAlgebraicRewriteRule {
         return v;
     }
 
+    protected static <T> boolean extractComplexExpressions(ILogicalOperator op, List<T> exprList,
+            Function<T, Mutable<ILogicalExpression>> exprGetter, Predicate<ILogicalExpression> retainPredicate,
+            IOptimizationContext context) throws AlgebricksException {
+        if (!hasComplexExpressions(exprList, exprGetter)) {
+            return false;
+        }
+        boolean rewritten = false;
+        Mutable<ILogicalOperator> inputOpRef = op.getInputs().get(0);
+        for (T item : exprList) {
+            Mutable<ILogicalExpression> exprMutable = exprGetter.apply(item);
+            ILogicalExpression expr = exprMutable.getValue();
+            if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE && !retainPredicate.test(expr)) {
+                LogicalVariable v = extractExprIntoAssignOpRef(expr, inputOpRef, context);
+                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+                vRef.setSourceLocation(expr.getSourceLocation());
+                exprMutable.setValue(vRef);
+                rewritten = true;
+            }
+        }
+        context.computeAndSetTypeEnvironmentForOperator(op);
+        return rewritten;
+    }
+
+    private static <T> boolean hasComplexExpressions(List<T> exprList,
+            Function<T, Mutable<ILogicalExpression>> exprGetter) {
+        for (T item : exprList) {
+            Mutable<ILogicalExpression> exprMutable = exprGetter.apply(item);
+            if (exprMutable.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                return true;
+            }
+        }
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index cdab2f4..52b3f59 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -515,7 +515,7 @@ public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
                 case LOCAL_GROUPING_PROPERTY: {
                     LocalGroupingProperty g = (LocalGroupingProperty) prop;
                     Collection<LogicalVariable> vars =
-                            (g.getPreferredOrderEnforcer() != null) ? g.getPreferredOrderEnforcer() : g.getColumnSet();
+                            !g.getPreferredOrderEnforcer().isEmpty() ? g.getPreferredOrderEnforcer() : g.getColumnSet();
                     List<OrderColumn> orderColumns = new ArrayList<>();
                     for (LogicalVariable v : vars) {
                         OrderColumn oc = new OrderColumn(v, OrderKind.ASC);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
index eb2bee6..06b2e16 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
@@ -18,17 +18,17 @@
  */
 package org.apache.hyracks.algebricks.rewriter.rules;
 
-import org.apache.commons.lang3.mutable.Mutable;
+import java.util.List;
+import java.util.function.Function;
 
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 
@@ -55,8 +55,8 @@ public class ExtractGbyExpressionsRule extends AbstractExtractExprRule {
         }
         context.addToDontApplySet(this, op1);
         GroupByOperator g = (GroupByOperator) op1;
-        boolean r1 = gbyExprWasRewritten(g, context);
-        boolean r2 = decorExprWasRewritten(g, context);
+        boolean r1 = extractComplexExpressions(g, g.getGroupByList(), context);
+        boolean r2 = extractComplexExpressions(g, g.getDecorList(), context);
         boolean fired = r1 || r2;
         if (fired) {
             context.computeAndSetTypeEnvironmentForOperator(g);
@@ -64,56 +64,15 @@ public class ExtractGbyExpressionsRule extends AbstractExtractExprRule {
         return fired;
     }
 
-    private boolean gbyExprWasRewritten(GroupByOperator g, IOptimizationContext context) throws AlgebricksException {
-        if (!gbyHasComplexExpr(g)) {
-            return false;
-        }
-        Mutable<ILogicalOperator> opRef2 = g.getInputs().get(0);
-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : g.getGroupByList()) {
-            ILogicalExpression expr = gbyPair.second.getValue();
-            if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
-                vRef.setSourceLocation(expr.getSourceLocation());
-                gbyPair.second.setValue(vRef);
-            }
-        }
-        return true;
-    }
-
-    private boolean decorExprWasRewritten(GroupByOperator g, IOptimizationContext context) throws AlgebricksException {
-        if (!decorHasComplexExpr(g)) {
-            return false;
-        }
-        Mutable<ILogicalOperator> opRef2 = g.getInputs().get(0);
-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> decorPair : g.getDecorList()) {
-            ILogicalExpression expr = decorPair.second.getValue();
-            if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
-                vRef.setSourceLocation(expr.getSourceLocation());
-                decorPair.second.setValue(vRef);
-            }
-        }
-        return true;
-    }
-
-    private boolean gbyHasComplexExpr(GroupByOperator g) {
-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : g.getGroupByList()) {
-            if (gbyPair.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                return true;
-            }
-        }
-        return false;
+    private static boolean extractComplexExpressions(ILogicalOperator op,
+            List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> exprList, IOptimizationContext context)
+            throws AlgebricksException {
+        return extractComplexExpressions(op, exprList, Pair::getSecond, context);
     }
 
-    private boolean decorHasComplexExpr(GroupByOperator g) {
-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : g.getDecorList()) {
-            if (gbyPair.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                return true;
-            }
-        }
-        return false;
+    public static <T> boolean extractComplexExpressions(ILogicalOperator op, List<T> exprList,
+            Function<T, Mutable<ILogicalExpression>> exprGetter, IOptimizationContext context)
+            throws AlgebricksException {
+        return extractComplexExpressions(op, exprList, exprGetter, t -> false, context);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java
index aa58985..e6f86be 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushNestedOrderByUnderPreSortedGroupByRule.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.algebricks.rewriter.rules;
 
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -39,6 +40,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperato
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
@@ -82,6 +84,10 @@ public class PushNestedOrderByUnderPreSortedGroupByRule implements IAlgebraicRew
         if (!isIndependentFromChildren(order1)) {
             return false;
         }
+        if (OperatorManipulationUtil.ancestorOfOperators(order1.getInputs().get(0).getValue(),
+                EnumSet.of(LogicalOperatorTag.ORDER))) {
+            return false;
+        }
         AbstractPhysicalOperator pOrder1 = (AbstractPhysicalOperator) op2.getPhysicalOperator();
         if (pOrder1.getOperatorTag() != PhysicalOperatorTag.STABLE_SORT
                 && pOrder1.getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
index 6b09894..1388ccb 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
@@ -60,6 +60,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.LogicalOperatorDeepCopyWithNewVariablesVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.visitors.IQueryOperatorVisitor;
@@ -256,6 +257,11 @@ class ReplaceNtsWithSubplanInputOperatorVisitor implements IQueryOperatorVisitor
         return visit(op);
     }
 
+    @Override
+    public ILogicalOperator visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        return visit(op);
+    }
+
     private ILogicalOperator visit(ILogicalOperator op) throws AlgebricksException {
         List<Map<LogicalVariable, LogicalVariable>> varMapSnapshots = new ArrayList<>();
         for (Mutable<ILogicalOperator> childRef : op.getInputs()) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml b/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
index bcc537a..c8f7cbf 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
@@ -73,6 +73,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IWindowAggregateEvaluator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IWindowAggregateEvaluator.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IWindowAggregateEvaluator.java
new file mode 100644
index 0000000..b4030f7
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IWindowAggregateEvaluator.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.runtime.base;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IWindowAggregateEvaluator extends IRunningAggregateEvaluator {
+    default void configure(IBinaryComparator[] orderComparators) {
+    }
+
+    void initPartition(long partitionLength) throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/AbstractRunningAggregatePushRuntime.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/AbstractRunningAggregatePushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/AbstractRunningAggregatePushRuntime.java
new file mode 100644
index 0000000..27354cb
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggrun/AbstractRunningAggregatePushRuntime.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.runtime.operators.aggrun;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public abstract class AbstractRunningAggregatePushRuntime<T extends IRunningAggregateEvaluator>
+        extends AbstractOneInputOneOutputOneFramePushRuntime {
+    protected final IHyracksTaskContext ctx;
+    private final IRunningAggregateEvaluatorFactory[] aggFactories;
+    private final Class<T> aggEvalClass;
+    protected final List<T> aggEvals;
+    private final int[] projectionList;
+    private final int[] projectionToOutColumns;
+    private final IPointable p = VoidPointable.FACTORY.createPointable();
+    private final ArrayTupleBuilder tupleBuilder;
+    private boolean first;
+
+    public AbstractRunningAggregatePushRuntime(int[] outColumns, IRunningAggregateEvaluatorFactory[] aggFactories,
+            int[] projectionList, IHyracksTaskContext ctx, Class<T> aggEvalClass) {
+        this.ctx = ctx;
+        this.projectionList = projectionList;
+        this.aggFactories = aggFactories;
+        this.aggEvalClass = aggEvalClass;
+        aggEvals = new ArrayList<>(aggFactories.length);
+        tupleBuilder = new ArrayTupleBuilder(projectionList.length);
+        projectionToOutColumns = new int[projectionList.length];
+
+        for (int j = 0; j < projectionList.length; j++) {
+            projectionToOutColumns[j] = Arrays.binarySearch(outColumns, projectionList[j]);
+        }
+        first = true;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        super.open();
+        if (first) {
+            first = false;
+            init();
+        }
+        for (T aggEval : aggEvals) {
+            aggEval.init();
+        }
+    }
+
+    protected void init() throws HyracksDataException {
+        initAccessAppendRef(ctx);
+        for (IRunningAggregateEvaluatorFactory aggFactory : aggFactories) {
+            IRunningAggregateEvaluator aggEval = aggFactory.createRunningAggregateEvaluator(ctx);
+            aggEvals.add(aggEvalClass.cast(aggEval));
+        }
+    }
+
+    protected void produceTuples(IFrameTupleAccessor accessor, int beginIdx, int endIdx) throws HyracksDataException {
+        for (int t = beginIdx; t <= endIdx; t++) {
+            tRef.reset(accessor, t);
+            produceTuple(tupleBuilder, accessor, t, tRef);
+            appendToFrameFromTupleBuilder(tupleBuilder);
+        }
+    }
+
+    private void produceTuple(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+            FrameTupleReference tupleRef) throws HyracksDataException {
+        tb.reset();
+        for (int f = 0; f < projectionList.length; f++) {
+            int k = projectionToOutColumns[f];
+            if (k >= 0) {
+                aggEvals.get(k).step(tupleRef, p);
+                tb.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
+            } else {
+                tb.addField(accessor, tIndex, projectionList[f]);
+            }
+        }
+    }
+
+    @Override
+    public void flush() throws HyracksDataException {
+        appender.flush(writer);
+    }
+}
\ No newline at end of file


[3/5] asterixdb git commit: [ASTERIXDB-2466][FUN] Implement window functions

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
index b754533..3fb3507 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
@@ -62,6 +62,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppQueryExpressionVisitor;
@@ -424,6 +425,16 @@ public class FreeVariableVisitor extends AbstractSqlppQueryExpressionVisitor<Voi
         return null;
     }
 
+    @Override
+    public Void visit(WindowExpression winExpr, Collection<VariableExpr> freeVars) throws CompilationException {
+        winExpr.getExpr().accept(this, freeVars);
+        if (winExpr.hasPartitionList()) {
+            visit(winExpr.getPartitionList(), freeVars);
+        }
+        visit(winExpr.getOrderbyList(), freeVars);
+        return null;
+    }
+
     private void visitLetClauses(List<LetClause> letClauses, Collection<VariableExpr> freeVars)
             throws CompilationException {
         if (letClauses == null || letClauses.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
index 0973bec..e4d2a27 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
@@ -27,6 +27,7 @@ import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
+import org.apache.asterix.lang.common.clause.OrderbyClause;
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
 import org.apache.asterix.lang.common.expression.VariableExpr;
@@ -47,6 +48,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
@@ -331,4 +333,28 @@ public class SqlppAstPrintVisitor extends QueryPrintVisitor implements ISqlppVis
         return null;
     }
 
+    @Override
+    public Void visit(WindowExpression winExpr, Integer step) throws CompilationException {
+        out.print(skip(step) + "WINDOW");
+        winExpr.getExpr().accept(this, step + 1);
+        out.println();
+        out.println(skip(step) + "OVER (");
+        if (winExpr.hasPartitionList()) {
+            out.println(skip(step + 1) + "PARTITION BY");
+            List<Expression> partitionList = winExpr.getPartitionList();
+            for (Expression expr : partitionList) {
+                expr.accept(this, step + 2);
+                out.println();
+            }
+        }
+        out.println(skip(step + 1) + "ORDER BY");
+        List<Expression> orderbyList = winExpr.getOrderbyList();
+        List<OrderbyClause.OrderModifier> orderbyModifierList = winExpr.getOrderbyModifierList();
+        for (int i = 0, ln = orderbyList.size(); i < ln; i++) {
+            orderbyList.get(i).accept(this, step + 2);
+            out.println(" " + orderbyModifierList.get(i));
+        }
+        out.println(skip(step) + ")");
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
index e00a3bd..0c70572 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
@@ -50,6 +50,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
@@ -411,4 +412,20 @@ public class SqlppCloneAndSubstituteVariablesVisitor extends CloneAndSubstituteV
         newCaseExpr.setSourceLocation(caseExpr.getSourceLocation());
         return new Pair<>(newCaseExpr, env);
     }
+
+    @Override
+    public Pair<ILangExpression, VariableSubstitutionEnvironment> visit(WindowExpression winExpr,
+            VariableSubstitutionEnvironment env) throws CompilationException {
+        Expression newExpr = (Expression) winExpr.getExpr().accept(this, env).first;
+        List<Expression> newPartitionList = winExpr.hasPartitionList()
+                ? VariableCloneAndSubstitutionUtil.visitAndCloneExprList(winExpr.getPartitionList(), env, this) : null;
+        List<Expression> newOrderbyList =
+                VariableCloneAndSubstitutionUtil.visitAndCloneExprList(winExpr.getOrderbyList(), env, this);
+        List<OrderbyClause.OrderModifier> newOrderbyModifierList = new ArrayList<>(winExpr.getOrderbyModifierList());
+        WindowExpression newWinExpr =
+                new WindowExpression(newExpr, newPartitionList, newOrderbyList, newOrderbyModifierList);
+        newWinExpr.setSourceLocation(winExpr.getSourceLocation());
+        newWinExpr.addHints(winExpr.getHints());
+        return new Pair<>(newWinExpr, env);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
index 755cc69..99368f8 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
@@ -44,6 +44,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
@@ -330,4 +331,25 @@ public class SqlppFormatPrintVisitor extends FormatPrintVisitor implements ISqlp
         out.print(SqlppVariableUtil.toUserDefinedName(v.getVar().getValue()));
         return null;
     }
+
+    @Override
+    public Void visit(WindowExpression windowExpr, Integer step) throws CompilationException {
+        out.print(skip(step) + "window ");
+        windowExpr.getExpr().accept(this, step + 2);
+        out.print(skip(step) + " over (");
+        if (windowExpr.hasPartitionList()) {
+            List<Expression> partitionList = windowExpr.getPartitionList();
+            for (int i = 0, ln = partitionList.size(); i < ln; i++) {
+                if (i > 0) {
+                    out.print(COMMA);
+                }
+                Expression partExpr = partitionList.get(i);
+                partExpr.accept(this, step + 2);
+            }
+        }
+        out.print(" order by ");
+        printDelimitedObyExpressions(windowExpr.getOrderbyList(), windowExpr.getOrderbyModifierList(), step + 2);
+        out.println(skip(step) + ")");
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
index 5396768..92ebd60 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
@@ -34,6 +34,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 
 /**
  * A dummy abstract visitor to allow an implementation to only fill in necessary stuff.
@@ -110,4 +111,8 @@ public abstract class AbstractSqlppAstVisitor<R, T> extends AbstractAstVisitor<R
         return null;
     }
 
+    @Override
+    public R visit(WindowExpression winExpr, T arg) throws CompilationException {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
index 4d35914..a3bb592 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
@@ -61,6 +61,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 
 public class AbstractSqlppSimpleExpressionVisitor
@@ -321,6 +322,16 @@ public class AbstractSqlppSimpleExpressionVisitor
     }
 
     @Override
+    public Expression visit(WindowExpression winExpr, ILangExpression arg) throws CompilationException {
+        winExpr.setExpr(visit(winExpr.getExpr(), arg));
+        if (winExpr.hasPartitionList()) {
+            winExpr.setPartitionList(visit(winExpr.getPartitionList(), winExpr));
+        }
+        winExpr.setOrderbyList(visit(winExpr.getOrderbyList(), winExpr));
+        return winExpr;
+    }
+
+    @Override
     public Expression visit(FieldAccessor fa, ILangExpression arg) throws CompilationException {
         fa.setExpr(visit(fa.getExpr(), fa));
         return fa;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/ISqlppVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/ISqlppVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/ISqlppVisitor.java
index 02d9142..13addaf 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/ISqlppVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/ISqlppVisitor.java
@@ -34,6 +34,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 
 public interface ISqlppVisitor<R, T> extends ILangVisitor<R, T> {
 
@@ -64,4 +65,6 @@ public interface ISqlppVisitor<R, T> extends ILangVisitor<R, T> {
     R visit(HavingClause havingClause, T arg) throws CompilationException;
 
     R visit(CaseExpression caseExpression, T arg) throws CompilationException;
+
+    R visit(WindowExpression windowExpression, T arg) throws CompilationException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 13d1f8d..e19ee7a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -166,6 +166,7 @@ import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.common.clause.WhereClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.optype.JoinType;
 import org.apache.asterix.lang.sqlpp.optype.SetOpType;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
@@ -2588,7 +2589,7 @@ FieldBinding FieldBinding() throws ParseException:
 
 Expression FunctionCallExpr() throws ParseException:
 {
-  CallExpr callExpr;
+  Expression resultExpr;
   List<Expression> argList = new ArrayList<Expression>();
   Expression tmp = null;
   int arity = 0;
@@ -2596,6 +2597,10 @@ Expression FunctionCallExpr() throws ParseException:
   String hint = null;
   boolean star = false;
   boolean distinct = false;
+  Token overToken = null;
+  Expression partitionExpr = null;
+  List<Expression> partitionExprs = new ArrayList<Expression>();
+  OrderbyClause orderByClause = null;
 }
 {
   funcName = FunctionName()
@@ -2634,7 +2639,7 @@ Expression FunctionCallExpr() throws ParseException:
       if (signature == null) {
         signature = new FunctionSignature(funcName.dataverse, fqFunctionName, arity);
       }
-      callExpr = FunctionMapUtil.normalizedListInputFunctions(new CallExpr(signature,argList));
+      CallExpr callExpr = FunctionMapUtil.normalizedListInputFunctions(new CallExpr(signature,argList));
       if (hint != null) {
         if (hint.startsWith(INDEXED_NESTED_LOOP_JOIN_HINT)) {
           callExpr.addHint(IndexedNLJoinExpressionAnnotation.INSTANCE);
@@ -2643,8 +2648,29 @@ Expression FunctionCallExpr() throws ParseException:
         }
       }
       callExpr.setSourceLocation(funcName.sourceLoc);
-      return callExpr;
+      resultExpr = callExpr;
     }
+
+  (
+    <OVER> { overToken = token; }
+    <LEFTPAREN>
+    (
+      <PARTITION> <BY>
+      partitionExpr = Expression() { partitionExprs.add(partitionExpr); }
+      ( <COMMA> partitionExpr = Expression() { partitionExprs.add(partitionExpr); } )*
+    )?
+    orderByClause = OrderbyClause()
+    <RIGHTPAREN>
+    {
+      WindowExpression winExp = new WindowExpression(callExpr, partitionExprs, orderByClause.getOrderbyList(),
+        orderByClause.getModifierList());
+      resultExpr = addSourceLocation(winExp, overToken);
+    }
+  )?
+
+  {
+     return resultExpr;
+  }
 }
 
 Expression ParenthesizedExpression() throws ParseException:
@@ -3393,7 +3419,9 @@ TOKEN [IGNORE_CASE]:
   | <ORDER : "order">
   | <OUTER : "outer">
   | <OUTPUT : "output">
+  | <OVER: "over">
   | <PATH : "path">
+  | <PARTITION : "partition">
   | <POLICY : "policy">
   | <PRESORTED : "pre-sorted">
   | <PRIMARY : "primary">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
----------------------------------------------------------------------
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 3fb03a1..ccb124a 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
@@ -132,10 +132,13 @@ import org.apache.asterix.om.typecomputer.impl.UnorderedListConstructorTypeCompu
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.properties.UnpartitionedPropertyComputer;
 
 public class BuiltinFunctions {
 
@@ -162,6 +165,9 @@ public class BuiltinFunctions {
     private static final Map<IFunctionInfo, IFunctionInfo> scalarToAggregateFunctionMap = new HashMap<>();
     private static final Map<IFunctionInfo, IFunctionInfo> distinctToRegularScalarAggregateFunctionMap =
             new HashMap<>();
+    private static final Map<IFunctionInfo, IFunctionInfo> builtinWindowFunctions = new HashMap<>();
+    private static final Set<IFunctionInfo> builtinWindowFunctionsWithOrderArgs = new HashSet<>();
+    private static final Set<IFunctionInfo> builtinWindowFunctionsWithMaterialization = new HashSet<>();
 
     private static final Map<IFunctionInfo, SpatialFilterKind> spatialFilterFunctions = new HashMap<>();
 
@@ -809,6 +815,26 @@ public class BuiltinFunctions {
     public static final FunctionIdentifier SCALAR_SQL_VAR_POP_DISTINCT =
             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 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);
+
     // unnesting functions
     public static final FunctionIdentifier SCAN_COLLECTION =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "scan-collection", 1);
@@ -1727,6 +1753,19 @@ public class BuiltinFunctions {
         addFunction(SQL_VAR_POP_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_VAR_POP_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
 
+        // Window functions
+
+        addFunction(ROW_NUMBER, AInt64TypeComputer.INSTANCE, true);
+        addPrivateFunction(ROW_NUMBER_IMPL, AInt64TypeComputer.INSTANCE, true);
+        addFunction(RANK, AInt64TypeComputer.INSTANCE, true);
+        addPrivateFunction(RANK_IMPL, AInt64TypeComputer.INSTANCE, true);
+        addFunction(DENSE_RANK, AInt64TypeComputer.INSTANCE, true);
+        addPrivateFunction(DENSE_RANK_IMPL, AInt64TypeComputer.INSTANCE, true);
+        addFunction(PERCENT_RANK, ADoubleTypeComputer.INSTANCE, true);
+        addPrivateFunction(PERCENT_RANK_IMPL, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(NTILE, AInt64TypeComputer.INSTANCE, true);
+        addPrivateFunction(NTILE_IMPL, AInt64TypeComputer.INSTANCE, true);
+
         // Similarity functions
         addFunction(EDIT_DISTANCE_CONTAINS, OrderedListOfAnyTypeComputer.INSTANCE, true);
         addFunction(SIMILARITY_JACCARD, AFloatTypeComputer.INSTANCE, true);
@@ -2487,6 +2526,15 @@ public class BuiltinFunctions {
     }
 
     static {
+        // Window functions
+        addWindowFunction(ROW_NUMBER, ROW_NUMBER_IMPL, false, false);
+        addWindowFunction(RANK, RANK_IMPL, true, false);
+        addWindowFunction(DENSE_RANK, DENSE_RANK_IMPL, true, false);
+        addWindowFunction(PERCENT_RANK, PERCENT_RANK_IMPL, true, true);
+        addWindowFunction(NTILE, NTILE_IMPL, false, true);
+    }
+
+    static {
         addUnnestFun(RANGE, true);
         addUnnestFun(SCAN_COLLECTION, false);
         addUnnestFun(SUBSET_COLLECTION, false);
@@ -2667,6 +2715,40 @@ public class BuiltinFunctions {
                 getAsterixFunctionInfo(regularscalarfi));
     }
 
+    public static void addWindowFunction(FunctionIdentifier fi, FunctionIdentifier implfi, boolean requiresOrderArgs,
+            boolean requiresMaterialization) {
+        IFunctionInfo implFinfo = getAsterixFunctionInfo(implfi);
+        builtinWindowFunctions.put(getAsterixFunctionInfo(fi), implFinfo);
+        if (requiresOrderArgs) {
+            builtinWindowFunctionsWithOrderArgs.add(implFinfo);
+        }
+        if (requiresMaterialization) {
+            builtinWindowFunctionsWithMaterialization.add(implFinfo);
+        }
+    }
+
+    public static boolean isBuiltinWindowFunction(FunctionIdentifier fi) {
+        return builtinWindowFunctions.containsKey(getAsterixFunctionInfo(fi));
+    }
+
+    public static boolean windowFunctionRequiresOrderArgs(FunctionIdentifier implfi) {
+        return builtinWindowFunctionsWithOrderArgs.contains(getAsterixFunctionInfo(implfi));
+    }
+
+    public static boolean windowFunctionRequiresMaterialization(FunctionIdentifier implfi) {
+        return builtinWindowFunctionsWithMaterialization.contains(getAsterixFunctionInfo(implfi));
+    }
+
+    public static AbstractFunctionCallExpression makeWindowFunctionExpression(FunctionIdentifier scalarfi,
+            List<Mutable<ILogicalExpression>> args) {
+        IFunctionInfo finfo = getAsterixFunctionInfo(scalarfi);
+        IFunctionInfo implFinfo = builtinWindowFunctions.get(finfo);
+        if (implFinfo == null) {
+            throw new IllegalStateException("no implementation for window function " + finfo);
+        }
+        return new StatefulFunctionCallExpression(implFinfo, UnpartitionedPropertyComputer.INSTANCE, args);
+    }
+
     static {
         spatialFilterFunctions.put(getAsterixFunctionInfo(BuiltinFunctions.SPATIAL_INTERSECT), SpatialFilterKind.SI);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
----------------------------------------------------------------------
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 19c33db..1b0c7b6 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
@@ -439,6 +439,11 @@ import org.apache.asterix.runtime.evaluators.functions.temporal.UnixTimeFromDate
 import org.apache.asterix.runtime.evaluators.functions.temporal.UnixTimeFromTimeInMsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.YearMonthDurationGreaterThanComparatorDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.YearMonthDurationLessThanComparatorDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.DenseRankRunningAggregateDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.NtileRunningAggregateDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.PercentRankRunningAggregateDescriptor;
+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.unnestingfunctions.std.RangeDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
@@ -632,6 +637,13 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.add(ScalarSqlVarAggregateDescriptor.FACTORY);
         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);
+
         // boolean functions
         fc.add(AndDescriptor.FACTORY);
         fc.add(OrDescriptor.FACTORY);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/AbstractRankRunningAggregateEvaluator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/AbstractRankRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/AbstractRankRunningAggregateEvaluator.java
new file mode 100644
index 0000000..54bcc1a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/AbstractRankRunningAggregateEvaluator.java
@@ -0,0 +1,130 @@
+/*
+ * 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 java.io.DataOutput;
+
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IWindowAggregateEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * Base evaluator implementation for ranking window functions:
+ * {@code rank()}, {@code dense_rank()}, {@code percent_rank()}
+ */
+public abstract class AbstractRankRunningAggregateEvaluator implements IWindowAggregateEvaluator {
+
+    private final IScalarEvaluator[] args;
+
+    private final ArrayBackedValueStorage[] argPrevValues;
+
+    private final IPointable[] argCurrValues;
+
+    private final boolean dense;
+
+    protected final SourceLocation sourceLoc;
+
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+
+    private IBinaryComparator[] argComparators;
+
+    private boolean first;
+
+    private long rank;
+
+    private long groupSize;
+
+    AbstractRankRunningAggregateEvaluator(IScalarEvaluator[] args, boolean dense, SourceLocation sourceLoc) {
+        this.args = args;
+        this.dense = dense;
+        this.sourceLoc = sourceLoc;
+        argPrevValues = new ArrayBackedValueStorage[args.length];
+        argCurrValues = new IPointable[args.length];
+        for (int i = 0; i < args.length; i++) {
+            argPrevValues[i] = new ArrayBackedValueStorage();
+            argCurrValues[i] = VoidPointable.FACTORY.createPointable();
+        }
+    }
+
+    @Override
+    public void configure(IBinaryComparator[] orderComparators) {
+        argComparators = orderComparators;
+    }
+
+    @Override
+    public void init() throws HyracksDataException {
+    }
+
+    @Override
+    public void initPartition(long partitionLength) {
+        first = true;
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        for (int i = 0; i < args.length; i++) {
+            args[i].evaluate(tuple, argCurrValues[i]);
+        }
+
+        computeRank();
+
+        for (int i = 0; i < args.length; i++) {
+            argPrevValues[i].assign(argCurrValues[i]);
+        }
+
+        resultStorage.reset();
+        writeResult(rank, resultStorage.getDataOutput());
+        result.set(resultStorage);
+    }
+
+    protected abstract void writeResult(long rank, DataOutput out) throws HyracksDataException;
+
+    private void computeRank() throws HyracksDataException {
+        if (first) {
+            rank = 1;
+            groupSize = 1;
+            first = false;
+        } else if (sameGroup()) {
+            groupSize++;
+        } else {
+            rank += dense ? 1 : groupSize;
+            groupSize = 1;
+        }
+    }
+
+    private boolean sameGroup() throws HyracksDataException {
+        for (int i = 0; i < args.length; i++) {
+            IPointable v1 = argPrevValues[i];
+            IPointable v2 = argCurrValues[i];
+            IBinaryComparator cmp = argComparators[i];
+            if (cmp.compare(v1.getByteArray(), v1.getStartOffset(), v1.getLength(), v2.getByteArray(),
+                    v2.getStartOffset(), v2.getLength()) != 0) {
+                return false;
+            }
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..6e51559
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/DenseRankRunningAggregateDescriptor.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.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.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Descriptor {@code dense_rank()} window function
+ */
+public class DenseRankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = DenseRankRunningAggregateDescriptor::new;
+
+    @Override
+    public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IRunningAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                IScalarEvaluator[] evals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    evals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RankRunningAggregateEvaluator(evals, true, sourceLoc);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.DENSE_RANK_IMPL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..5157451
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateDescriptor.java
@@ -0,0 +1,57 @@
+/*
+ * 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;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Descriptor {@code ntile()} window function
+ */
+public class NtileRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = NtileRunningAggregateDescriptor::new;
+
+    @Override
+    public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IRunningAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new NtileRunningAggregateEvaluator(args[0].createScalarEvaluator(ctx), getIdentifier());
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NTILE_IMPL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..9bd306e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/NtileRunningAggregateEvaluator.java
@@ -0,0 +1,115 @@
+/*
+ * 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.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+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.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * Evaluator {@code ntile()} window function
+ */
+public class NtileRunningAggregateEvaluator implements IWindowAggregateEvaluator {
+
+    private final IScalarEvaluator evalNumGroups;
+
+    private final VoidPointable argNumGroups = VoidPointable.FACTORY.createPointable();
+
+    private final FunctionIdentifier funId;
+
+    @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);
+
+    private long partitionLength;
+
+    private long groupSize;
+
+    private long groupRemainder;
+
+    private long resultValue;
+
+    private long count;
+
+    NtileRunningAggregateEvaluator(IScalarEvaluator evalNumGroups, FunctionIdentifier funId) {
+        this.evalNumGroups = evalNumGroups;
+        this.funId = funId;
+    }
+
+    @Override
+    public void init() throws HyracksDataException {
+    }
+
+    @Override
+    public void initPartition(long partitionLength) {
+        this.partitionLength = partitionLength;
+        resultValue = 0;
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        if (resultValue == 0) {
+            evaluateGroupSize(tuple);
+            resultValue = count = 1;
+        } else if (count < groupSize) {
+            count++;
+        } else if (count == groupSize && groupRemainder > 0) {
+            groupRemainder--;
+            count++;
+        } else {
+            resultValue++;
+            count = 1;
+        }
+
+        resultStorage.reset();
+        aInt64.setValue(resultValue);
+        serde.serialize(aInt64, resultStorage.getDataOutput());
+        result.set(resultStorage);
+    }
+
+    private void evaluateGroupSize(IFrameTupleReference tuple) throws HyracksDataException {
+        evalNumGroups.evaluate(tuple, argNumGroups);
+        byte[] bytes = argNumGroups.getByteArray();
+        int offset = argNumGroups.getStartOffset();
+        long numGroups = ATypeHierarchy.getLongValue(funId.getName(), 0, bytes, offset);
+        if (numGroups > partitionLength || numGroups <= 0) {
+            groupSize = partitionLength;
+            groupRemainder = 0;
+        } else {
+            groupSize = partitionLength / numGroups;
+            groupRemainder = partitionLength % numGroups;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..180ca99
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateDescriptor.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.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.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Descriptor {@code percent_rank()} window function
+ */
+public class PercentRankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = PercentRankRunningAggregateDescriptor::new;
+
+    @Override
+    public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IRunningAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                IScalarEvaluator[] evals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    evals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new PercentRankRunningAggregateEvaluator(evals, sourceLoc);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.PERCENT_RANK_IMPL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..c73d9fd
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/PercentRankRunningAggregateEvaluator.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 java.io.DataOutput;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * Evaluator {@code percent_rank()} window function
+ */
+class PercentRankRunningAggregateEvaluator extends AbstractRankRunningAggregateEvaluator {
+
+    private final AMutableDouble aDouble = new AMutableDouble(0);
+
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<ADouble> serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+    private double divisor;
+
+    PercentRankRunningAggregateEvaluator(IScalarEvaluator[] args, SourceLocation sourceLoc) {
+        super(args, false, sourceLoc);
+    }
+
+    @Override
+    public void initPartition(long partitionLength) {
+        super.initPartition(partitionLength);
+        divisor = (double) partitionLength - 1;
+    }
+
+    @Override
+    protected void writeResult(long rank, DataOutput out) throws HyracksDataException {
+        double percentRank = (rank - 1) / divisor;
+        aDouble.setValue(percentRank);
+        serde.serialize(aDouble, out);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..d5db134
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateDescriptor.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.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.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Descriptor {@code rank()} window function
+ */
+public class RankRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = RankRunningAggregateDescriptor::new;
+
+    @Override
+    public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IRunningAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                IScalarEvaluator[] evals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    evals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RankRunningAggregateEvaluator(evals, false, sourceLoc);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RANK_IMPL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateEvaluator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateEvaluator.java
new file mode 100644
index 0000000..56ab299
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RankRunningAggregateEvaluator.java
@@ -0,0 +1,52 @@
+/*
+ * 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 java.io.DataOutput;
+
+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.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * Evaluator for {@code rank()} and {@code dense_rank()} window functions
+ */
+class RankRunningAggregateEvaluator extends AbstractRankRunningAggregateEvaluator {
+
+    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+    @SuppressWarnings("unchecked")
+    private final ISerializerDeserializer<AInt64> serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+
+    RankRunningAggregateEvaluator(IScalarEvaluator[] args, boolean dense, SourceLocation sourceLoc) {
+        super(args, dense, sourceLoc);
+    }
+
+    protected void writeResult(long rank, DataOutput out) throws HyracksDataException {
+        aInt64.setValue(rank);
+        serde.serialize(aInt64, out);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7464751
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateDescriptor.java
@@ -0,0 +1,56 @@
+/*
+ * 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 {@code row_number()} window function
+ */
+public class RowNumberRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = RowNumberRunningAggregateDescriptor::new;
+
+    @Override
+    public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IRunningAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IHyracksTaskContext ctx) {
+                return new RowNumberRunningAggregateEvaluator();
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ROW_NUMBER_IMPL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateEvaluator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateEvaluator.java
new file mode 100644
index 0000000..75fface
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/RowNumberRunningAggregateEvaluator.java
@@ -0,0 +1,67 @@
+/*
+ * 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 {@code row_number()} window function
+ */
+class RowNumberRunningAggregateEvaluator 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);
+
+    private long resultValue;
+
+    @Override
+    public void init() {
+        // nothing to do
+    }
+
+    @Override
+    public void initPartition(long partitionLength) {
+        resultValue = 0;
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultValue++;
+
+        resultStorage.reset();
+        aInt64.setValue(resultValue);
+        serde.serialize(aInt64, resultStorage.getDataOutput());
+        result.set(resultStorage);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Pair.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Pair.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Pair.java
index 2dd71cd..a0d5880 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Pair.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/utils/Pair.java
@@ -33,6 +33,22 @@ public class Pair<T1, T2> implements Serializable {
         this.second = second;
     }
 
+    public void setFirst(T1 value) {
+        first = value;
+    }
+
+    public T1 getFirst() {
+        return first;
+    }
+
+    public void setSecond(T2 value) {
+        second = value;
+    }
+
+    public T2 getSecond() {
+        return second;
+    }
+
     @Override
     public String toString() {
         return first + "," + second;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
index 6bd0d02..d996caf 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
@@ -89,7 +89,7 @@ public interface ILogicalOperator {
      */
 
     public PhysicalRequirements getRequiredPhysicalPropertiesForChildren(IPhysicalPropertiesVector requiredProperties,
-            IOptimizationContext context);
+            IOptimizationContext context) throws AlgebricksException;
 
     /**
      * @return the physical properties that this operator delivers, based on

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IPhysicalOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IPhysicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IPhysicalOperator.java
index 2a92aba..a88ec64 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IPhysicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IPhysicalOperator.java
@@ -40,7 +40,7 @@ public interface IPhysicalOperator {
      * @return for each child, one vector of required physical properties
      */
     public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
-            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context);
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) throws AlgebricksException;
 
     /**
      * @return the physical properties that this operator delivers, based on

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index 3794328..4466408 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -56,4 +56,5 @@ public enum LogicalOperatorTag {
     WRITE,
     WRITE_RESULT,
     INTERSECT,
+    WINDOW
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index ac1de5a..5d19134 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -79,4 +79,5 @@ public enum PhysicalOperatorTag {
     UPDATE,
     WRITE_RESULT,
     INTERSECT,
+    WINDOW
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
index 51040fd..e15ed92 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractAssignOperator.java
@@ -26,6 +26,7 @@ import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 
 /**
@@ -71,4 +72,17 @@ public abstract class AbstractAssignOperator extends AbstractLogicalOperator {
         return modif;
     }
 
+    protected VariablePropagationPolicy createVariablePropagationPolicy(boolean propagateInputVars) {
+        return new VariablePropagationPolicy() {
+            @Override
+            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources) {
+                if (propagateInputVars) {
+                    target.addAllVariables(sources[0]);
+                }
+                for (LogicalVariable v : variables) {
+                    target.addVariable(v);
+                }
+            }
+        };
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
index 1dbf15e..d4a9d37 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
@@ -115,7 +115,7 @@ public abstract class AbstractLogicalOperator implements ILogicalOperator {
 
     @Override
     public final PhysicalRequirements getRequiredPhysicalPropertiesForChildren(
-            IPhysicalPropertiesVector requiredProperties, IOptimizationContext context) {
+            IPhysicalPropertiesVector requiredProperties, IOptimizationContext context) throws AlgebricksException {
         return physicalOperator.getRequiredPropertiesForChildren(this, requiredProperties, context);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
index b4a59a8..35cb087 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
@@ -57,16 +57,7 @@ public class AggregateOperator extends AbstractAssignOperator {
 
     @Override
     public VariablePropagationPolicy getVariablePropagationPolicy() {
-        return new VariablePropagationPolicy() {
-
-            @Override
-            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
-                    throws AlgebricksException {
-                for (LogicalVariable v : variables) {
-                    target.addVariable(v);
-                }
-            }
-        };
+        return createVariablePropagationPolicy(false);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
index 861d74c..202c291 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AssignOperator.java
@@ -65,18 +65,7 @@ public class AssignOperator extends AbstractAssignOperator {
 
     @Override
     public VariablePropagationPolicy getVariablePropagationPolicy() {
-        return new VariablePropagationPolicy() {
-
-            @Override
-            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
-                    throws AlgebricksException {
-                target.addAllVariables(sources[0]);
-                for (LogicalVariable v : variables) {
-                    target.addVariable(v);
-                }
-            }
-        };
-
+        return createVariablePropagationPolicy(true);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/RunningAggregateOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/RunningAggregateOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/RunningAggregateOperator.java
index ef16613..8a710cc 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/RunningAggregateOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/RunningAggregateOperator.java
@@ -49,17 +49,7 @@ public class RunningAggregateOperator extends AbstractAssignOperator {
 
     @Override
     public VariablePropagationPolicy getVariablePropagationPolicy() {
-        return new VariablePropagationPolicy() {
-
-            @Override
-            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
-                    throws AlgebricksException {
-                target.addAllVariables(sources[0]);
-                for (LogicalVariable v : variables) {
-                    target.addVariable(v);
-                }
-            }
-        };
+        return createVariablePropagationPolicy(true);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/WindowOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/WindowOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/WindowOperator.java
new file mode 100644
index 0000000..aa1791f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/WindowOperator.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+/**
+ * Window operator evaluates window functions. It has the following components:
+ * <ul>
+ * <li>{@link #partitionExpressions} - define how input data must be partitioned</li>
+ * <li>{@link #orderExpressions} - define how data inside these partitions must be ordered</li>
+ * <li>{@link #expressions} - window function expressions (running aggregates)</li>
+ * <li>{@link #variables} - output variables containing return values of these functions</li>
+ * </ul>
+ *
+ * Window operator does not change cardinality of the input stream.
+ */
+public class WindowOperator extends AbstractAssignOperator {
+
+    private final List<Mutable<ILogicalExpression>> partitionExpressions;
+
+    private final List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExpressions;
+
+    public WindowOperator(List<Mutable<ILogicalExpression>> partitionExpressions,
+            List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExpressions,
+            List<LogicalVariable> variables, List<Mutable<ILogicalExpression>> expressions) {
+        super(variables, expressions);
+        this.partitionExpressions = new ArrayList<>();
+        if (partitionExpressions != null) {
+            this.partitionExpressions.addAll(partitionExpressions);
+        }
+        this.orderExpressions = new ArrayList<>();
+        if (orderExpressions != null) {
+            this.orderExpressions.addAll(orderExpressions);
+        }
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.WINDOW;
+    }
+
+    public List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> getOrderExpressions() {
+        return orderExpressions;
+    }
+
+    public List<Mutable<ILogicalExpression>> getPartitionExpressions() {
+        return partitionExpressions;
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitWindowOperator(this, arg);
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        boolean mod = super.acceptExpressionTransform(visitor);
+        for (Mutable<ILogicalExpression> expr : partitionExpressions) {
+            mod |= visitor.transform(expr);
+        }
+        for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExpressions) {
+            mod |= visitor.transform(p.second);
+        }
+        return mod;
+    }
+
+    @Override
+    public VariablePropagationPolicy getVariablePropagationPolicy() {
+        return createVariablePropagationPolicy(true);
+    }
+
+    @Override
+    public boolean isMap() {
+        return false;
+    }
+
+    @Override
+    public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+        IVariableTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
+        int n = variables.size();
+        for (int i = 0; i < n; i++) {
+            env.setVarType(variables.get(i), ctx.getExpressionTypeComputer().getType(expressions.get(i).getValue(),
+                    ctx.getMetadataProvider(), env));
+        }
+        return env;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
index 9d853eb..8535204 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
@@ -61,6 +61,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
@@ -160,6 +161,11 @@ public class CardinalityInferenceVisitor implements ILogicalOperatorVisitor<Long
     }
 
     @Override
+    public Long visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        return op.getInputs().get(0).getValue().accept(this, arg);
+    }
+
+    @Override
     public Long visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
         return adjustCardinalityForTupleReductionOperator(op.getInputs().get(0).getValue().accept(this, arg));
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index 16fc1ed..7042794 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -78,6 +78,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
@@ -812,4 +813,9 @@ public class FDsAndEquivClassesVisitor implements ILogicalOperatorVisitor<Void,
         return null;
     }
 
+    @Override
+    public Void visitWindowOperator(WindowOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
 }


[4/5] asterixdb git commit: [ASTERIXDB-2466][FUN] Implement window functions

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.6.query.sqlpp
new file mode 100644
index 0000000..8bf71a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.6.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 NTILE()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+from q0_rnd() rnd
+select q1_ntile(rnd.p, rnd.n, 5) res, rnd.p, rnd.n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.7.query.sqlpp
new file mode 100644
index 0000000..e47db20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 NTILE() without partition clause
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q2_ntile_no_partition(7, 11)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_02/ntile_02.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_02/ntile_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_02/ntile_02.1.query.sqlpp
new file mode 100644
index 0000000..3a46f18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_02/ntile_02.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 NTILE() with a non-integer argument
+ * Expected Res : FAILURE
+ */
+
+from range(1,10) t
+select ntile("string") over (order by t desc)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.1.ddl.sqlpp
new file mode 100644
index 0000000..841d9b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.1.ddl.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * 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 PERCENT_RANK()
+ * Expected Res : SUCCESS
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.t1Type as open {
+  c1 : bigint
+};
+
+create dataset t1(t1Type) primary key c1;
+
+create dataset tRnd(t1Type) primary key c1;
+
+create function q0_rnd() {
+  let
+    rnd = tobigint((select value tRnd.rnd from tRnd where c1 = 1)[0] * 1000),
+    p = case when rnd >= 10 then rnd else 10 end,
+    n = tobigint( 1000 * 100 / p)
+  select p, n
+};
+
+create function q1_percent_rank(P, N, D) {
+  from
+    t1,
+    range(t1.one, P) p,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    rank = (N - n) * D + 1,
+    result_expected = (rank - 1) / (N * D - 1),
+    result_actual = percent_rank() over ( partition by t1.c2, p order by n desc ),
+    result_delta_raw = result_expected - result_actual,
+    result_delta = case when result_delta_raw < 0.001 then 0 else result_delta_raw end
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};
+
+create function q2_percent_rank_no_partition(N, D) {
+  from
+    t1,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    rank = ((t1.c2 - 1) * N * D) + (N - n) * D + 1,
+    result_expected = (rank - 1) / (4 * N * D - 1),
+    result_actual = percent_rank() over ( order by t1.c2, n desc ),
+    result_delta_raw = result_expected - result_actual,
+    result_delta = case when result_delta_raw < 0.001 then 0 else result_delta_raw end
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.2.update.sqlpp
new file mode 100644
index 0000000..5c6d500
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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 PERCENT_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+set `import-private-functions` `true`;
+
+insert into t1
+select element { "c1":1, "c2": 1, "one": 1 };
+
+insert into t1
+select v c1, v c2, 1 one
+from range(2, (select value count(*) from storage_components("test","t1") t)[0]) v;
+
+insert into tRnd
+select 1 c1, random() rnd;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.3.query.sqlpp
new file mode 100644
index 0000000..e091953
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 PERCENT_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_percent_rank(1, 2, 4)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.4.query.sqlpp
new file mode 100644
index 0000000..62f68dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 PERCENT_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_percent_rank(1000, 10, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.5.query.sqlpp
new file mode 100644
index 0000000..3120eb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 PERCENT_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_percent_rank(10, 1000, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.6.query.sqlpp
new file mode 100644
index 0000000..8f38718
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.6.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 PERCENT_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+from q0_rnd() rnd
+select q1_percent_rank(rnd.p, rnd.n, 3) res, rnd.p, rnd.n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.7.query.sqlpp
new file mode 100644
index 0000000..a3b9352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/percent_rank_01/percent_rank_01.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 PERCENT_RANK() without partition clause
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q2_percent_rank_no_partition(7, 11)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.1.ddl.sqlpp
new file mode 100644
index 0000000..020fe8d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.1.ddl.sqlpp
@@ -0,0 +1,74 @@
+/*
+ * 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 RANK()
+ * Expected Res : SUCCESS
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.t1Type as open {
+  c1 : bigint
+};
+
+create dataset t1(t1Type) primary key c1;
+
+create dataset tRnd(t1Type) primary key c1;
+
+create function q0_rnd() {
+  let
+    rnd = tobigint((select value tRnd.rnd from tRnd where c1 = 1)[0] * 1000),
+    p = case when rnd >= 10 then rnd else 10 end,
+    n = tobigint( 1000 * 100 / p)
+  select p, n
+};
+
+create function q1_rank(P, N, D) {
+  from
+    t1,
+    range(t1.one, P) p,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    result_expected = (N - n) * D + 1,
+    result_actual = rank() over ( partition by t1.c2, p order by n desc ),
+    result_delta = result_expected - result_actual
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};
+
+create function q2_rank_no_partition(N, D) {
+  from
+    t1,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    result_expected = ((t1.c2 - 1) * N * D) + (N - n) * D + 1,
+    result_actual = rank() over ( order by t1.c2, n desc ),
+    result_delta = result_expected - result_actual
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};
+
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.2.update.sqlpp
new file mode 100644
index 0000000..8c4147f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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 RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+set `import-private-functions` `true`;
+
+insert into t1
+select element { "c1":1, "c2": 1, "one": 1 };
+
+insert into t1
+select v c1, v c2, 1 one
+from range(2, (select value count(*) from storage_components("test","t1") t)[0]) v;
+
+insert into tRnd
+select 1 c1, random() rnd;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.3.query.sqlpp
new file mode 100644
index 0000000..914d7c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_rank(1, 1, 3)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.4.query.sqlpp
new file mode 100644
index 0000000..2c34967
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_rank(1000, 10, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.5.query.sqlpp
new file mode 100644
index 0000000..b489f29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_rank(10, 1000, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.6.query.sqlpp
new file mode 100644
index 0000000..eb214f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.6.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 RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+from q0_rnd() rnd
+select q1_rank(rnd.p, rnd.n, 3) res, rnd.p, rnd.n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.7.query.sqlpp
new file mode 100644
index 0000000..83687ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/rank_01/rank_01.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 RANK() without partition clause
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q2_rank_no_partition(7, 11)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.1.ddl.sqlpp
new file mode 100644
index 0000000..5943177
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.1.ddl.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * 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 ROW_NUMBER()
+ * Expected Res : SUCCESS
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.t1Type as open {
+  c1 : bigint
+};
+
+create dataset t1(t1Type) primary key c1;
+
+create dataset tRnd(t1Type) primary key c1;
+
+create function q0_rnd() {
+  let
+    rnd = tobigint((select value tRnd.rnd from tRnd where c1 = 1)[0] * 1000),
+    p = case when rnd >= 10 then rnd else 10 end,
+    n = tobigint( 1000 * 100 / p)
+  select p, n
+};
+
+create function q1_row_number(P, N, D) {
+  from
+    t1,
+    range(t1.one, P) p,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    result_expected = (N - n) * D + d,
+    result_actual = row_number() over ( partition by t1.c2, p order by n desc, d ),
+    result_delta = result_expected - result_actual
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};
+
+create function q2_row_number_no_partition(N, D) {
+  from
+    t1,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    result_expected = ((t1.c2 - 1) * N * D) + ((N - n) * D + d),
+    result_actual = row_number() over ( order by t1.c2, n desc, d ),
+    result_delta = result_expected - result_actual
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.2.update.sqlpp
new file mode 100644
index 0000000..3691f05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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 ROW_NUMBER()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+set `import-private-functions` `true`;
+
+insert into t1
+select element { "c1":1, "c2": 1, "one": 1 };
+
+insert into t1
+select v c1, v c2, 1 one
+from range(2, (select value count(*) from storage_components("test","t1") t)[0]) v;
+
+insert into tRnd
+select 1 c1, random() rnd;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.3.query.sqlpp
new file mode 100644
index 0000000..659d3e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 ROW_NUMBER()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_row_number(1, 1, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.4.query.sqlpp
new file mode 100644
index 0000000..e8122d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 ROW_NUMBER()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_row_number(1000, 10, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.5.query.sqlpp
new file mode 100644
index 0000000..381871b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 ROW_NUMBER()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_row_number(10, 1000, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.6.query.sqlpp
new file mode 100644
index 0000000..672511c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.6.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 ROW_NUMBER()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+from q0_rnd() rnd
+select q1_row_number(rnd.p, rnd.n, 3) res, rnd.p, rnd.n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.7.query.sqlpp
new file mode 100644
index 0000000..80c54f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/row_number_01/row_number_01.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 ROW_NUMBER() without partition clause
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q2_row_number_no_partition(7, 11)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.3.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.3.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.4.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.4.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.5.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.5.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.6.regexadm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.6.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.6.regexadm
new file mode 100644
index 0000000..7278103
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.6.regexadm
@@ -0,0 +1 @@
+.*"min_delta": 0, "max_delta": 0.*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.7.adm
new file mode 100644
index 0000000..6115ead
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/dense_rank_01/dense_rank_01.7.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.3.adm
new file mode 100644
index 0000000..42b9daf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.3.adm
@@ -0,0 +1 @@
+{ "rank_min_delta": 0, "rank_max_delta": 0, "percent_rank_min_delta": 0, "percent_rank_max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.4.adm
new file mode 100644
index 0000000..711a89b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.4.adm
@@ -0,0 +1,4 @@
+{ "p": 1, "rn": 4 }
+{ "p": 2, "rn": 3 }
+{ "p": 3, "rn": 2 }
+{ "p": 4, "rn": 1 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.5.adm
new file mode 100644
index 0000000..4e533c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.5.adm
@@ -0,0 +1,4 @@
+{ "p": 1, "gg": [ { "c1": 1, "rn": 4 }, { "c1": 2, "rn": 3 }, { "c1": 3, "rn": 2 }, { "c1": 4, "rn": 1 } ] }
+{ "p": 2, "gg": [ { "c1": 1, "rn": 4 }, { "c1": 2, "rn": 3 }, { "c1": 3, "rn": 2 }, { "c1": 4, "rn": 1 } ] }
+{ "p": 3, "gg": [ { "c1": 1, "rn": 4 }, { "c1": 2, "rn": 3 }, { "c1": 3, "rn": 2 }, { "c1": 4, "rn": 1 } ] }
+{ "p": 4, "gg": [ { "c1": 1, "rn": 4 }, { "c1": 2, "rn": 3 }, { "c1": 3, "rn": 2 }, { "c1": 4, "rn": 1 } ] }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.3.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.3.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.4.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.4.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.5.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.5.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.6.regexadm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.6.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.6.regexadm
new file mode 100644
index 0000000..7278103
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.6.regexadm
@@ -0,0 +1 @@
+.*"min_delta": 0, "max_delta": 0.*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.7.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/ntile_01/ntile_01.7.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.3.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.3.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.4.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.4.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.5.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.5.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.6.regexadm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.6.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.6.regexadm
new file mode 100644
index 0000000..7278103
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.6.regexadm
@@ -0,0 +1 @@
+.*"min_delta": 0, "max_delta": 0.*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.7.adm
new file mode 100644
index 0000000..6115ead
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/percent_rank_01/percent_rank_01.7.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.3.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.3.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.4.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.4.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.5.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.5.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.6.regexadm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.6.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.6.regexadm
new file mode 100644
index 0000000..7278103
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.6.regexadm
@@ -0,0 +1 @@
+.*"min_delta": 0, "max_delta": 0.*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.7.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/rank_01/rank_01.7.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.3.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.3.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.4.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.4.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.5.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.5.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.6.regexadm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.6.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.6.regexadm
new file mode 100644
index 0000000..7278103
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.6.regexadm
@@ -0,0 +1 @@
+.*"min_delta": 0, "max_delta": 0.*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.7.adm
new file mode 100644
index 0000000..6a071d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/row_number_01/row_number_01.7.adm
@@ -0,0 +1 @@
+{ "min_delta": 0, "max_delta": 0 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
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 1a9c7dd..591d441 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -8933,6 +8933,45 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="window">
+    <test-case FilePath="window">
+      <compilation-unit name="dense_rank_01">
+        <output-dir compare="Text">dense_rank_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="misc_01">
+        <output-dir compare="Text">misc_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="ntile_01">
+        <output-dir compare="Text">ntile_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="ntile_02">
+        <output-dir compare="Text">ntile_01</output-dir>
+        <expected-error>ASX0002: Type mismatch</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="percent_rank_01">
+        <output-dir compare="Text">percent_rank_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="rank_01">
+        <output-dir compare="Text">rank_01</output-dir>
+      </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>
+    </test-case>
+  </test-group>
   <test-group name="writers">
     <test-case FilePath="writers">
       <compilation-unit name="print_01">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
----------------------------------------------------------------------
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 1bf2447..0bf446e 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
@@ -180,6 +180,7 @@ public class ErrorCode {
     public static final int ILLEGAL_SET_PARAMETER = 1092;
     public static final int COMPILATION_TRANSLATION_ERROR = 1093;
     public static final int RANGE_MAP_ERROR = 1094;
+    public static final int COMPILATION_EXPECTED_FUNCTION_CALL = 1095;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
----------------------------------------------------------------------
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 5629b97..8c17ec6 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -167,6 +167,7 @@
 1092 = Parameter %1$s cannot be set
 1093 = A parser error has occurred. The detail exception: %1$s
 1094 = Cannot parse range map: %1$s
+1095 = Expected function call
 
 # Feed Errors
 3001 = Illegal state.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
index 3afff76..1066408 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
@@ -38,6 +38,7 @@ public interface Expression extends ILangExpression {
         UNION_EXPRESSION,
         SELECT_EXPRESSION,
         PRIMARY_EXPRESSION,
-        CASE_EXPRESSION
+        CASE_EXPRESSION,
+        WINDOW_EXPRESSION
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
index e415a63..0b907bc 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
@@ -242,7 +242,7 @@ public class CloneAndSubstituteVariablesVisitor extends
             VariableSubstitutionEnvironment env) throws CompilationException {
         List<Expression> exprList =
                 VariableCloneAndSubstitutionUtil.visitAndCloneExprList(oc.getOrderbyList(), env, this);
-        OrderbyClause oc2 = new OrderbyClause(exprList, oc.getModifierList());
+        OrderbyClause oc2 = new OrderbyClause(exprList, new ArrayList<>(oc.getModifierList()));
         oc2.setNumFrames(oc.getNumFrames());
         oc2.setNumTuples(oc.getNumTuples());
         oc2.setRangeMap(oc.getRangeMap());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 7ba01ce..c7f2a5d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -295,20 +295,7 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
     @Override
     public Void visit(OrderbyClause oc, Integer step) throws CompilationException {
         out.print(skip(step) + "order by ");
-        List<OrderModifier> mlist = oc.getModifierList();
-        List<Expression> list = oc.getOrderbyList();
-        int index = 0;
-        int size = list.size();
-        for (Expression expr : oc.getOrderbyList()) {
-            expr.accept(this, step);
-            OrderModifier orderModifier = mlist.get(index);
-            if (orderModifier != OrderModifier.ASC) {
-                out.print(" " + orderModifier.toString().toLowerCase());
-            }
-            if (++index < size) {
-                out.print(COMMA);
-            }
-        }
+        printDelimitedObyExpressions(oc.getOrderbyList(), oc.getModifierList(), step);
         out.println();
         return null;
     }
@@ -897,6 +884,22 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
         }
     }
 
+    protected void printDelimitedObyExpressions(List<Expression> list, List<OrderModifier> mlist, Integer step)
+            throws CompilationException {
+        int index = 0;
+        int size = list.size();
+        for (Expression expr : list) {
+            expr.accept(this, step);
+            OrderModifier orderModifier = mlist.get(index);
+            if (orderModifier != OrderModifier.ASC) {
+                out.print(orderModifier.toString().toLowerCase());
+            }
+            if (++index < size) {
+                out.print(COMMA);
+            }
+        }
+    }
+
     protected void printDelimitedExpressions(List<? extends Expression> exprs, String delimiter, int step)
             throws CompilationException {
         int index = 0;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/WindowExpression.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/WindowExpression.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/WindowExpression.java
new file mode 100644
index 0000000..d1c4e59
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/WindowExpression.java
@@ -0,0 +1,105 @@
+/*
+ * 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.lang.sqlpp.expression;
+
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractExpression;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.clause.OrderbyClause;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
+
+public class WindowExpression extends AbstractExpression {
+    private Expression expr;
+    private List<Expression> partitionList;
+    private List<Expression> orderbyList;
+    private List<OrderbyClause.OrderModifier> orderbyModifierList;
+
+    public WindowExpression(Expression expr, List<Expression> partitionList, List<Expression> orderbyList,
+            List<OrderbyClause.OrderModifier> orderbyModifierList) {
+        if (expr == null || orderbyList == null) {
+            throw new NullPointerException();
+        }
+        this.expr = expr;
+        this.partitionList = partitionList;
+        this.orderbyList = orderbyList;
+        this.orderbyModifierList = orderbyModifierList;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.WINDOW_EXPRESSION;
+    }
+
+    public Expression getExpr() {
+        return expr;
+    }
+
+    public void setExpr(Expression expr) {
+        if (expr == null) {
+            throw new NullPointerException();
+        }
+        this.expr = expr;
+    }
+
+    public boolean hasPartitionList() {
+        return partitionList != null && !partitionList.isEmpty();
+    }
+
+    public List<Expression> getPartitionList() {
+        return partitionList;
+    }
+
+    public void setPartitionList(List<Expression> partitionList) {
+        if (partitionList == null) {
+            throw new NullPointerException();
+        }
+        this.partitionList = partitionList;
+    }
+
+    public List<Expression> getOrderbyList() {
+        return orderbyList;
+    }
+
+    public void setOrderbyList(List<Expression> orderbyList) {
+        if (orderbyList == null) {
+            throw new NullPointerException();
+        }
+        this.orderbyList = orderbyList;
+    }
+
+    public List<OrderbyClause.OrderModifier> getOrderbyModifierList() {
+        return orderbyModifierList;
+    }
+
+    public void setOrderbyModifierList(List<OrderbyClause.OrderModifier> orderbyModifierList) {
+        if (orderbyModifierList == null) {
+            throw new NullPointerException();
+        }
+        this.orderbyModifierList = orderbyModifierList;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return ((ISqlppVisitor<R, T>) visitor).visit(this, arg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index ec8b011..5aa5a8c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -50,6 +50,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.parser.FunctionParser;
 import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.GenerateColumnNameVisitor;
@@ -392,5 +393,18 @@ public class SqlppQueryRewriter implements IQueryRewriter {
             return null;
         }
 
+        @Override
+        public Void visit(WindowExpression winExpr, Void arg) throws CompilationException {
+            winExpr.getExpr().accept(this, arg);
+            if (winExpr.hasPartitionList()) {
+                for (Expression expr : winExpr.getPartitionList()) {
+                    expr.accept(this, arg);
+                }
+            }
+            for (Expression expr : winExpr.getOrderbyList()) {
+                expr.accept(this, arg);
+            }
+            return null;
+        }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
index db5b780..c9b2dea 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
@@ -44,6 +44,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
 import org.apache.asterix.lang.sqlpp.visitor.SqlppCloneAndSubstituteVariablesVisitor;
@@ -243,6 +244,22 @@ public class SqlppInlineUdfsVisitor extends AbstractInlineUdfsVisitor
         return inlined || result.first;
     }
 
+    @Override
+    public Boolean visit(WindowExpression winExpr, List<FunctionDecl> funcs) throws CompilationException {
+        Pair<Boolean, Expression> result = inlineUdfsInExpr(winExpr.getExpr(), funcs);
+        winExpr.setExpr(result.second);
+        boolean inlined = result.first;
+        if (winExpr.hasPartitionList()) {
+            Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getPartitionList(), funcs);
+            winExpr.setPartitionList(inlinedList.second);
+            inlined |= inlinedList.first;
+        }
+        Pair<Boolean, List<Expression>> inlinedList = inlineUdfsInExprList(winExpr.getOrderbyList(), funcs);
+        winExpr.setOrderbyList(inlinedList.second);
+        inlined |= inlinedList.first;
+        return inlined;
+    }
+
     private Map<Expression, Expression> extractLetBindingVariableExpressionMappings(List<LetClause> letClauses)
             throws CompilationException {
         Map<Expression, Expression> varExprMap = new HashMap<>();
@@ -253,5 +270,4 @@ public class SqlppInlineUdfsVisitor extends AbstractInlineUdfsVisitor
         }
         return varExprMap;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
index fe0c3a9..42b43e8 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
@@ -54,6 +54,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppQueryExpressionVisitor;
 
 /**
@@ -228,4 +229,9 @@ public class CheckDatasetOnlyResolutionVisitor extends AbstractSqlppQueryExpress
     public Boolean visit(CaseExpression caseExpr, ILangExpression arg) throws CompilationException {
         return false;
     }
+
+    @Override
+    public Boolean visit(WindowExpression windowExpression, ILangExpression arg) throws CompilationException {
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
index cd91405..6ea21a6 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
@@ -57,6 +57,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppQueryExpressionVisitor;
 
@@ -274,4 +275,10 @@ public class CheckSql92AggregateVisitor extends AbstractSqlppQueryExpressionVisi
                 || visitExprList(caseExpr.getThenExprs(), arg) || caseExpr.getElseExpr().accept(this, arg);
     }
 
+    @Override
+    public Boolean visit(WindowExpression winExpr, ILangExpression arg) throws CompilationException {
+        return winExpr.getExpr().accept(this, arg)
+                || (winExpr.hasPartitionList() && visitExprList(winExpr.getPartitionList(), arg))
+                || visitExprList(winExpr.getOrderbyList(), arg);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
index c53d8d2..34e918e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
@@ -61,6 +61,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppQueryExpressionVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -287,6 +288,12 @@ public class CheckSubqueryVisitor extends AbstractSqlppQueryExpressionVisitor<Bo
     }
 
     @Override
+    public Boolean visit(WindowExpression winExpr, ILangExpression arg) throws CompilationException {
+        return visit(winExpr.getExpr(), arg) || (winExpr.hasPartitionList() && visit(winExpr.getPartitionList(), arg))
+                || visit(winExpr.getOrderbyList(), arg);
+    }
+
+    @Override
     public Boolean visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
         return visit(callExpr.getExprList(), arg);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index 766540e..7dca268 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -65,6 +65,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppQueryExpressionVisitor;
@@ -283,7 +284,7 @@ public class DeepCopyVisitor extends AbstractSqlppQueryExpressionVisitor<ILangEx
         for (Expression orderExpr : oc.getOrderbyList()) {
             newOrderbyList.add((Expression) orderExpr.accept(this, arg));
         }
-        OrderbyClause copy = new OrderbyClause(newOrderbyList, oc.getModifierList());
+        OrderbyClause copy = new OrderbyClause(newOrderbyList, new ArrayList<>(oc.getModifierList()));
         copy.setSourceLocation(oc.getSourceLocation());
         return copy;
     }
@@ -495,6 +496,19 @@ public class DeepCopyVisitor extends AbstractSqlppQueryExpressionVisitor<ILangEx
         return copy;
     }
 
+    @Override
+    public ILangExpression visit(WindowExpression winExpr, Void arg) throws CompilationException {
+        Expression newExpr = (Expression) winExpr.getExpr().accept(this, arg);
+        List<Expression> newPartitionList =
+                winExpr.hasPartitionList() ? copyExprList(winExpr.getPartitionList(), arg) : null;
+        List<Expression> newOrderbyList = copyExprList(winExpr.getOrderbyList(), arg);
+        List<OrderbyClause.OrderModifier> newOrderbyModifierList = new ArrayList<>(winExpr.getOrderbyModifierList());
+        WindowExpression copy = new WindowExpression(newExpr, newPartitionList, newOrderbyList, newOrderbyModifierList);
+        copy.setSourceLocation(winExpr.getSourceLocation());
+        copy.addHints(winExpr.getHints());
+        return copy;
+    }
+
     private List<Expression> copyExprList(List<Expression> exprs, Void arg) throws CompilationException {
         List<Expression> newExprList = new ArrayList<>();
         for (Expression expr : exprs) {


[5/5] asterixdb git commit: [ASTERIXDB-2466][FUN] Implement window functions

Posted by dl...@apache.org.
[ASTERIXDB-2466][FUN] Implement window functions

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

Details:
- Implement window functions with SQL syntax:
  function() OVER ((PARTITION BY expr1, expr2, ...)? ORDER BY exprA, exprB, ...)
- Where supported functions are:
  ROW_NUMBER(), RANK(), DENSE_RANK(), PERCENT_RANK(), NTILE()

Change-Id: Ia28af8773cb11049c38d440c51b9c3cd1ed2bab4
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3002
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ali Alsuliman <al...@gmail.com>


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

Branch: refs/heads/master
Commit: fdedf6263cca9bb87a2baf0eccb7864a26c8aaf9
Parents: b9d55c4
Author: Dmitry Lychagin <dm...@couchbase.com>
Authored: Tue Oct 30 18:12:40 2018 -0700
Committer: Dmitry Lychagin <dm...@couchbase.com>
Committed: Wed Oct 31 20:54:12 2018 -0700

----------------------------------------------------------------------
 .../asterix/optimizer/base/RuleCollections.java |   2 +
 .../rules/ExtractOrderExpressionsRule.java      |  34 +---
 .../rules/ExtractWindowExpressionsRule.java     |  68 +++++++
 .../rules/SetAsterixPhysicalOperatorsRule.java  |  48 +++++
 .../SweepIllegalNonfunctionalFunctions.java     |  18 +-
 .../subplan/InlineAllNtsInSubplanVisitor.java   |   6 +
 ...neLeftNtsInSubplanJoinFlatteningVisitor.java |   7 +
 .../SubplanSpecialFlatteningCheckVisitor.java   |   6 +
 .../LangExpressionToPlanTranslator.java         |  13 +-
 .../SqlppExpressionToPlanTranslator.java        |  78 ++++++++
 .../app/resource/OperatorResourcesComputer.java |  10 +
 .../app/resource/PlanStagesGenerator.java       |   7 +
 .../app/resource/RequiredCapacityVisitor.java   |  13 ++
 .../optimizerts/queries/window/window_01.sqlpp  |  58 ++++++
 .../optimizerts/results/window/window_01.plan   |  30 +++
 .../dense_rank_01/dense_rank_01.1.ddl.sqlpp     |  72 +++++++
 .../dense_rank_01/dense_rank_01.2.update.sqlpp  |  36 ++++
 .../dense_rank_01/dense_rank_01.3.query.sqlpp   |  26 +++
 .../dense_rank_01/dense_rank_01.4.query.sqlpp   |  26 +++
 .../dense_rank_01/dense_rank_01.5.query.sqlpp   |  26 +++
 .../dense_rank_01/dense_rank_01.6.query.sqlpp   |  27 +++
 .../dense_rank_01/dense_rank_01.7.query.sqlpp   |  26 +++
 .../window/misc_01/misc_01.1.ddl.sqlpp          |  33 ++++
 .../window/misc_01/misc_01.2.update.sqlpp       |  33 ++++
 .../window/misc_01/misc_01.3.query.sqlpp        |  48 +++++
 .../window/misc_01/misc_01.4.query.sqlpp        |  32 ++++
 .../window/misc_01/misc_01.5.query.sqlpp        |  32 ++++
 .../window/ntile_01/ntile_01.1.ddl.sqlpp        |  75 ++++++++
 .../window/ntile_01/ntile_01.2.update.sqlpp     |  36 ++++
 .../window/ntile_01/ntile_01.3.query.sqlpp      |  26 +++
 .../window/ntile_01/ntile_01.4.query.sqlpp      |  26 +++
 .../window/ntile_01/ntile_01.5.query.sqlpp      |  26 +++
 .../window/ntile_01/ntile_01.6.query.sqlpp      |  27 +++
 .../window/ntile_01/ntile_01.7.query.sqlpp      |  26 +++
 .../window/ntile_02/ntile_02.1.query.sqlpp      |  25 +++
 .../percent_rank_01/percent_rank_01.1.ddl.sqlpp |  76 ++++++++
 .../percent_rank_01.2.update.sqlpp              |  36 ++++
 .../percent_rank_01.3.query.sqlpp               |  26 +++
 .../percent_rank_01.4.query.sqlpp               |  26 +++
 .../percent_rank_01.5.query.sqlpp               |  26 +++
 .../percent_rank_01.6.query.sqlpp               |  27 +++
 .../percent_rank_01.7.query.sqlpp               |  26 +++
 .../window/rank_01/rank_01.1.ddl.sqlpp          |  74 +++++++
 .../window/rank_01/rank_01.2.update.sqlpp       |  36 ++++
 .../window/rank_01/rank_01.3.query.sqlpp        |  26 +++
 .../window/rank_01/rank_01.4.query.sqlpp        |  26 +++
 .../window/rank_01/rank_01.5.query.sqlpp        |  26 +++
 .../window/rank_01/rank_01.6.query.sqlpp        |  27 +++
 .../window/rank_01/rank_01.7.query.sqlpp        |  26 +++
 .../row_number_01/row_number_01.1.ddl.sqlpp     |  72 +++++++
 .../row_number_01/row_number_01.2.update.sqlpp  |  36 ++++
 .../row_number_01/row_number_01.3.query.sqlpp   |  26 +++
 .../row_number_01/row_number_01.4.query.sqlpp   |  26 +++
 .../row_number_01/row_number_01.5.query.sqlpp   |  26 +++
 .../row_number_01/row_number_01.6.query.sqlpp   |  27 +++
 .../row_number_01/row_number_01.7.query.sqlpp   |  26 +++
 .../window/dense_rank_01/dense_rank_01.3.adm    |   1 +
 .../window/dense_rank_01/dense_rank_01.4.adm    |   1 +
 .../window/dense_rank_01/dense_rank_01.5.adm    |   1 +
 .../dense_rank_01/dense_rank_01.6.regexadm      |   1 +
 .../window/dense_rank_01/dense_rank_01.7.adm    |   1 +
 .../results/window/misc_01/misc_01.3.adm        |   1 +
 .../results/window/misc_01/misc_01.4.adm        |   4 +
 .../results/window/misc_01/misc_01.5.adm        |   4 +
 .../results/window/ntile_01/ntile_01.3.adm      |   1 +
 .../results/window/ntile_01/ntile_01.4.adm      |   1 +
 .../results/window/ntile_01/ntile_01.5.adm      |   1 +
 .../results/window/ntile_01/ntile_01.6.regexadm |   1 +
 .../results/window/ntile_01/ntile_01.7.adm      |   1 +
 .../percent_rank_01/percent_rank_01.3.adm       |   1 +
 .../percent_rank_01/percent_rank_01.4.adm       |   1 +
 .../percent_rank_01/percent_rank_01.5.adm       |   1 +
 .../percent_rank_01/percent_rank_01.6.regexadm  |   1 +
 .../percent_rank_01/percent_rank_01.7.adm       |   1 +
 .../results/window/rank_01/rank_01.3.adm        |   1 +
 .../results/window/rank_01/rank_01.4.adm        |   1 +
 .../results/window/rank_01/rank_01.5.adm        |   1 +
 .../results/window/rank_01/rank_01.6.regexadm   |   1 +
 .../results/window/rank_01/rank_01.7.adm        |   1 +
 .../window/row_number_01/row_number_01.3.adm    |   1 +
 .../window/row_number_01/row_number_01.4.adm    |   1 +
 .../window/row_number_01/row_number_01.5.adm    |   1 +
 .../row_number_01/row_number_01.6.regexadm      |   1 +
 .../window/row_number_01/row_number_01.7.adm    |   1 +
 .../resources/runtimets/testsuite_sqlpp.xml     |  39 ++++
 .../asterix/common/exceptions/ErrorCode.java    |   1 +
 .../main/resources/asx_errormsg/en.properties   |   1 +
 .../asterix/lang/common/base/Expression.java    |   3 +-
 .../CloneAndSubstituteVariablesVisitor.java     |   2 +-
 .../lang/common/visitor/FormatPrintVisitor.java |  31 +--
 .../lang/sqlpp/expression/WindowExpression.java | 105 ++++++++++
 .../lang/sqlpp/rewrites/SqlppQueryRewriter.java |  16 +-
 .../visitor/SqlppInlineUdfsVisitor.java         |  18 +-
 .../CheckDatasetOnlyResolutionVisitor.java      |   6 +
 .../visitor/CheckSql92AggregateVisitor.java     |   7 +
 .../sqlpp/visitor/CheckSubqueryVisitor.java     |   7 +
 .../lang/sqlpp/visitor/DeepCopyVisitor.java     |  16 +-
 .../lang/sqlpp/visitor/FreeVariableVisitor.java |  11 ++
 .../sqlpp/visitor/SqlppAstPrintVisitor.java     |  26 +++
 ...SqlppCloneAndSubstituteVariablesVisitor.java |  17 ++
 .../sqlpp/visitor/SqlppFormatPrintVisitor.java  |  22 +++
 .../visitor/base/AbstractSqlppAstVisitor.java   |   5 +
 .../AbstractSqlppSimpleExpressionVisitor.java   |  11 ++
 .../lang/sqlpp/visitor/base/ISqlppVisitor.java  |   3 +
 .../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj |  34 +++-
 .../asterix/om/functions/BuiltinFunctions.java  |  82 ++++++++
 .../runtime/functions/FunctionCollection.java   |  12 ++
 .../AbstractRankRunningAggregateEvaluator.java  | 130 +++++++++++++
 .../DenseRankRunningAggregateDescriptor.java    |  62 ++++++
 .../std/NtileRunningAggregateDescriptor.java    |  57 ++++++
 .../std/NtileRunningAggregateEvaluator.java     | 115 +++++++++++
 .../PercentRankRunningAggregateDescriptor.java  |  62 ++++++
 .../PercentRankRunningAggregateEvaluator.java   |  62 ++++++
 .../std/RankRunningAggregateDescriptor.java     |  62 ++++++
 .../std/RankRunningAggregateEvaluator.java      |  52 +++++
 .../RowNumberRunningAggregateDescriptor.java    |  56 ++++++
 .../std/RowNumberRunningAggregateEvaluator.java |  67 +++++++
 .../hyracks/algebricks/common/utils/Pair.java   |  16 ++
 .../core/algebra/base/ILogicalOperator.java     |   2 +-
 .../core/algebra/base/IPhysicalOperator.java    |   2 +-
 .../core/algebra/base/LogicalOperatorTag.java   |   1 +
 .../core/algebra/base/PhysicalOperatorTag.java  |   1 +
 .../logical/AbstractAssignOperator.java         |  14 ++
 .../logical/AbstractLogicalOperator.java        |   2 +-
 .../operators/logical/AggregateOperator.java    |  11 +-
 .../operators/logical/AssignOperator.java       |  13 +-
 .../logical/RunningAggregateOperator.java       |  12 +-
 .../operators/logical/WindowOperator.java       | 118 ++++++++++++
 .../visitors/CardinalityInferenceVisitor.java   |   6 +
 .../visitors/FDsAndEquivClassesVisitor.java     |   6 +
 .../visitors/IsomorphismOperatorVisitor.java    |  28 ++-
 .../IsomorphismVariableMappingVisitor.java      |   8 +
 ...OperatorDeepCopyWithNewVariablesVisitor.java |  15 ++
 .../visitors/LogicalPropertiesVisitor.java      |   8 +-
 .../visitors/OperatorDeepCopyVisitor.java       |  16 ++
 .../visitors/PrimaryKeyVariablesVisitor.java    |   5 +
 .../visitors/ProducedVariableVisitor.java       |   7 +
 .../logical/visitors/SchemaVariableVisitor.java |   6 +
 .../visitors/SubstituteVariableVisitor.java     |  59 +++---
 .../logical/visitors/UsedVariableVisitor.java   |  14 ++
 .../physical/RunningAggregatePOperator.java     |   2 +-
 .../operators/physical/WindowPOperator.java     | 191 +++++++++++++++++++
 .../LogicalOperatorPrettyPrintVisitor.java      |  29 ++-
 .../LogicalOperatorPrettyPrintVisitorJson.java  |  67 +++++--
 .../algebra/util/OperatorManipulationUtil.java  |   8 +
 .../visitors/ILogicalOperatorVisitor.java       |   3 +
 .../core/jobgen/impl/JobGenHelper.java          |  30 ++-
 .../core/utils/LogicalOperatorDotVisitor.java   |  47 +++--
 .../rewriter/rules/AbstractExtractExprRule.java |  47 ++++-
 .../rules/EnforceStructuralPropertiesRule.java  |   2 +-
 .../rules/ExtractGbyExpressionsRule.java        |  67 ++-----
 ...hNestedOrderByUnderPreSortedGroupByRule.java |   6 +
 ...placeNtsWithSubplanInputOperatorVisitor.java |   6 +
 .../algebricks/algebricks-runtime/pom.xml       |   5 +
 .../runtime/base/IWindowAggregateEvaluator.java |  30 +++
 .../AbstractRunningAggregatePushRuntime.java    | 111 +++++++++++
 .../aggrun/AbstractWindowPushRuntime.java       | 172 +++++++++++++++++
 .../aggrun/MaterializingWindowPushRuntime.java  | 152 +++++++++++++++
 .../aggrun/RunningAggregatePushRuntime.java     |  42 ++++
 .../aggrun/RunningAggregateRuntimeFactory.java  |  59 ++++++
 .../aggrun/SimpleWindowPushRuntime.java         |  54 ++++++
 .../operators/aggrun/WindowRuntimeFactory.java  |  65 +++++++
 .../std/RunningAggregateRuntimeFactory.java     | 140 --------------
 .../tests/pushruntime/PushRuntimeTest.java      |   2 +-
 .../hyracks/api/exceptions/ErrorCode.java       |   1 +
 .../src/main/resources/errormsg/en.properties   |   1 +
 .../dataflow/common/io/RunFileWriter.java       |   5 +
 .../preclustered/PreclusteredGroupWriter.java   |  14 +-
 168 files changed, 4398 insertions(+), 369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 1010a84..e7fb579 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -61,6 +61,7 @@ import org.apache.asterix.optimizer.rules.ListifyUnnestingFunctionRule;
 import org.apache.asterix.optimizer.rules.LoadRecordFieldsRule;
 import org.apache.asterix.optimizer.rules.MetaFunctionToMetaVariableRule;
 import org.apache.asterix.optimizer.rules.NestGroupByRule;
+import org.apache.asterix.optimizer.rules.ExtractWindowExpressionsRule;
 import org.apache.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
 import org.apache.asterix.optimizer.rules.PushAggregateIntoNestedSubplanRule;
 import org.apache.asterix.optimizer.rules.PushFieldAccessRule;
@@ -177,6 +178,7 @@ public final class RuleCollections {
         normalization.add(new ExtractGbyExpressionsRule());
         normalization.add(new ExtractDistinctByExpressionsRule());
         normalization.add(new ExtractOrderExpressionsRule());
+        normalization.add(new ExtractWindowExpressionsRule());
 
         // IntroduceStaticTypeCastRule should go before
         // IntroduceDynamicTypeCastRule to

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
index 585077f..826db09 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
@@ -18,18 +18,16 @@
  */
 package org.apache.asterix.optimizer.rules;
 
-import org.apache.commons.lang3.mutable.Mutable;
+import java.util.List;
 
 import org.apache.asterix.optimizer.base.AnalysisUtil;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
@@ -57,30 +55,12 @@ public class ExtractOrderExpressionsRule extends AbstractExtractExprRule {
         context.addToDontApplySet(this, op1);
         OrderOperator oo = (OrderOperator) op1;
 
-        if (!orderHasComplexExpr(oo)) {
-            return false;
-        }
-        Mutable<ILogicalOperator> opRef2 = oo.getInputs().get(0);
-        for (Pair<IOrder, Mutable<ILogicalExpression>> orderPair : oo.getOrderExpressions()) {
-            ILogicalExpression expr = orderPair.second.getValue();
-            if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE && !AnalysisUtil.isAccessToFieldRecord(expr)) {
-                LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
-                vRef.setSourceLocation(expr.getSourceLocation());
-                orderPair.second.setValue(vRef);
-            }
-        }
-        context.computeAndSetTypeEnvironmentForOperator(oo);
-        return true;
+        return extractComplexExpressions(oo, oo.getOrderExpressions(), context);
     }
 
-    private boolean orderHasComplexExpr(OrderOperator oo) {
-        for (Pair<IOrder, Mutable<ILogicalExpression>> orderPair : oo.getOrderExpressions()) {
-            if (orderPair.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                return true;
-            }
-        }
-        return false;
+    static boolean extractComplexExpressions(ILogicalOperator op,
+            List<Pair<IOrder, Mutable<ILogicalExpression>>> exprList, IOptimizationContext context)
+            throws AlgebricksException {
+        return extractComplexExpressions(op, exprList, Pair::getSecond, AnalysisUtil::isAccessToFieldRecord, context);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractWindowExpressionsRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractWindowExpressionsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractWindowExpressionsRule.java
new file mode 100644
index 0000000..042dae2
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractWindowExpressionsRule.java
@@ -0,0 +1,68 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.function.Function;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.rewriter.rules.AbstractExtractExprRule;
+import org.apache.hyracks.algebricks.rewriter.rules.ExtractGbyExpressionsRule;
+
+/**
+ * Extract complex expressions from window operator's partition and order definitions
+ */
+public class ExtractWindowExpressionsRule extends AbstractExtractExprRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.WINDOW) {
+            return false;
+        }
+        if (context.checkIfInDontApplySet(this, op)) {
+            return false;
+        }
+        context.addToDontApplySet(this, op);
+
+        WindowOperator winOp = (WindowOperator) op;
+
+        boolean rewritten = ExtractGbyExpressionsRule.extractComplexExpressions(winOp, winOp.getPartitionExpressions(),
+                Function.identity(), context);
+
+        rewritten |= ExtractOrderExpressionsRule.extractComplexExpressions(winOp, winOp.getOrderExpressions(), context);
+
+        if (rewritten) {
+            context.computeAndSetTypeEnvironmentForOperator(op);
+        }
+        return rewritten;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
----------------------------------------------------------------------
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 2dc5f2e..fdebd14 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
@@ -59,9 +59,13 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOpe
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.WindowPOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.rewriter.util.JoinUtils;
@@ -276,6 +280,12 @@ public class SetAsterixPhysicalOperatorsRule implements IAlgebraicRewriteRule {
                     }
                     break;
                 }
+                case WINDOW: {
+                    WindowOperator winOp = (WindowOperator) op;
+                    WindowPOperator physOp = createWindowPOperator(winOp);
+                    op.setPhysicalOperator(physOp);
+                    break;
+                }
             }
         }
         if (op.hasNestedPlans()) {
@@ -330,4 +340,42 @@ public class SetAsterixPhysicalOperatorsRule implements IAlgebraicRewriteRule {
         aggOp.setMergeExpressions(mergeExpressionRefs);
     }
 
+    private static WindowPOperator createWindowPOperator(WindowOperator winOp) throws CompilationException {
+        List<Mutable<ILogicalExpression>> partitionExprs = winOp.getPartitionExpressions();
+        List<LogicalVariable> partitionColumns = new ArrayList<>(partitionExprs.size());
+        for (Mutable<ILogicalExpression> pe : partitionExprs) {
+            ILogicalExpression partExpr = pe.getValue();
+            if (partExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, winOp.getSourceLocation(),
+                        "Window partition/order expression has not been normalized");
+            }
+            LogicalVariable var = ((VariableReferenceExpression) partExpr).getVariableReference();
+            partitionColumns.add(var);
+        }
+        List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExprs = winOp.getOrderExpressions();
+        List<OrderColumn> orderColumns = new ArrayList<>(orderExprs.size());
+        for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExprs) {
+            ILogicalExpression orderExpr = p.second.getValue();
+            if (orderExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, winOp.getSourceLocation(),
+                        "Window partition/order expression has not been normalized");
+            }
+            LogicalVariable var = ((VariableReferenceExpression) orderExpr).getVariableReference();
+            orderColumns.add(new OrderColumn(var, p.first.getKind()));
+        }
+        boolean partitionMaterialization = false;
+        for (Mutable<ILogicalExpression> exprRef : winOp.getExpressions()) {
+            ILogicalExpression expr = exprRef.getValue();
+            if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, winOp.getSourceLocation(),
+                        expr.getExpressionTag());
+            }
+            AbstractFunctionCallExpression callExpr = (AbstractFunctionCallExpression) expr;
+            if (BuiltinFunctions.windowFunctionRequiresMaterialization(callExpr.getFunctionIdentifier())) {
+                partitionMaterialization = true;
+                break;
+            }
+        }
+        return new WindowPOperator(partitionColumns, partitionMaterialization, orderColumns);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index 0c91e9b..a4250c0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -65,13 +65,13 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import org.apache.hyracks.algebricks.rewriter.rules.AbstractExtractExprRule;
 
-public class SweepIllegalNonfunctionalFunctions extends AbstractExtractExprRule implements IAlgebraicRewriteRule {
+public class SweepIllegalNonfunctionalFunctions implements IAlgebraicRewriteRule {
 
     private final IllegalNonfunctionalFunctionSweeperOperatorVisitor visitor;
 
@@ -313,6 +313,20 @@ public class SweepIllegalNonfunctionalFunctions extends AbstractExtractExprRule
             sweepExpression(op.getRangeMapExpression().getValue(), op);
             return null;
         }
+
+        @Override
+        public Void visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+            for (Mutable<ILogicalExpression> me : op.getPartitionExpressions()) {
+                sweepExpression(me.getValue(), op);
+            }
+            for (Pair<IOrder, Mutable<ILogicalExpression>> p : op.getOrderExpressions()) {
+                sweepExpression(p.second.getValue(), op);
+            }
+            for (Mutable<ILogicalExpression> me : op.getExpressions()) {
+                sweepExpression(me.getValue(), op);
+            }
+            return null;
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 22ef303..c540bbc 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -82,6 +82,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.LogicalOperatorDeepCopyWithNewVariablesVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
@@ -649,6 +650,11 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
                 "Forward operator should have been disqualified for this rewriting!");
     }
 
+    @Override
+    public ILogicalOperator visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        return visitSingleInputOperator(op);
+    }
+
     /**
      * Wraps an AggregateOperator or RunningAggregateOperator with a group-by
      * operator where the group-by keys are variables in keyVarsToEnforce. Note

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index b862a6f..3063e0a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -65,6 +65,8 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.visitors.IQueryOperatorVisitor;
 
@@ -384,6 +386,11 @@ class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisit
                 "Nested subplans with a forward operator should have been disqualified for this rewriting!");
     }
 
+    @Override
+    public ILogicalOperator visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        return visitSingleInputOperator(op);
+    }
+
     private ILogicalOperator visitSingleInputOperator(ILogicalOperator op) throws AlgebricksException {
         if (op.getInputs().size() == 1) {
             // Deals with single input operators.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
index e2b104d..288b01a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
@@ -52,6 +52,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.IQueryOperatorVisitor;
 
 /**
@@ -238,6 +239,11 @@ class SubplanSpecialFlatteningCheckVisitor implements IQueryOperatorVisitor<Bool
                 "Forward operator should have been disqualified for this rewriting!");
     }
 
+    @Override
+    public Boolean visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        return visitInputs(op);
+    }
+
     private boolean visitInputs(ILogicalOperator op) throws AlgebricksException {
         for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
             if (childRef.getValue().accept(this, null)) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 19fe02d..6d70ba5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -120,6 +120,7 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressio
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
@@ -151,6 +152,7 @@ import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.UnpartitionedPropertyComputer;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.io.FileSplit;
@@ -871,6 +873,8 @@ class LangExpressionToPlanTranslator
                     new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(fi), args);
             ufce.setReturnsUniqueValues(BuiltinFunctions.returnsUniqueValues(fi));
             f = ufce;
+        } else if (BuiltinFunctions.isBuiltinWindowFunction(fi)) {
+            f = BuiltinFunctions.makeWindowFunctionExpression(fi, args);
         } else {
             f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fi), args);
         }
@@ -1150,7 +1154,7 @@ class LangExpressionToPlanTranslator
         for (Expression e : oc.getOrderbyList()) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(e, topOp);
             OrderModifier m = modifIter.next();
-            OrderOperator.IOrder comp = (m == OrderModifier.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
+            OrderOperator.IOrder comp = translateOrderModifier(m);
             ord.getOrderExpressions().add(new Pair<>(comp, new MutableObject<>(p.first)));
             topOp = p.second;
         }
@@ -1170,6 +1174,10 @@ class LangExpressionToPlanTranslator
         return new Pair<>(ord, null);
     }
 
+    protected OrderOperator.IOrder translateOrderModifier(OrderModifier m) {
+        return m == OrderModifier.ASC ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
+    }
+
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
@@ -1587,8 +1595,7 @@ class LangExpressionToPlanTranslator
                 || k == Kind.FIELD_ACCESSOR_EXPRESSION;
         noNesting = noNesting || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION
                 || k == Kind.IF_EXPRESSION;
-        return noNesting || k == Kind.CASE_EXPRESSION;
-
+        return noNesting || k == Kind.CASE_EXPRESSION || k == Kind.WINDOW_EXPRESSION;
     }
 
     protected <T> List<T> mkSingletonArrayList(T item) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
----------------------------------------------------------------------
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 03c4bc5..dfe0208 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
@@ -38,6 +38,7 @@ import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
+import org.apache.asterix.lang.common.clause.OrderbyClause;
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.FieldBinding;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
@@ -68,6 +69,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
 import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
 import org.apache.asterix.lang.sqlpp.optype.JoinType;
 import org.apache.asterix.lang.sqlpp.optype.SetOpType;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
@@ -94,6 +96,7 @@ import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
@@ -110,10 +113,12 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
@@ -1017,4 +1022,77 @@ public class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTransla
         opExpr.getArguments().add(new MutableObject<>(rhsExpr));
         return opExpr;
     }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visit(WindowExpression winExpr, Mutable<ILogicalOperator> tupSource)
+            throws CompilationException {
+        SourceLocation sourceLoc = winExpr.getSourceLocation();
+        Mutable<ILogicalOperator> currentOpRef = tupSource;
+
+        List<Mutable<ILogicalExpression>> partExprListOut = null;
+        if (winExpr.hasPartitionList()) {
+            List<Expression> partExprList = winExpr.getPartitionList();
+            partExprListOut = new ArrayList<>(partExprList.size());
+            for (Expression partExpr : partExprList) {
+                Pair<ILogicalOperator, LogicalVariable> partExprResult = partExpr.accept(this, currentOpRef);
+                VariableReferenceExpression partExprOut = new VariableReferenceExpression(partExprResult.second);
+                partExprOut.setSourceLocation(partExpr.getSourceLocation());
+                partExprListOut.add(new MutableObject<>(partExprOut));
+                currentOpRef = new MutableObject<>(partExprResult.first);
+            }
+        }
+
+        List<Expression> orderExprList = winExpr.getOrderbyList();
+        List<OrderbyClause.OrderModifier> orderModifierList = winExpr.getOrderbyModifierList();
+        List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExprListOut =
+                new ArrayList<>(orderExprList.size());
+        for (int i = 0, ln = orderExprList.size(); i < ln; i++) {
+            Expression orderExpr = orderExprList.get(i);
+            OrderbyClause.OrderModifier orderModifier = orderModifierList.get(i);
+            Pair<ILogicalOperator, LogicalVariable> orderExprResult = orderExpr.accept(this, currentOpRef);
+            VariableReferenceExpression orderExprOut = new VariableReferenceExpression(orderExprResult.second);
+            orderExprOut.setSourceLocation(orderExpr.getSourceLocation());
+            OrderOperator.IOrder orderModifierOut = translateOrderModifier(orderModifier);
+            orderExprListOut.add(new Pair<>(orderModifierOut, new MutableObject<>(orderExprOut)));
+            currentOpRef = new MutableObject<>(orderExprResult.first);
+        }
+
+        Expression expr = winExpr.getExpr();
+        Pair<ILogicalOperator, LogicalVariable> exprResult = expr.accept(this, currentOpRef);
+        ILogicalOperator exprOp = exprResult.first;
+        if (exprOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc);
+        }
+        AssignOperator exprAssignOp = (AssignOperator) exprOp;
+        currentOpRef = exprAssignOp.getInputs().get(0);
+        List<LogicalVariable> exprAssignVars = exprAssignOp.getVariables();
+        if (exprAssignVars.size() != 1) {
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc);
+        }
+        LogicalVariable exprAssignVar = exprAssignVars.get(0);
+        List<Mutable<ILogicalExpression>> exprAssignExprs = exprAssignOp.getExpressions();
+        ILogicalExpression exprAssignExpr = exprAssignExprs.get(0).getValue();
+        if (exprAssignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            throw new CompilationException(ErrorCode.COMPILATION_EXPECTED_FUNCTION_CALL, sourceLoc);
+        }
+        AbstractFunctionCallExpression callExpr = (AbstractFunctionCallExpression) exprAssignExpr;
+        if (BuiltinFunctions.windowFunctionRequiresOrderArgs(callExpr.getFunctionIdentifier())) {
+            List<Mutable<ILogicalExpression>> callArgs = callExpr.getArguments();
+            for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : orderExprListOut) {
+                callArgs.add(new MutableObject<>(p.second.getValue().cloneExpression()));
+            }
+        }
+
+        WindowOperator winOp = new WindowOperator(partExprListOut, orderExprListOut, exprAssignVars, exprAssignExprs);
+        winOp.setSourceLocation(sourceLoc);
+        winOp.getInputs().add(currentOpRef);
+
+        // must return ASSIGN
+        LogicalVariable assignVar = context.newVar();
+        AssignOperator assignOp =
+                new AssignOperator(assignVar, new MutableObject<>(new VariableReferenceExpression(exprAssignVar)));
+        assignOp.setSourceLocation(sourceLoc);
+        assignOp.getInputs().add(new MutableObject<>(winOp));
+        return new Pair<>(assignOp, assignVar);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
index c7db521..8fc3b82 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
@@ -24,6 +24,8 @@ import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.WindowPOperator;
 
 public class OperatorResourcesComputer {
 
@@ -106,6 +108,8 @@ public class OperatorResourcesComputer {
             case INNERJOIN:
             case LEFTOUTERJOIN:
                 return getOperatorRequiredMemory(operator, joinMemorySize);
+            case WINDOW:
+                return getWindowRequiredMemory((WindowOperator) operator);
             default:
                 throw new IllegalStateException("Unrecognized operator: " + operator.getOperatorTag());
         }
@@ -138,4 +142,10 @@ public class OperatorResourcesComputer {
         }
         return 2L * MAX_BUFFER_PER_CONNECTION * numComputationPartitions * numComputationPartitions * frameSize;
     }
+
+    private long getWindowRequiredMemory(WindowOperator op) {
+        WindowPOperator physOp = (WindowPOperator) op.getPhysicalOperator();
+        int frameCount = physOp.isPartitionMaterialization() ? 3 : 2;
+        return getOperatorRequiredMemory(op, frameSize * frameCount);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
index 0023a7a..25e51bb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
@@ -63,6 +63,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
@@ -316,6 +317,12 @@ public class PlanStagesGenerator implements ILogicalOperatorVisitor<Void, Void>
         return null;
     }
 
+    @Override
+    public Void visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        visit(op);
+        return null;
+    }
+
     public List<PlanStage> getStages() {
         return stages;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
index 368a244..c0fca94 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
@@ -61,8 +61,10 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.WindowPOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 import org.apache.hyracks.api.job.resource.IClusterCapacity;
 
@@ -311,6 +313,17 @@ public class RequiredCapacityVisitor implements ILogicalOperatorVisitor<Void, Vo
         return null;
     }
 
+    @Override
+    public Void visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+        WindowPOperator physOp = (WindowPOperator) op.getPhysicalOperator();
+        visitInternal(op, true);
+        addOutputBuffer(op); // + previous frame
+        if (physOp.isPartitionMaterialization()) {
+            addOutputBuffer(op); // + run frame
+        }
+        return null;
+    }
+
     // Calculates the memory usage for exchange operators.
     private void calculateMemoryUsageForExchange(ExchangeOperator op) throws AlgebricksException {
         visitInternal(op, false);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/window_01.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/window_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/window_01.sqlpp
new file mode 100644
index 0000000..25613c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/window/window_01.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 multiple window functions in a single statement
+ * Expected Res : SUCCESS
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.t1Type as open {
+  c1 : bigint
+};
+
+create dataset t1(t1Type) primary key c1;
+
+create function q1_mixed(P, N, D) {
+  from
+    t1,
+    range(t1.one, P) p,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    rank_result_expected = (N - n) * D + 1,
+    rank_result_actual = rank() over ( partition by t1.c2, p order by n desc ),
+    rank_result_delta = rank_result_expected - rank_result_actual,
+
+    percent_rank_result_expected = (rank_result_expected - 1) / (N * D - 1),
+    percent_rank_result_actual = percent_rank() over ( partition by t1.c2, p order by n desc ),
+    percent_rank_result_delta_raw = percent_rank_result_expected - percent_rank_result_actual,
+    percent_rank_result_delta = case when percent_rank_result_delta_raw < 0.001 then 0 else percent_rank_result_delta_raw end
+
+  select
+    min(rank_result_delta) rank_min_delta,
+    max(rank_result_delta) rank_max_delta,
+    min(percent_rank_result_delta) percent_rank_min_delta,
+    max(percent_rank_result_delta) percent_rank_max_delta
+};
+
+q1_mixed(2, 2, 2);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/window_01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/window_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/window_01.plan
new file mode 100644
index 0000000..91e9cfc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/window/window_01.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- WINDOW  |PARTITIONED|
+                              -- WINDOW  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$t1.c2(ASC), $$p(ASC), $$n(DESC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$t1.c2, $$p]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- UNNEST  |PARTITIONED|
+                                          -- UNNEST  |PARTITIONED|
+                                            -- UNNEST  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.1.ddl.sqlpp
new file mode 100644
index 0000000..9b6169e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.1.ddl.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * 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 DENSE_RANK()
+ * Expected Res : SUCCESS
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.t1Type as open {
+  c1 : bigint
+};
+
+create dataset t1(t1Type) primary key c1;
+
+create dataset tRnd(t1Type) primary key c1;
+
+create function q0_rnd() {
+  let
+    rnd = tobigint((select value tRnd.rnd from tRnd where c1 = 1)[0] * 1000),
+    p = case when rnd >= 10 then rnd else 10 end,
+    n = tobigint( 1000 * 100 / p)
+  select p, n
+};
+
+create function q1_dense_rank(P, N, D) {
+  from
+    t1,
+    range(t1.one, P) p,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    result_expected = N - n + 1,
+    result_actual = dense_rank() over ( partition by t1.c2, p order by n desc ),
+    result_delta = result_expected - result_actual
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};
+
+create function q2_dense_rank_no_partition(N, D) {
+  from
+    t1,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    result_expected = (t1.c2 - 1) * N + N - n + 1,
+    result_actual = dense_rank() over ( order by t1.c2, n desc ),
+    result_delta = result_expected - result_actual
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.2.update.sqlpp
new file mode 100644
index 0000000..6386573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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 DENSE_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+set `import-private-functions` `true`;
+
+insert into t1
+select element { "c1":1, "c2": 1, "one": 1 };
+
+insert into t1
+select v c1, v c2, 1 one
+from range(2, (select value count(*) from storage_components("test","t1") t)[0]) v;
+
+insert into tRnd
+select 1 c1, random() rnd;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.3.query.sqlpp
new file mode 100644
index 0000000..b88e7ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 DENSE_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_dense_rank(1, 1, 3)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.4.query.sqlpp
new file mode 100644
index 0000000..5c6e06a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 DENSE_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_dense_rank(1000, 10, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.5.query.sqlpp
new file mode 100644
index 0000000..29c8366
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 DENSE_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_dense_rank(10, 1000, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.6.query.sqlpp
new file mode 100644
index 0000000..1de6526
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.6.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 DENSE_RANK()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+from q0_rnd() rnd
+select q1_dense_rank(rnd.p, rnd.n, 3) res, rnd.p, rnd.n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.7.query.sqlpp
new file mode 100644
index 0000000..592dc4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/dense_rank_01/dense_rank_01.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 DENSE_RANK() without partition clause
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q2_dense_rank_no_partition(7, 11)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.1.ddl.sqlpp
new file mode 100644
index 0000000..b7aa06a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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  : Miscellaneous window function tests
+ * Expected Res : SUCCESS
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.t1Type as open {
+  c1 : bigint
+};
+
+create dataset t1(t1Type) primary key c1;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.2.update.sqlpp
new file mode 100644
index 0000000..21fd00e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.2.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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  : Miscellaneous window function tests
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+set `import-private-functions` `true`;
+
+insert into t1
+select element { "c1":1, "c2": 1, "one": 1 };
+
+insert into t1
+select v c1, v c2, 1 one
+from range(2, (select value count(*) from storage_components("test","t1") t)[0]) v;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.3.query.sqlpp
new file mode 100644
index 0000000..2948a3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.3.query.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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 multiple window functions in the same statement
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+with P as 10, N as 10, D as 4
+
+from
+  t1,
+  range(t1.one, P) p,
+  range(t1.one, N) n,
+  range(t1.one, D) d
+let
+  rank_result_expected = (N - n) * D + 1,
+  rank_result_actual = rank() over ( partition by t1.c2, p order by n desc ),
+  rank_result_delta = rank_result_expected - rank_result_actual,
+
+  percent_rank_result_expected = (rank_result_expected - 1) / (N * D - 1),
+  percent_rank_result_actual = percent_rank() over ( partition by t1.c2, p order by n desc ),
+  percent_rank_result_delta_raw = percent_rank_result_expected - percent_rank_result_actual,
+  percent_rank_result_delta = case when percent_rank_result_delta_raw < 0.001 then 0 else percent_rank_result_delta_raw end
+
+select
+  min(rank_result_delta) rank_min_delta,
+  max(rank_result_delta) rank_max_delta,
+  min(percent_rank_result_delta) percent_rank_min_delta,
+  max(percent_rank_result_delta) percent_rank_max_delta
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.4.query.sqlpp
new file mode 100644
index 0000000..d3f7916
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 after group by
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+with P as 4
+
+from t1, range(t1.one, P) p
+group by p
+select p, row_number() over (order by p desc) rn
+order by p
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.5.query.sqlpp
new file mode 100644
index 0000000..c0548c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.5.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 inside nested plan
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+with P as 4
+
+from t1, range(t1.one, P) p
+group by p group as g
+select p, ( from g select g.t1.c1, row_number() over ( order by g.t1.c2 desc ) rn order by g.t1.c1 ) gg
+order by p
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.1.ddl.sqlpp
new file mode 100644
index 0000000..a81cab7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.1.ddl.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * 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 NTILE()
+ * Expected Res : SUCCESS
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.t1Type as open {
+  c1 : bigint
+};
+
+create dataset t1(t1Type) primary key c1;
+
+create dataset tRnd(t1Type) primary key c1;
+
+create function q0_rnd() {
+  let
+    rnd = tobigint((select value tRnd.rnd from tRnd where c1 = 1)[0] * 1000),
+    p = case when rnd >= 10 then rnd else 10 end,
+    n = tobigint( 1000 * 100 / p)
+  select p, n
+};
+
+create function q1_ntile(P, N, D) {
+  from
+    t1,
+    range(t1.one, P) p,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    rownum = (N - n) * D + d,
+    result_expected = tobigint( (rownum - 1) / N ) + 1,
+    result_actual = ntile(D) over ( partition by t1.c2, p order by n desc, d ),
+    result_delta = result_expected - result_actual
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};
+
+create function q2_ntile_no_partition(N, D) {
+  from
+    t1,
+    range(t1.one, N) n,
+    range(t1.one, D) d
+  let
+    rownum = ((t1.c2 - 1) * N * D) + ((N - n) * D + d),
+    result_expected = tobigint( (rownum - 1) / (4 * N) ) + 1,
+    result_actual = ntile(D) over ( order by t1.c2, n desc, d ),
+    result_delta = result_expected - result_actual
+  select
+    min(result_delta) min_delta,
+    max(result_delta) max_delta
+};
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.2.update.sqlpp
new file mode 100644
index 0000000..6286e28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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 NTILE()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+set `import-private-functions` `true`;
+
+insert into t1
+select element { "c1":1, "c2": 1, "one": 1 };
+
+insert into t1
+select v c1, v c2, 1 one
+from range(2, (select value count(*) from storage_components("test","t1") t)[0]) v;
+
+insert into tRnd
+select 1 c1, random() rnd;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.3.query.sqlpp
new file mode 100644
index 0000000..245bd58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 NTILE()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_ntile(1, 1, 2)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.4.query.sqlpp
new file mode 100644
index 0000000..405c2ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 NTILE()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_ntile(1000, 10, 3)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/fdedf626/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.5.query.sqlpp
new file mode 100644
index 0000000..5655f18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/ntile_01/ntile_01.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 NTILE()
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+q1_ntile(10, 1000, 4)