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 2015/01/08 01:36:53 UTC

[1/2] incubator-calcite git commit: Fix code style

Repository: incubator-calcite
Updated Branches:
  refs/heads/master b822a0d0f -> f2df8e7c4


Fix code style


Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/01a8064c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/01a8064c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/01a8064c

Branch: refs/heads/master
Commit: 01a8064c59985781ca1b59c45cad2d84cce5c9d5
Parents: b822a0d
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 5 23:24:44 2015 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 7 13:43:13 2015 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/test/RexProgramTest.java    | 17 ++++++++++-------
 1 file changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/01a8064c/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index fe31ada..d275072 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -436,15 +436,18 @@ public class RexProgramTest {
     checkCnf(and(aRef, or(bRef, and(cRef, dRef))),
         "AND(?0.a, OR(?0.b, ?0.c), OR(?0.b, ?0.d))");
 
-    checkCnf(and(aRef, or(bRef,
-            and(cRef, or(dRef,
-                and(eRef, or(fRef, gRef)))))),
+    checkCnf(
+        and(aRef, or(bRef, and(cRef, or(dRef, and(eRef, or(fRef, gRef)))))),
         "AND(?0.a, OR(?0.b, ?0.c), OR(?0.b, ?0.d, ?0.e), OR(?0.b, ?0.d, ?0.f, ?0.g))");
 
-    checkCnf(and(aRef, or(bRef,
-            and(cRef, or(dRef,
-                and(eRef, or(fRef,
-                    and(gRef, or(trueLiteral, falseLiteral)))))))),
+    checkCnf(
+        and(aRef,
+            or(bRef,
+                and(cRef,
+                    or(dRef,
+                        and(eRef,
+                            or(fRef,
+                                and(gRef, or(trueLiteral, falseLiteral)))))))),
         "AND(?0.a, OR(?0.b, ?0.c), OR(?0.b, ?0.d, ?0.e), OR(?0.b, ?0.d, ?0.f, ?0.g))");
   }
 


[2/2] incubator-calcite git commit: [CALCITE-494] Support NEXT/CURRENT VALUE FOR syntax for using sequences

Posted by jh...@apache.org.
[CALCITE-494] Support NEXT/CURRENT VALUE FOR syntax for using sequences

The implementation includes SQL parsing, validation (assuming that a sequence is declared in the catalog as a table of type 'SEQUENCE' and with a single column), and a straw man implementation (using thread-local maps from sequence name to value).


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

Branch: refs/heads/master
Commit: f2df8e7c49012bfd15f1aa355c13d4f1f92632b4
Parents: 01a8064
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 5 23:26:26 2015 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 7 14:13:53 2015 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       | 21 ++++++-
 .../calcite/adapter/enumerable/RexImpTable.java | 10 ++++
 .../apache/calcite/runtime/SqlFunctions.java    | 37 ++++++++++++
 .../java/org/apache/calcite/schema/Schema.java  | 14 ++++-
 .../java/org/apache/calcite/sql/SqlKind.java    | 10 ++++
 .../sql/fun/SqlSequenceValueOperator.java       | 60 ++++++++++++++++++++
 .../calcite/sql/fun/SqlStdOperatorTable.java    | 22 +++----
 .../calcite/sql/validate/SqlValidatorImpl.java  |  7 ++-
 .../sql2rel/StandardConvertletTable.java        | 15 +++++
 .../org/apache/calcite/util/BuiltInMethod.java  |  3 +
 .../calcite/sql/parser/SqlParserTest.java       | 31 ++++++++++
 .../apache/calcite/sql/test/SqlAdvisorTest.java |  2 +
 .../java/org/apache/calcite/test/JdbcTest.java  | 30 ++++++++--
 core/src/test/resources/sql/sequence.oq         | 56 ++++++++++++++++++
 14 files changed, 294 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 3f9a608..cc9a555 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -51,6 +51,7 @@ import org.apache.calcite.sql.SqlDelete;
 import org.apache.calcite.sql.SqlDynamicParam;
 import org.apache.calcite.sql.SqlExplain;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlInsert;
@@ -2767,6 +2768,8 @@ SqlNode AtomicRowExpression() :
     e = NewSpecification() { return e; }
     |
     e = CaseExpression() { return e; }
+    |
+    e = SequenceExpression() { return e; }
 }
 
 SqlNode CaseExpression() :
@@ -2814,6 +2817,23 @@ SqlNode CaseExpression() :
     }
 }
 
