You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2014/06/28 00:27:53 UTC

[6/7] [OPTIQ-310] Implement LEAD, LAG and NTILE windowed aggregates.

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/99f39682/core/src/main/java/org/eigenbase/sql/fun/SqlLeadLagAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/fun/SqlLeadLagAggFunction.java b/core/src/main/java/org/eigenbase/sql/fun/SqlLeadLagAggFunction.java
new file mode 100644
index 0000000..f996300
--- /dev/null
+++ b/core/src/main/java/org/eigenbase/sql/fun/SqlLeadLagAggFunction.java
@@ -0,0 +1,94 @@
+/*
+// Licensed to Julian Hyde under one or more contributor license
+// agreements. See the NOTICE file distributed with this work for
+// additional information regarding copyright ownership.
+//
+// Julian Hyde 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.eigenbase.sql.fun;
+
+import java.util.List;
+
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.sql.*;
+import org.eigenbase.sql.type.*;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * <code>LEAD</code> and <code>LAG</code> aggregate functions
+ * return the value of given expression evaluated at given offset.
+ */
+public class SqlLeadLagAggFunction extends SqlAggFunction {
+  private static final SqlSingleOperandTypeChecker OPERAND_TYPES =
+      OperandTypes.or(
+          OperandTypes.ANY,
+          OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC),
+          OperandTypes.and(
+              OperandTypes.family(
+                  SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.ANY)
+              // Arguments 1 and 3 must have same type
+              , new SameOperandTypeChecker(3) {
+                @Override
+                protected List<Integer> getOperandList(int operandCount) {
+                  return ImmutableList.of(0, 2);
+                }
+              }));
+
+  private static final SqlReturnTypeInference RETURN_TYPE =
+      ReturnTypes.cascade(ReturnTypes.ARG0, new SqlTypeTransform() {
+        public RelDataType transformType(SqlOperatorBinding binding,
+            RelDataType type) {
+          // Result is NOT NULL if NOT NULL default value is provided
+          SqlTypeTransform transform;
+          if (binding.getOperandCount() < 3) {
+            transform = SqlTypeTransforms.FORCE_NULLABLE;
+          } else {
+            RelDataType defValueType = binding.getOperandType(2);
+            transform = defValueType.isNullable()
+                ? SqlTypeTransforms.FORCE_NULLABLE
+                : SqlTypeTransforms.TO_NOT_NULLABLE;
+          }
+          return transform.transformType(binding, type);
+        }
+      });
+
+  public SqlLeadLagAggFunction(boolean isLead) {
+    super(
+        isLead ? "LEAD" : "LAG",
+        SqlKind.OTHER_FUNCTION,
+        RETURN_TYPE,
+        null,
+        OPERAND_TYPES,
+        SqlFunctionCategory.NUMERIC);
+  }
+
+  @Override public boolean requiresOrder() {
+    return true;
+  }
+
+  @Override public boolean allowsFraming() {
+    return false;
+  }
+
+  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before optiq-0.9");
+  }
+
+  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before optiq-0.9");
+  }
+}
+
+// End SqlLeadLagAggFunction.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/99f39682/core/src/main/java/org/eigenbase/sql/fun/SqlNtileAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/fun/SqlNtileAggFunction.java b/core/src/main/java/org/eigenbase/sql/fun/SqlNtileAggFunction.java
new file mode 100644
index 0000000..ed13608
--- /dev/null
+++ b/core/src/main/java/org/eigenbase/sql/fun/SqlNtileAggFunction.java
@@ -0,0 +1,58 @@
+/*
+// Licensed to Julian Hyde under one or more contributor license
+// agreements. See the NOTICE file distributed with this work for
+// additional information regarding copyright ownership.
+//
+// Julian Hyde 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.eigenbase.sql.fun;
+
+import java.util.List;
+
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.sql.SqlAggFunction;
+import org.eigenbase.sql.SqlFunctionCategory;
+import org.eigenbase.sql.SqlKind;
+import org.eigenbase.sql.type.OperandTypes;
+import org.eigenbase.sql.type.ReturnTypes;
+
+/**
+ * <code>NTILE</code> aggregate function
+ * return the value of given expression evaluated at given offset.
+ */
+public class SqlNtileAggFunction extends SqlAggFunction {
+  public SqlNtileAggFunction() {
+    super(
+        "NTILE",
+        SqlKind.OTHER_FUNCTION,
+        ReturnTypes.INTEGER,
+        null,
+        OperandTypes.POSITIVE_INTEGER_LITERAL,
+        SqlFunctionCategory.NUMERIC);
+  }
+
+  @Override public boolean requiresOrder() {
+    return true;
+  }
+
+  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before optiq-0.9");
+  }
+
+  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
+    throw new UnsupportedOperationException("remove before optiq-0.9");
+  }
+}
+
+// End SqlNtileAggFunction.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/99f39682/core/src/main/java/org/eigenbase/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/eigenbase/sql/fun/SqlStdOperatorTable.java
index afb062d..87f9177 100644
--- a/core/src/main/java/org/eigenbase/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/eigenbase/sql/fun/SqlStdOperatorTable.java
@@ -624,6 +624,24 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
       new SqlFirstLastValueAggFunction(true);
 
   /**
+   * <code>LEAD</code> aggregate function.
+   */
+  public static final SqlAggFunction LEAD =
+      new SqlLeadLagAggFunction(true);
+
+  /**
+   * <code>LAG</code> aggregate function.
+   */
+  public static final SqlAggFunction LAG =
+      new SqlLeadLagAggFunction(false);
+
+  /**
+   * <code>NTILE</code> aggregate function.
+   */
+  public static final SqlAggFunction NTILE =
+      new SqlNtileAggFunction();
+
+  /**
    * <code>SINGLE_VALUE</code> aggregate function.
    */
   public static final SqlAggFunction SINGLE_VALUE =

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/99f39682/core/src/main/java/org/eigenbase/sql/type/CompositeOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/type/CompositeOperandTypeChecker.java b/core/src/main/java/org/eigenbase/sql/type/CompositeOperandTypeChecker.java
index 7bb4996..e1e3f9e 100644
--- a/core/src/main/java/org/eigenbase/sql/type/CompositeOperandTypeChecker.java
+++ b/core/src/main/java/org/eigenbase/sql/type/CompositeOperandTypeChecker.java
@@ -294,6 +294,12 @@ public class CompositeOperandTypeChecker
       default:
         if (!rule.checkOperandTypes(callBinding, false)) {
           typeErrorCount++;
+          if (composition == Composition.AND) {
+            // Avoid trying other rules in AND if the first one fails.
+            break label;
+          }
+        } else if (composition == Composition.OR) {
+          break label; // true OR any == true, just break
         }
         break;
       }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/99f39682/core/src/main/java/org/eigenbase/sql/type/SqlSingleOperandTypeChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/type/SqlSingleOperandTypeChecker.java b/core/src/main/java/org/eigenbase/sql/type/SqlSingleOperandTypeChecker.java