+SqlCall SequenceExpression() :
+{
+    final SqlParserPos pos;
+    final SqlOperator f;
+    final SqlNode sequenceRef;
+}
+{
+    (
+        <NEXT> { f = SqlStdOperatorTable.NEXT_VALUE; pos = getPos(); }
+    |
+        <CURRENT> { f = SqlStdOperatorTable.CURRENT_VALUE; pos = getPos(); }
+    )
+    <VALUE> <FOR> sequenceRef = CompoundIdentifier() {
+        return f.createCall(pos, sequenceRef);
+    }
+}
+
 /**
  * Parses an expression for setting a option in SQL, such as QUOTED_IDENTIFIERS,
  * or explain plan level (physical/logical).
@@ -5191,7 +5211,6 @@ String CommonNonReservedKeyWord() :
         | <NAME>
         | <NAMES>
         | <NESTING>
-        | <NEXT>
         | <NORMALIZED>
         | <NULLABLE>
         | <NULLS>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index dba24af..8cc9e80 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -103,6 +103,7 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_ROLE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_TIME;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_TIMESTAMP;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_VALUE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DATETIME_PLUS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DENSE_RANK;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE;
@@ -140,6 +141,7 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MIN;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MOD;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTIPLY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NEXT_VALUE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_EQUALS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_LIKE;
@@ -281,6 +283,14 @@ public class RexImpTable {
     map.put(ARRAY_VALUE_CONSTRUCTOR, value);
     map.put(ITEM, new ItemImplementor());
 
+    // Sequences
+    defineImplementor(CURRENT_VALUE, NullPolicy.STRICT,
+        new MethodImplementor(BuiltInMethod.SEQUENCE_CURRENT_VALUE.method),
+        false);
+    defineImplementor(NEXT_VALUE, NullPolicy.STRICT,
+        new MethodImplementor(BuiltInMethod.SEQUENCE_NEXT_VALUE.method),
+        false);
+
     // System functions
     final SystemFunctionImplementor systemFunctionImplementor =
         new SystemFunctionImplementor();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 11b359b..fabdf09 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -33,9 +33,11 @@ import java.sql.SQLException;
 import java.sql.Timestamp;
 import java.text.DecimalFormat;
 import java.util.Date;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.TimeZone;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
 /**
@@ -65,6 +67,19 @@ public class SqlFunctions {
         }
       };
 
+  /** Holds, for each thread, a map from sequence name to sequence current
+   * value.
+   *
+   * <p>This is a straw man of an implementation whose main goal is to prove
+   * that sequences can be parsed, validated and planned. A real application
+   * will want persistent values for sequences, shared among threads. */
+  private static final ThreadLocal<Map<String, AtomicLong>> THREAD_SEQUENCES =
+      new ThreadLocal<Map<String, AtomicLong>>() {
+        @Override protected Map<String, AtomicLong> initialValue() {
+          return new HashMap<String, AtomicLong>();
+        }
+      };
+
   private SqlFunctions() {
   }
 
@@ -1190,6 +1205,28 @@ public class SqlFunctions {
     }
   }
 
+  /** Support the {@code CURRENT VALUE OF sequence} operator. */
+  @NonDeterministic
+  public static long sequenceCurrentValue(String key) {
+    return getAtomicLong(key).get();
+  }
+
+  /** Support the {@code NEXT VALUE OF sequence} operator. */
+  @NonDeterministic
+  public static long sequenceNextValue(String key) {
+    return getAtomicLong(key).incrementAndGet();
+  }
+
+  private static AtomicLong getAtomicLong(String key) {
+    final Map<String, AtomicLong> map = THREAD_SEQUENCES.get();
+    AtomicLong atomic = map.get(key);
+    if (atomic == null) {
+      atomic = new AtomicLong();
+      map.put(key, atomic);
+    }
+    return atomic;
+  }
+
   /** Support the SLICE function. */
   public static List slice(List list) {
     return list;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/schema/Schema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Schema.java b/core/src/main/java/org/apache/calcite/schema/Schema.java
index 2f374d8..ecda690 100644
--- a/core/src/main/java/org/apache/calcite/schema/Schema.java
+++ b/core/src/main/java/org/apache/calcite/schema/Schema.java
@@ -166,11 +166,21 @@ public interface Schema {
      * structure during query planning. */
     STAR,
 
-    /** Index table. (Used by Apache Phoenix.) */
+    /** Index table.
+     *
+     * <p>Used by Apache Phoenix. */
     INDEX,
 
-    /** Join table. (Used by Apache Phoenix.) */
+    /** Join table.
+     *
+     * <p>Used by Apache Phoenix. */
     JOIN,
+
+    /** Sequence table.
+     *
+     * <p>Used by Apache Phoenix, and others. Must have a single BIGINT column
+     * called "$seq". */
+    SEQUENCE,
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 4499055..8d37d4c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -483,6 +483,16 @@ public enum SqlKind {
   CAST,
 
   /**
+   * The "NEXT VALUE OF sequence" operator.
+   */
+  NEXT_VALUE,
+
+  /**
+   * The "CURRENT VALUE OF sequence" operator.
+   */
+  CURRENT_VALUE,
+
+  /**
    * The "TRIM" function.
    */
   TRIM,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java
new file mode 100644
index 0000000..73b224e
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java
@@ -0,0 +1,60 @@
+/*
+ * 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.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+/** Operator that returns the current or next value of a sequence. */
+public class SqlSequenceValueOperator extends SqlSpecialOperator {
+  /** Creates a SqlSequenceValueOperator. */
+  SqlSequenceValueOperator(SqlKind kind) {
+    super(kind.name(), kind, 100);
+    assert kind == SqlKind.NEXT_VALUE || kind == SqlKind.CURRENT_VALUE;
+  }
+
+  @Override public boolean isDeterministic() {
+    return false;
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
+      int rightPrec) {
+    writer.sep(kind == SqlKind.NEXT_VALUE
+        ? "NEXT VALUE FOR" : "CURRENT VALUE FOR");
+    call.getOperandList().get(0).unparse(writer, 0, 0);
+  }
+
+  @Override public RelDataType deriveType(SqlValidator validator,
+      SqlValidatorScope scope, SqlCall call) {
+    final RelDataTypeFactory typeFactory = validator.getTypeFactory();
+    return typeFactory.createTypeWithNullability(
+        typeFactory.createSqlType(SqlTypeName.BIGINT), false);
+  }
+
+  @Override public void validateCall(SqlCall call, SqlValidator validator,
+      SqlValidatorScope scope, SqlValidatorScope operandScope) {
+  }
+}
+
+// End SqlSequenceValueOperator.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index d988c55..9e5644d 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -1530,18 +1530,15 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   /**
    * The sequence next value function: <code>NEXT VALUE FOR sequence</code>
    */
-  public static final SqlFunction NEXT_VALUE =
-      new SqlFunction(
-          "NEXT_VALUE",
-          SqlKind.OTHER_FUNCTION,
-          ReturnTypes.BIGINT,
-          null,
-          OperandTypes.CHARACTER,
-          SqlFunctionCategory.SYSTEM) {
-        public boolean isDeterministic() {
-          return false;
-        }
-      };
+  public static final SqlOperator NEXT_VALUE =
+      new SqlSequenceValueOperator(SqlKind.NEXT_VALUE);
+
+  /**
+   * The sequence current value function: <code>CURRENT VALUE FOR
+   * sequence</code>
+   */
+  public static final SqlOperator CURRENT_VALUE =
+      new SqlSequenceValueOperator(SqlKind.CURRENT_VALUE);
 
   /**
    * The <code>TABLESAMPLE</code> operator.
@@ -1597,7 +1594,6 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
     }
     return instance;
   }
-
 }
 
 // End SqlStdOperatorTable.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 6e82f4c..adc459c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -4133,7 +4133,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       this.validator = validator;
     }
 
-    public SqlNode visit(SqlIdentifier id) {
+    @Override public SqlNode visit(SqlIdentifier id) {
       // First check for builtin functions which don't have
       // parentheses, like "LOCALTIME".
       SqlCall call =
@@ -4148,10 +4148,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       return fqId;
     }
 
-    // implement SqlScopedShuttle
-    protected SqlNode visitScoped(SqlCall call) {
+    @Override protected SqlNode visitScoped(SqlCall call) {
       switch (call.getKind()) {
       case SCALAR_QUERY:
+      case CURRENT_VALUE:
+      case NEXT_VALUE:
         return call;
       }
       // Only visits arguments which are expressions. We don't want to

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index 7ecfd37..850d798 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -59,6 +59,7 @@ import org.apache.calcite.sql.fun.SqlMultisetValueConstructor;
 import org.apache.calcite.sql.fun.SqlOverlapsOperator;
 import org.apache.calcite.sql.fun.SqlQuarterFunction;
 import org.apache.calcite.sql.fun.SqlRowOperator;
+import org.apache.calcite.sql.fun.SqlSequenceValueOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
@@ -679,6 +680,20 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     return cx.getRexBuilder().makeCall(returnType, fun, exprs);
   }
 
+  public RexNode convertSequenceValue(
+      SqlRexContext cx,
+      SqlSequenceValueOperator fun,
+      SqlCall call) {
+    final List<SqlNode> operands = call.getOperandList();
+    assert operands.size() == 1;
+    assert operands.get(0) instanceof SqlIdentifier;
+    String key = ((SqlIdentifier) operands.get(0)).names.toString();
+    RelDataType returnType =
+        cx.getValidator().getValidatedNodeType(call);
+    return cx.getRexBuilder().makeCall(returnType, fun,
+        ImmutableList.<RexNode>of(cx.getRexBuilder().makeLiteral(key)));
+  }
+
   public RexNode convertAggregateFunction(
       SqlRexContext cx,
       SqlAggFunction fun,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 5ef8a89..b03d0d4 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -274,6 +274,9 @@ public enum BuiltInMethod {
       Timestamp.class),
   TIMESTAMP_TO_LONG_OPTIONAL_OFFSET(SqlFunctions.class, "toLongOptional",
       Timestamp.class, TimeZone.class),
+  SEQUENCE_CURRENT_VALUE(SqlFunctions.class, "sequenceCurrentValue",
+      String.class),
+  SEQUENCE_NEXT_VALUE(SqlFunctions.class, "sequenceNextValue", String.class),
   SLICE(SqlFunctions.class, "slice", List.class),
   ELEMENT(SqlFunctions.class, "element", List.class),
   SELECTIVITY(Selectivity.class, "getSelectivity", RexNode.class),

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 1a66182..e419651 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -5717,6 +5717,37 @@ public class SqlParserTest {
         "(?s)Encountered \",\" at line 1, column 23\\..*");
   }
 
+  @Test public void testSequence() {
+    sql("select next value for my_schema.my_seq from t")
+        .ok("SELECT (NEXT VALUE FOR `MY_SCHEMA`.`MY_SEQ`)\n"
+                + "FROM `T`");
+    sql("select next value for my_schema.my_seq as s from t")
+        .ok("SELECT (NEXT VALUE FOR `MY_SCHEMA`.`MY_SEQ`) AS `S`\n"
+                + "FROM `T`");
+    sql("select next value for my_seq as s from t")
+        .ok("SELECT (NEXT VALUE FOR `MY_SEQ`) AS `S`\n"
+                + "FROM `T`");
+    sql("select 1 + next value for s + current value for s from t")
+        .ok("SELECT ((1 + (NEXT VALUE FOR `S`)) + (CURRENT VALUE FOR `S`))\n"
+                + "FROM `T`");
+    sql("select 1 from t where next value for my_seq < 10")
+        .ok("SELECT 1\n"
+                + "FROM `T`\n"
+                + "WHERE ((NEXT VALUE FOR `MY_SEQ`) < 10)");
+    sql("select 1 from t\n"
+        + "where next value for my_seq < 10 fetch next 3 rows only")
+        .ok("SELECT 1\n"
+                + "FROM `T`\n"
+                + "WHERE ((NEXT VALUE FOR `MY_SEQ`) < 10)\n"
+                + "FETCH NEXT 3 ROWS ONLY");
+    sql("insert into t values next value for my_seq, current value for my_seq")
+        .ok("INSERT INTO `T`\n"
+                + "(VALUES (ROW((NEXT VALUE FOR `MY_SEQ`))), (ROW((CURRENT VALUE FOR `MY_SEQ`))))");
+    sql("insert into t values (1, current value for my_seq)")
+        .ok("INSERT INTO `T`\n"
+                + "(VALUES (ROW(1, (CURRENT VALUE FOR `MY_SEQ`))))");
+  }
+
   //~ Inner Interfaces -------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index 65e24a6..14aec93 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -115,6 +115,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(COVAR_POP)",
           "KEYWORD(COVAR_SAMP)",
           "KEYWORD(CUME_DIST)",
+          "KEYWORD(CURRENT)",
           "KEYWORD(CURRENT_CATALOG)",
           "KEYWORD(CURRENT_DATE)",
           "KEYWORD(CURRENT_DEFAULT_TRANSFORM_GROUP)",
@@ -147,6 +148,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(MOD)",
           "KEYWORD(MULTISET)",
           "KEYWORD(NEW)",
+          "KEYWORD(NEXT)",
           "KEYWORD(NOT)",
           "KEYWORD(NULL)",
           "KEYWORD(NULLIF)",

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index ebaceec..add91fa 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -4321,6 +4321,7 @@ public class JdbcTest {
 
   /** Runs the dummy script, which is checked in empty but which you may
    * use as scratch space during development. */
+  // Do not add '@Ignore'; just remember not to commit changes to dummy.oq
   @Test public void testRunDummy() throws Exception {
     checkRun("sql/dummy.oq");
   }
@@ -4337,11 +4338,6 @@ public class JdbcTest {
     checkRun("sql/outer.oq");
   }
 
-  @Ignore
-  @Test public void testRunFoo() throws Exception {
-    checkRun("/tmp/foo.oq");
-  }
-
   @Test public void testRunWinAgg() throws Exception {
     checkRun("sql/winagg.oq");
   }
@@ -4350,6 +4346,10 @@ public class JdbcTest {
     checkRun("sql/misc.oq");
   }
 
+  @Test public void testRunSequence() throws Exception {
+    checkRun("sql/sequence.oq");
+  }
+
   @Test public void testRunSubquery() throws Exception {
     checkRun("sql/subquery.oq");
   }
@@ -4404,6 +4404,26 @@ public class JdbcTest {
                           new ReflectiveSchemaTest.CatchallSchema()))
                   .connect();
             }
+            if (name.equals("seq")) {
+              final Connection connection = CalciteAssert.that()
+                  .withSchema("s", new AbstractSchema())
+                  .connect();
+              connection.unwrap(CalciteConnection.class).getRootSchema()
+                  .getSubSchema("s")
+                  .add("my_seq",
+                      new AbstractTable() {
+                        public RelDataType getRowType(
+                            RelDataTypeFactory typeFactory) {
+                          return typeFactory.builder()
+                              .add("$seq", SqlTypeName.BIGINT).build();
+                        }
+
+                        @Override public Schema.TableType getJdbcTableType() {
+                          return Schema.TableType.SEQUENCE;
+                        }
+                      });
+              return connection;
+            }
             throw new RuntimeException("unknown connection '" + name + "'");
           }
         });

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/f2df8e7c/core/src/test/resources/sql/sequence.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/sequence.oq b/core/src/test/resources/sql/sequence.oq
new file mode 100644
index 0000000..cf18122
--- /dev/null
+++ b/core/src/test/resources/sql/sequence.oq
@@ -0,0 +1,56 @@
+# sequence.oq - Sequences
+#
+# 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.
+#
+
+!use seq
+!set outputformat mysql
+
+select next value for my_seq as c from (values 1, 2);
++---+
+| C |
++---+
+| 1 |
+| 2 |
++---+
+(2 rows)
+
+!ok
+select current value for my_seq as c from (values 1, 2);
++---+
+| C |
++---+
+| 2 |
+| 2 |
++---+
+(2 rows)
+
+!ok
+
+# Sequences appear in the catalog as tables of type 'SEQUENCE'
+select * from "metadata".tables;
++----------+------------+-----------+--------------+---------+---------+-----------+----------+------------------------+---------------+
+| tableCat | tableSchem | tableName | tableType    | remarks | typeCat | typeSchem | typeName | selfReferencingColName | refGeneration |
++----------+------------+-----------+--------------+---------+---------+-----------+----------+------------------------+---------------+
+|          | metadata   | COLUMNS   | SYSTEM_TABLE |         |         |           |          |                        |               |
+|          | metadata   | TABLES    | SYSTEM_TABLE |         |         |           |          |                        |               |
+|          | s          | my_seq    | SEQUENCE     |         |         |           |          |                        |               |
++----------+------------+-----------+--------------+---------+---------+-----------+----------+------------------------+---------------+
+(3 rows)
+
+!ok
+
+# End sequence.oq