index 323d7b0..f81c916 100644
--- a/core/src/main/java/org/eigenbase/sql/type/SqlSingleOperandTypeChecker.java
+++ b/core/src/main/java/org/eigenbase/sql/type/SqlSingleOperandTypeChecker.java
@@ -21,7 +21,7 @@ import org.eigenbase.sql.*;
 
 /**
  * SqlSingleOperandTypeChecker is an extension of {@link SqlOperandTypeChecker}
- * for implementations which are cabable of checking the type of a single
+ * for implementations which are capable of checking the type of a single
  * operand in isolation. This isn't meaningful for all type-checking rules (e.g.
  * SameOperandTypeChecker requires two operands to have matching types, so
  * checking one in isolation is meaningless).

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/99f39682/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java b/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
index 26f72bd..76d54a2 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
@@ -146,7 +146,7 @@ public class JdbcTest {
 
 
   public static final String START_OF_GROUP_DATA =
-      "values"
+      "(values"
       + "(1,0,1),\n"
       + "(2,0,1),\n"
       + "(3,1,2),\n"
@@ -3142,20 +3142,16 @@ public class JdbcTest {
             "deptno=20; R=4"); // 4 for rank and 2 for dense_rank
   }
 
-  /**
-   * Tests start_of_group approach for grouping of adjacent intervals.
-   * This is a step1, implemented as last_value.
-   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
-   */
-  @Test public void testStartOfGroupLastValueStep1() {
+  private void startOfGroupStep1(String startOfGroup) {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .query(
-                "select t.*\n"
+            "select t.*\n"
                 + "  from (\n"
                 + "       select  t.*,\n"
-                + "               case when val = last_value(val) over (order by rn rows between 1 preceding and 1 preceding) then 0 else 1 end start_of_group\n"
-                + "         from ("
+                + "               case when " + startOfGroup
+                + " then 0 else 1 end start_of_group\n"
+                + "         from "
                 + START_OF_GROUP_DATA
                 + ") t\n")
         .typeIs(
@@ -3169,25 +3165,20 @@ public class JdbcTest {
             "RN=6; VAL=0; EXPECTED=3; START_OF_GROUP=0",
             "RN=7; VAL=1; EXPECTED=4; START_OF_GROUP=1",
             "RN=8; VAL=1; EXPECTED=4; START_OF_GROUP=0");
-
   }
 
-  /**
-   * Tests start_of_group approach for grouping of adjacent intervals.
-   * This is a step2, that gets the final group numbers
-   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
-   */
-  @Test public void testStartOfGroupLastValueStep2() {
+  private void startOfGroupStep2(String startOfGroup) {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .query(
-                "select t.*\n"
+            "select t.*\n"
                 // current row is assumed, group_id should be NOT NULL
                 + "       ,sum(start_of_group) over (order by rn rows unbounded preceding) group_id\n"
                 + "  from (\n"
                 + "       select  t.*,\n"
-                + "               case when val = last_value(val) over (order by rn rows between 1 preceding and 1 preceding) then 0 else 1 end start_of_group\n"
-                + "         from ("
+                + "               case when " + startOfGroup
+                + " then 0 else 1 end start_of_group\n"
+                + "         from "
                 + START_OF_GROUP_DATA
                 + ") t\n")
         .typeIs(
@@ -3201,27 +3192,22 @@ public class JdbcTest {
             "RN=6; VAL=0; EXPECTED=3; START_OF_GROUP=0; GROUP_ID=3",
             "RN=7; VAL=1; EXPECTED=4; START_OF_GROUP=1; GROUP_ID=4",
             "RN=8; VAL=1; EXPECTED=4; START_OF_GROUP=0; GROUP_ID=4");
-
   }
 
-  /**
-   * Tests start_of_group approach for grouping of adjacent intervals.
-   * This is a step3, that aggregates the computed groups
-   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
-   */
-  @Test public void testStartOfGroupLastValueStep3() {
+  private void startOfGroupStep3(String startOfGroup) {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .query(
-                "select group_id, min(rn) min_rn, max(rn) max_rn, count(rn) cnt_rn, avg(val) avg_val"
+            "select group_id, min(rn) min_rn, max(rn) max_rn, count(rn) cnt_rn, avg(val) avg_val"
                 + " from (\n"
                 + "select t.*\n"
                 // current row is assumed, group_id should be NOT NULL
                 + "       ,sum(start_of_group) over (order by rn rows unbounded preceding) group_id\n"
                 + "  from (\n"
                 + "       select  t.*,\n"
-                + "               case when val = last_value(val) over (order by rn rows between 1 preceding and 1 preceding) then 0 else 1 end start_of_group\n"
-                + "         from ("
+                + "               case when " + startOfGroup
+                + " then 0 else 1 end start_of_group\n"
+                + "         from "
                 + START_OF_GROUP_DATA
                 + ") t\n"
                 + ") group by group_id\n")
@@ -3232,7 +3218,36 @@ public class JdbcTest {
             "GROUP_ID=2; MIN_RN=3; MAX_RN=3; CNT_RN=1; AVG_VAL=1",
             "GROUP_ID=3; MIN_RN=4; MAX_RN=6; CNT_RN=3; AVG_VAL=0",
             "GROUP_ID=4; MIN_RN=7; MAX_RN=8; CNT_RN=2; AVG_VAL=1");
+  }
+
+  /**
+   * Tests start_of_group approach for grouping of adjacent intervals.
+   * This is a step1, implemented as last_value.
+   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   */
+  @Test public void testStartOfGroupLastValueStep1() {
+    startOfGroupStep1(
+        "val = last_value(val) over (order by rn rows between 1 preceding and 1 preceding)");
+  }
 
+  /**
+   * Tests start_of_group approach for grouping of adjacent intervals.
+   * This is a step2, that gets the final group numbers
+   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   */
+  @Test public void testStartOfGroupLastValueStep2() {
+    startOfGroupStep2(
+        "val = last_value(val) over (order by rn rows between 1 preceding and 1 preceding)");
+  }
+
+  /**
+   * Tests start_of_group approach for grouping of adjacent intervals.
+   * This is a step3, that aggregates the computed groups
+   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   */
+  @Test public void testStartOfGroupLastValueStep3() {
+    startOfGroupStep3(
+        "val = last_value(val) over (order by rn rows between 1 preceding and 1 preceding)");
   }
 
   /**
@@ -3240,94 +3255,225 @@ public class JdbcTest {
    * This is a step1, implemented as last_value.
    * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
    */
-  @Ignore("LEAD/LAG is not implemented yet")
   @Test public void testStartOfGroupLagStep1() {
+    startOfGroupStep1("val = lag(val) over (order by rn)");
+  }
+
+  /**
+   * Tests start_of_group approach for grouping of adjacent intervals.
+   * This is a step2, that gets the final group numbers
+   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   */
+  @Test public void testStartOfGroupLagValueStep2() {
+    startOfGroupStep2("val = lag(val) over (order by rn)");
+  }
+
+  /**
+   * Tests start_of_group approach for grouping of adjacent intervals.
+   * This is a step3, that aggregates the computed groups
+   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   */
+  @Test public void testStartOfGroupLagStep3() {
+    startOfGroupStep3("val = lag(val) over (order by rn)");
+  }
+
+  /**
+   * Tests start_of_group approach for grouping of adjacent intervals.
+   * This is a step1, implemented as last_value.
+   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   */
+  @Test public void testStartOfGroupLeadStep1() {
+    startOfGroupStep1("val = lead(val, -1) over (order by rn)");
+  }
+
+  /**
+   * Tests start_of_group approach for grouping of adjacent intervals.
+   * This is a step2, that gets the final group numbers
+   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   */
+  @Test public void testStartOfGroupLeadValueStep2() {
+    startOfGroupStep2("val = lead(val, -1) over (order by rn)");
+  }
+
+  /**
+   * Tests start_of_group approach for grouping of adjacent intervals.
+   * This is a step3, that aggregates the computed groups
+   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   */
+  @Test public void testStartOfGroupLeadStep3() {
+    startOfGroupStep3("val = lead(val, -1) over (order by rn)");
+  }
+
+  /**
+   * Tests default value of LAG function.
+   */
+  @Test public void testLagDefaultValue() {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .query(
-                "select t.*\n"
-                + "  from (\n"
-                + "       select  t.*,\n"
-                + "               case when val = lag(val) over (order by rn) then 0 else 1 end start_of_group\n"
-                + "         from ("
-                + START_OF_GROUP_DATA
-                + ") t\n")
+            "select t.*, lag(rn+expected,1,42) over (order by rn) l\n"
+            + " from " + START_OF_GROUP_DATA)
         .typeIs(
-            "[RN INTEGER NOT NULL, VAL INTEGER NOT NULL, EXPECTED INTEGER NOT NULL, START_OF_GROUP INTEGER NOT NULL]")
+            "[RN INTEGER NOT NULL, VAL INTEGER NOT NULL, EXPECTED INTEGER NOT NULL, L INTEGER NOT NULL]")
         .returnsUnordered(
-            "RN=1; VAL=0; EXPECTED=1; START_OF_GROUP=1",
-            "RN=2; VAL=0; EXPECTED=1; START_OF_GROUP=0",
-            "RN=3; VAL=1; EXPECTED=2; START_OF_GROUP=1",
-            "RN=4; VAL=0; EXPECTED=3; START_OF_GROUP=1",
-            "RN=5; VAL=0; EXPECTED=3; START_OF_GROUP=0",
-            "RN=6; VAL=0; EXPECTED=3; START_OF_GROUP=0",
-            "RN=7; VAL=1; EXPECTED=4; START_OF_GROUP=1",
-            "RN=8; VAL=1; EXPECTED=4; START_OF_GROUP=0");
+            "RN=1; VAL=0; EXPECTED=1; L=42",
+            "RN=2; VAL=0; EXPECTED=1; L=2",
+            "RN=3; VAL=1; EXPECTED=2; L=3",
+            "RN=4; VAL=0; EXPECTED=3; L=5",
+            "RN=5; VAL=0; EXPECTED=3; L=7",
+            "RN=6; VAL=0; EXPECTED=3; L=8",
+            "RN=7; VAL=1; EXPECTED=4; L=9",
+            "RN=8; VAL=1; EXPECTED=4; L=11");
+  }
 
+  /**
+   * Tests default value of LEAD function.
+   */
+  @Test public void testLeadDefaultValue() {
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.REGULAR)
+        .query(
+            "select t.*, lead(rn+expected,1,42) over (order by rn) l\n"
+            + " from " + START_OF_GROUP_DATA)
+        .typeIs(
+            "[RN INTEGER NOT NULL, VAL INTEGER NOT NULL, EXPECTED INTEGER NOT NULL, L INTEGER NOT NULL]")
+        .returnsUnordered(
+            "RN=1; VAL=0; EXPECTED=1; L=3",
+            "RN=2; VAL=0; EXPECTED=1; L=5",
+            "RN=3; VAL=1; EXPECTED=2; L=7",
+            "RN=4; VAL=0; EXPECTED=3; L=8",
+            "RN=5; VAL=0; EXPECTED=3; L=9",
+            "RN=6; VAL=0; EXPECTED=3; L=11",
+            "RN=7; VAL=1; EXPECTED=4; L=12",
+            "RN=8; VAL=1; EXPECTED=4; L=42");
   }
 
   /**
-   * Tests start_of_group approach for grouping of adjacent intervals.
-   * This is a step2, that gets the final group numbers
-   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   * Tests expression in offset value of LAG function.
    */
-  @Ignore("LEAD/LAG is not implemented yet")
-  @Test public void testStartOfGroupLagValueStep2() {
+  @Test public void testLagExpressionOffset() {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .query(
-                "select t.*\n"
-                + "       ,sum(start_of_group) over (order by rn rows between unbounded preceding and current row) group_id\n"
-                + "  from (\n"
-                + "       select  t.*,\n"
-                + "               case when val = lag(val) over (order by rn) then 0 else 1 end start_of_group\n"
-                + "         from ("
-                + START_OF_GROUP_DATA
-                + ") t\n")
+            "select t.*, lag(rn, expected, 42) over (order by rn) l\n"
+            + " from " + START_OF_GROUP_DATA)
         .typeIs(
-            "[RN INTEGER NOT NULL, VAL INTEGER NOT NULL, EXPECTED INTEGER NOT NULL, START_OF_GROUP INTEGER NOT NULL, GROUP_ID INTEGER NOT NULL]")
+            "[RN INTEGER NOT NULL, VAL INTEGER NOT NULL, EXPECTED INTEGER NOT NULL, L INTEGER NOT NULL]")
         .returnsUnordered(
-            "RN=1; VAL=0; EXPECTED=1; START_OF_GROUP=1; GROUP_ID=1",
-            "RN=2; VAL=0; EXPECTED=1; START_OF_GROUP=0; GROUP_ID=1",
-            "RN=3; VAL=1; EXPECTED=2; START_OF_GROUP=1; GROUP_ID=2",
-            "RN=4; VAL=0; EXPECTED=3; START_OF_GROUP=1; GROUP_ID=3",
-            "RN=5; VAL=0; EXPECTED=3; START_OF_GROUP=0; GROUP_ID=3",
-            "RN=6; VAL=0; EXPECTED=3; START_OF_GROUP=0; GROUP_ID=3",
-            "RN=7; VAL=1; EXPECTED=4; START_OF_GROUP=1; GROUP_ID=4",
-            "RN=8; VAL=1; EXPECTED=4; START_OF_GROUP=0; GROUP_ID=4");
+            "RN=1; VAL=0; EXPECTED=1; L=42",
+            "RN=2; VAL=0; EXPECTED=1; L=1",
+            "RN=3; VAL=1; EXPECTED=2; L=1",
+            "RN=4; VAL=0; EXPECTED=3; L=1",
+            "RN=5; VAL=0; EXPECTED=3; L=2",
+            "RN=6; VAL=0; EXPECTED=3; L=3",
+            "RN=7; VAL=1; EXPECTED=4; L=3",
+            "RN=8; VAL=1; EXPECTED=4; L=4");
+  }
 
+  /**
+   * Tests DATE as offset argument of LAG function.
+   */
+  @Test public void testLagInvalidOffsetArgument() {
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.REGULAR)
+        .query(
+            "select t.*, lag(rn, DATE '2014-06-20', 42) over (order by rn) l\n"
+            + " from " + START_OF_GROUP_DATA)
+        .throws_(
+            "Cannot apply 'LAG' to arguments of type 'LAG(<INTEGER>, <DATE>, <INTEGER>)'");
   }
 
   /**
-   * Tests start_of_group approach for grouping of adjacent intervals.
-   * This is a step3, that aggregates the computed groups
-   * http://timurakhmadeev.wordpress.com/2013/07/21/start_of_group/
+   * Tests NTILE(2).
    */
-  @Ignore("LEAD/LAG is not implemented yet")
-  @Test public void testStartOfGroupLagStep3() {
+  @Test public void testNtile1() {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .query(
-            "select group_id, min(rn) min_rn, max(rn) max_rn, count(rn) cnt_rn, avg(val) avg_val"
-                + " from (\n"
-                + "select t.*\n"
-                // current row is assumed, group_id should be NOT NULL
-                + "       ,sum(start_of_group) over (order by rn rows unbounded preceding) group_id\n"
-                + "  from (\n"
-                + "       select  t.*,\n"
-                + "               case when val = lag(val) over (order by rn) then 0 else 1 end start_of_group\n"
-                + "         from ("
-                + START_OF_GROUP_DATA
-                + ") t\n"
-                + ") group by group_id\n")
+            "select rn, ntile(1) over (order by rn) l\n"
+                + " from " + START_OF_GROUP_DATA)
         .typeIs(
-            "[GROUP_ID INTEGER NOT NULL, MIN_RN INTEGER NOT NULL, MAX_RN INTEGER NOT NULL, CNT_RN BIGINT NOT NULL, AVG_VAL INTEGER NOT NULL]")
+            "[RN INTEGER NOT NULL, L INTEGER NOT NULL]")
         .returnsUnordered(
-            "GROUP_ID=1; MIN_RN=1; MAX_RN=2; CNT_RN=2; AVG_VAL=0",
-            "GROUP_ID=2; MIN_RN=3; MAX_RN=3; CNT_RN=1; AVG_VAL=1",
-            "GROUP_ID=3; MIN_RN=4; MAX_RN=6; CNT_RN=3; AVG_VAL=0",
-            "GROUP_ID=4; MIN_RN=7; MAX_RN=8; CNT_RN=2; AVG_VAL=1");
+            "RN=1; L=1",
+            "RN=2; L=1",
+            "RN=3; L=1",
+            "RN=4; L=1",
+            "RN=5; L=1",
+            "RN=6; L=1",
+            "RN=7; L=1",
+            "RN=8; L=1");
+  }
 
+  /**
+   * Tests NTILE(2).
+   */
+  @Test public void testNtile2() {
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.REGULAR)
+        .query(
+            "select rn, ntile(2) over (order by rn) l\n"
+                + " from " + START_OF_GROUP_DATA)
+        .typeIs(
+            "[RN INTEGER NOT NULL, L INTEGER NOT NULL]")
+        .returnsUnordered(
+            "RN=1; L=1",
+            "RN=2; L=1",
+            "RN=3; L=1",
+            "RN=4; L=1",
+            "RN=5; L=2",
+            "RN=6; L=2",
+            "RN=7; L=2",
+            "RN=8; L=2");
+  }
+
+  /**
+   * Tests expression in offset value of LAG function.
+   */
+  @Ignore("Have no idea how to validate that expression is constant")
+  @Test public void testNtileConstantArgs() {
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.REGULAR)
+        .query(
+            "select rn, ntile(1+1) over (order by rn) l\n"
+                + " from " + START_OF_GROUP_DATA)
+        .typeIs(
+            "[RN INTEGER NOT NULL, VAL INTEGER NOT NULL, EXPECTED INTEGER NOT NULL, L INTEGER NOT NULL]")
+        .returnsUnordered(
+            "RN=1; L=1",
+            "RN=2; L=1",
+            "RN=3; L=1",
+            "RN=4; L=1",
+            "RN=5; L=2",
+            "RN=6; L=2",
+            "RN=7; L=2",
+            "RN=8; L=2");
+  }
+
+  /**
+   * Tests expression in offset value of LAG function.
+   */
+  @Test public void testNtileNegativeArg() {
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.REGULAR)
+        .query(
+            "select rn, ntile(-1) over (order by rn) l\n"
+                + " from " + START_OF_GROUP_DATA)
+        .throws_(
+            "Argument to function 'NTILE' must be a positive integer literal");
+  }
+
+  /**
+   * Tests expression in offset value of LAG function.
+   */
+  @Test public void testNtileDecimalArg() {
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.REGULAR)
+        .query(
+            "select rn, ntile(3.141592653) over (order by rn) l\n"
+                + " from " + START_OF_GROUP_DATA)
+        .throws_(
+            "Cannot apply 'NTILE' to arguments of type 'NTILE(<DECIMAL(10, 9)>)'");
   }
 
   /** Tests for FIRST_VALUE */