You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by dz...@apache.org on 2023/04/18 15:26:37 UTC

[drill] 09/15: DRILL-8410: Upgrade to Calcite 1.34 (#2775)

This is an automated email from the ASF dual-hosted git repository.

dzamo pushed a commit to branch 1.21
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 3efe718b958dbcfe2e27a6425d25f98a772425cc
Author: James Turton <91...@users.noreply.github.com>
AuthorDate: Thu Mar 16 14:23:46 2023 +0200

    DRILL-8410: Upgrade to Calcite 1.34 (#2775)
    
    * Upgade to Calcite 1.34.
    * Support the QUALIFY clause introduced in Calcite 1.34.
    * Add the Calcite core parser grammar to the Drill source to exclude CALCITE-5469.
    * Upgrade Apache parent pom to 29.
---
 exec/java-exec/pom.xml                             |   67 +-
 .../java-exec/src/main/codegen/templates/Parser.jj | 8758 ++++++++++++++++++++
 .../drill/exec/planner/sql/SqlSelectBuilder.java   |    8 +-
 .../sql/parser/CompoundIdentifierConverter.java    |    2 +-
 .../org/apache/drill/exec/TestWindowFunctions.java |   41 +
 pom.xml                                            |    8 +-
 6 files changed, 8854 insertions(+), 30 deletions(-)

diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 0836af4fdb..69d8210901 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -818,6 +818,51 @@
         </dependency>
       </dependencies>
     </profile>
+    <profile>
+      <!-- If Drill has not provided its own parser grammar at the location
+      below then we default to Calcite's. You'd have hoped that a simple
+      overWrite=false setting in the relevant maven-dependency-plugin unpack
+      goal in this pom would have been sufficient but alas this whole new
+      profile is required. -->
+      <id>calcite-parser-grammar</id>
+      <activation>
+        <file>
+          <missing>${basedir}/src/main/codegen/templates/Parser.jj</missing>
+        </file>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <executions>
+             <!-- Extract parser grammar template from calcite-core.jar and put
+          it under ${project.build.directory}/codegen/templates where all freemarker templates are. -->
+              <execution>
+                <id>unpack-parser-template</id>
+                <phase>initialize</phase>
+                <goals>
+                  <goal>unpack</goal>
+                </goals>
+                <configuration>
+                  <overWrite>false</overWrite>
+                  <artifactItems>
+                    <artifactItem>
+                      <groupId>${calcite.groupId}</groupId>
+                      <artifactId>calcite-core</artifactId>
+                      <type>jar</type>
+                      <overWrite>false</overWrite>
+                      <outputDirectory>${project.build.directory}/</outputDirectory>
+                      <includes>codegen/templates/Parser.jj</includes>
+                    </artifactItem>
+                  </artifactItems>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
   <build>
     <plugins>
@@ -849,28 +894,6 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
         <executions>
-        <!-- Extract parser grammar template from calcite-core.jar and put
-          it under ${project.build.directory}/codegen/templates where all freemarker templates are. -->
-          <execution>
-            <id>unpack-parser-template</id>
-            <phase>initialize</phase>
-            <goals>
-              <goal>unpack</goal>
-            </goals>
-            <configuration>
-              <artifactItems>
-                <artifactItem>
-                  <groupId>${calcite.groupId}</groupId>
-                  <artifactId>calcite-core</artifactId>
-                  <type>jar</type>
-                  <overWrite>true</overWrite>
-                  <outputDirectory>${project.build.directory}/</outputDirectory>
-                  <includes>codegen/templates/Parser.jj</includes>
-                </artifactItem>
-              </artifactItems>
-            </configuration>
-          </execution>
-
         <!-- Extract ValueVectorTypes.tdd from drill-vector.jar and put
           it under ${project.build.directory}/codegen/data where all freemarker data files are. -->
           <execution>
diff --git a/exec/java-exec/src/main/codegen/templates/Parser.jj b/exec/java-exec/src/main/codegen/templates/Parser.jj
new file mode 100644
index 0000000000..b5d9f96cf7
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/Parser.jj
@@ -0,0 +1,8758 @@
+/*
+ * 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.
+ */
+
+// TODO: Delete this file to reinstate its extraction from calcite-core.jar
+// once CALCITE-5579 is resolved and the incompatible grammar changes introduced
+// by CALCITE-5469 have been backed out. Also see: exec/java-exec/pom.xml.
+
+<@pp.dropOutputFile />
+
+<@pp.changeOutputFile name="javacc/Parser.jj" />
+
+options {
+    STATIC = false;
+    IGNORE_CASE = true;
+    UNICODE_INPUT = true;
+}
+
+
+PARSER_BEGIN(${parser.class})
+
+package ${parser.package};
+
+<#list (parser.imports!default.parser.imports) as importStr>
+import ${importStr};
+</#list>
+
+import org.apache.calcite.avatica.util.Casing;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlAlter;
+import org.apache.calcite.sql.SqlBasicTypeNameSpec;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlCollectionTypeNameSpec;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDelete;
+import org.apache.calcite.sql.SqlDescribeSchema;
+import org.apache.calcite.sql.SqlDescribeTable;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlExplain;
+import org.apache.calcite.sql.SqlExplainFormat;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlHint;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlInsertKeyword;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlJdbcDataTypeName;
+import org.apache.calcite.sql.SqlJdbcFunctionCall;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlJsonConstructorNullClause;
+import org.apache.calcite.sql.SqlJsonEncoding;
+import org.apache.calcite.sql.SqlJsonExistsErrorBehavior;
+import org.apache.calcite.sql.SqlJsonEmptyOrError;
+import org.apache.calcite.sql.SqlJsonQueryEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlJsonQueryWrapperBehavior;
+import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlJsonValueReturning;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.apache.calcite.sql.SqlMerge;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOrderBy;
+import org.apache.calcite.sql.SqlPivot;
+import org.apache.calcite.sql.SqlPostfixOperator;
+import org.apache.calcite.sql.SqlPrefixOperator;
+import org.apache.calcite.sql.SqlRowTypeNameSpec;
+import org.apache.calcite.sql.SqlSampleSpec;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSelectKeyword;
+import org.apache.calcite.sql.SqlSetOption;
+import org.apache.calcite.sql.SqlSnapshot;
+import org.apache.calcite.sql.SqlTableRef;
+import org.apache.calcite.sql.SqlTypeNameSpec;
+import org.apache.calcite.sql.SqlUnnestOperator;
+import org.apache.calcite.sql.SqlUnpivot;
+import org.apache.calcite.sql.SqlUpdate;
+import org.apache.calcite.sql.SqlUserDefinedTypeNameSpec;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlInternalOperators;
+import org.apache.calcite.sql.fun.SqlLibraryOperators;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.parser.Span;
+import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.util.Glossary;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.SourceStringReader;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * SQL parser, generated from Parser.jj by JavaCC.
+ *
+ * <p>The public wrapper for this parser is {@link SqlParser}.
+ */
+public class ${parser.class} extends SqlAbstractParserImpl
+{
+    private static final Logger LOGGER = CalciteTrace.getParserTracer();
+
+    // Can't use quoted literal because of a bug in how JavaCC translates
+    // backslash-backslash.
+    private static final char BACKSLASH = 0x5c;
+    private static final char DOUBLE_QUOTE = 0x22;
+    private static final String DQ = DOUBLE_QUOTE + "";
+    private static final String DQDQ = DQ + DQ;
+    private static final SqlLiteral LITERAL_ZERO =
+        SqlLiteral.createExactNumeric("0", SqlParserPos.ZERO);
+    private static final SqlLiteral LITERAL_ONE =
+        SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO);
+    private static final SqlLiteral LITERAL_MINUS_ONE =
+        SqlLiteral.createExactNumeric("-1", SqlParserPos.ZERO);
+
+    private static Metadata metadata;
+
+    private Casing unquotedCasing;
+    private Casing quotedCasing;
+    private int identifierMaxLength;
+    private SqlConformance conformance;
+
+    /**
+     * {@link SqlParserImplFactory} implementation for creating parser.
+     */
+    public static final SqlParserImplFactory FACTORY = new SqlParserImplFactory() {
+        public SqlAbstractParserImpl getParser(Reader reader) {
+            final ${parser.class} parser = new ${parser.class}(reader);
+            if (reader instanceof SourceStringReader) {
+                final String sql =
+                    ((SourceStringReader) reader).getSourceString();
+                parser.setOriginalSql(sql);
+            }
+          return parser;
+        }
+    };
+
+    public SqlParseException normalizeException(Throwable ex) {
+        try {
+            if (ex instanceof ParseException) {
+                ex = cleanupParseException((ParseException) ex);
+            }
+            return convertException(ex);
+        } catch (ParseException e) {
+            throw new AssertionError(e);
+        }
+    }
+
+    public Metadata getMetadata() {
+        synchronized (${parser.class}.class) {
+            if (metadata == null) {
+                metadata = new MetadataImpl(
+                    new ${parser.class}(new java.io.StringReader("")));
+            }
+            return metadata;
+        }
+    }
+
+    public void setTabSize(int tabSize) {
+        jj_input_stream.setTabSize(tabSize);
+    }
+
+    public void switchTo(SqlAbstractParserImpl.LexicalState state) {
+        final int stateOrdinal =
+            Arrays.asList(${parser.class}TokenManager.lexStateNames)
+                .indexOf(state.name());
+        token_source.SwitchTo(stateOrdinal);
+    }
+
+    public void setQuotedCasing(Casing quotedCasing) {
+        this.quotedCasing = quotedCasing;
+    }
+
+    public void setUnquotedCasing(Casing unquotedCasing) {
+        this.unquotedCasing = unquotedCasing;
+    }
+
+    public void setIdentifierMaxLength(int identifierMaxLength) {
+        this.identifierMaxLength = identifierMaxLength;
+    }
+
+    public void setConformance(SqlConformance conformance) {
+        this.conformance = conformance;
+    }
+
+    public SqlNode parseSqlExpressionEof() throws Exception {
+        return SqlExpressionEof();
+    }
+
+    public SqlNode parseSqlStmtEof() throws Exception {
+        return SqlStmtEof();
+    }
+
+    public SqlNodeList parseSqlStmtList() throws Exception {
+        return SqlStmtList();
+    }
+
+    public SqlNode parseArray() throws SqlParseException {
+        switchTo(LexicalState.BQID);
+        try {
+          return ArrayLiteral();
+        } catch (ParseException ex) {
+          throw normalizeException(ex);
+        } catch (TokenMgrError ex) {
+          throw normalizeException(ex);
+        }
+    }
+
+    private SqlNode extend(SqlNode table, SqlNodeList extendList) {
+        return SqlStdOperatorTable.EXTEND.createCall(
+            Span.of(table, extendList).pos(), table, extendList);
+    }
+
+    /** Adds a warning that a token such as "HOURS" was used,
+    * whereas the SQL standard only allows "HOUR".
+    *
+    * <p>Currently, we silently add an exception to a list of warnings. In
+    * future, we may have better compliance checking, for example a strict
+    * compliance mode that throws if any non-standard features are used. */
+    private TimeUnit warn(TimeUnit timeUnit) throws ParseException {
+        final String token = getToken(0).image.toUpperCase(Locale.ROOT);
+        warnings.add(
+            SqlUtil.newContextException(getPos(),
+                RESOURCE.nonStandardFeatureUsed(token)));
+        return timeUnit;
+    }
+}
+
+PARSER_END(${parser.class})
+
+
+/***************************************
+ * Utility Codes for Semantic Analysis *
+ ***************************************/
+
+/* For Debug */
+JAVACODE
+void debug_message1() {
+    LOGGER.info("{} , {}", getToken(0).image, getToken(1).image);
+}
+
+JAVACODE String unquotedIdentifier() {
+    return SqlParserUtil.toCase(getToken(0).image, unquotedCasing);
+}
+
+/**
+ * Allows parser to be extended with new types of table references.  The
+ * default implementation of this production is empty.
+ */
+SqlNode ExtendedTableRef() :
+{
+}
+{
+    UnusedExtension()
+    {
+        return null;
+    }
+}
+
+/**
+ * Allows an OVER clause following a table expression as an extension to
+ * standard SQL syntax. The default implementation of this production is empty.
+ */
+SqlNode TableOverOpt() :
+{
+}
+{
+    {
+        return null;
+    }
+}
+
+/*
+ * Parses dialect-specific keywords immediately following the SELECT keyword.
+ */
+void SqlSelectKeywords(List<SqlLiteral> keywords) :
+{}
+{
+    E()
+}
+
+/*
+ * Parses dialect-specific keywords immediately following the INSERT keyword.
+ */
+void SqlInsertKeywords(List<SqlLiteral> keywords) :
+{}
+{
+    E()
+}
+
+/*
+* Parse Floor/Ceil function parameters
+*/
+SqlNode FloorCeilOptions(Span s, boolean floorFlag) :
+{
+    SqlNode node;
+}
+{
+    node = StandardFloorCeilOptions(s, floorFlag) {
+        return node;
+    }
+}
+
+/*
+// This file contains the heart of a parser for SQL SELECT statements.
+// code can be shared between various parsers (for example, a DDL parser and a
+// DML parser) but is not a standalone JavaCC file. You need to prepend a
+// parser declaration (such as that in Parser.jj).
+*/
+
+/* Epsilon */
+JAVACODE
+void E() {}
+
+/** @Deprecated */
+JAVACODE List startList(Object o)
+{
+    List list = new ArrayList();
+    list.add(o);
+    return list;
+}
+
+/*
+ * NOTE jvs 6-Feb-2004: The straightforward way to implement the SQL grammar is
+ * to keep query expressions (SELECT, UNION, etc) separate from row expressions
+ * (+, LIKE, etc).  However, this is not possible with an LL(k) parser, because
+ * both kinds of expressions allow parenthesization, so no fixed amount of left
+ * context is ever good enough.  A sub-query can be a leaf in a row expression,
+ * and can include operators like UNION, so it's not even possible to use a
+ * syntactic lookahead rule like "look past an indefinite number of parentheses
+ * until you see SELECT, VALUES, or TABLE" (since at that point we still
+ * don't know whether we're parsing a sub-query like ((select ...) + x)
+ * vs. (select ... union select ...).
+ *
+ * The somewhat messy solution is to unify the two kinds of expression,
+ * and to enforce syntax rules using parameterized context.  This
+ * is the purpose of the ExprContext parameter.  It is passed to
+ * most expression productions, which check the expressions encountered
+ * against the context for correctness.  When a query
+ * element like SELECT is encountered, the production calls
+ * checkQueryExpression, which will throw an exception if
+ * a row expression was expected instead.  When a row expression like
+ * IN is encountered, the production calls checkNonQueryExpression
+ * instead.  It is very important to understand how this works
+ * when modifying the grammar.
+ *
+ * The commingling of expressions results in some bogus ambiguities which are
+ * resolved with LOOKAHEAD hints.  The worst example is comma.  SQL allows both
+ * (WHERE x IN (1,2)) and (WHERE x IN (select ...)).  This means when we parse
+ * the right-hand-side of an IN, we have to allow any kind of expression inside
+ * the parentheses.  Now consider the expression "WHERE x IN(SELECT a FROM b
+ * GROUP BY c,d)".  When the parser gets to "c,d" it doesn't know whether the
+ * comma indicates the end of the GROUP BY or the end of one item in an IN
+ * list.  Luckily, we know that select and comma-list are mutually exclusive
+ * within IN, so we use maximal munch for the GROUP BY comma.  However, this
+ * usage of hints could easily mask unintended ambiguities resulting from
+ * future changes to the grammar, making it very brittle.
+ */
+
+JAVACODE protected SqlParserPos getPos()
+{
+    return new SqlParserPos(
+        token.beginLine,
+        token.beginColumn,
+        token.endLine,
+        token.endColumn);
+}
+
+/** Starts a span at the current position. */
+JAVACODE Span span()
+{
+    return Span.of(getPos());
+}
+
+JAVACODE void checkQueryExpression(ExprContext exprContext)
+{
+    switch (exprContext) {
+    case ACCEPT_NON_QUERY:
+    case ACCEPT_SUB_QUERY:
+    case ACCEPT_CURSOR:
+        throw SqlUtil.newContextException(getPos(),
+            RESOURCE.illegalQueryExpression());
+    }
+}
+
+JAVACODE void checkNonQueryExpression(ExprContext exprContext)
+{
+    switch (exprContext) {
+    case ACCEPT_QUERY:
+        throw SqlUtil.newContextException(getPos(),
+            RESOURCE.illegalNonQueryExpression());
+    }
+}
+
+JAVACODE SqlNode checkNotJoin(SqlNode e)
+{
+    if (e instanceof SqlJoin) {
+        throw SqlUtil.newContextException(e.getParserPosition(),
+            RESOURCE.illegalJoinExpression());
+    }
+    return e;
+}
+
+/**
+ * Converts a ParseException (local to this particular instantiation
+ * of the parser) into a SqlParseException (common to all parsers).
+ */
+JAVACODE SqlParseException convertException(Throwable ex)
+{
+    if (ex instanceof SqlParseException) {
+        return (SqlParseException) ex;
+    }
+    SqlParserPos pos = null;
+    int[][] expectedTokenSequences = null;
+    String[] tokenImage = null;
+    if (ex instanceof ParseException) {
+        ParseException pex = (ParseException) ex;
+        expectedTokenSequences = pex.expectedTokenSequences;
+        tokenImage = pex.tokenImage;
+        if (pex.currentToken != null) {
+            final Token token = pex.currentToken.next;
+            // Checks token.image.equals("1") to avoid recursive call.
+            // The SqlAbstractParserImpl#MetadataImpl constructor uses constant "1" to
+            // throw intentionally to collect the expected tokens.
+            if (!token.image.equals("1")
+                && getMetadata().isKeyword(token.image)
+                && SqlParserUtil.allowsIdentifier(tokenImage, expectedTokenSequences)) {
+                // If the next token is a keyword, reformat the error message as:
+
+                // Incorrect syntax near the keyword '{keyword}' at line {line_number},
+                // column {column_number}.
+                final String expecting = ex.getMessage()
+                    .substring(ex.getMessage().indexOf("Was expecting"));
+                final String errorMsg = String.format("Incorrect syntax near the keyword '%s' "
+                        + "at line %d, column %d.\n%s",
+                    token.image,
+                    token.beginLine,
+                    token.beginColumn,
+                    expecting);
+                // Replace the ParseException with explicit error message.
+                ex = new ParseException(errorMsg);
+            }
+            pos = new SqlParserPos(
+                token.beginLine,
+                token.beginColumn,
+                token.endLine,
+                token.endColumn);
+        }
+    } else if (ex instanceof TokenMgrError) {
+        expectedTokenSequences = null;
+        tokenImage = null;
+        // Example:
+        //    Lexical error at line 3, column 24.  Encountered "#" after "a".
+        final java.util.regex.Pattern pattern = java.util.regex.Pattern.compile(
+            "(?s)Lexical error at line ([0-9]+), column ([0-9]+).*");
+        java.util.regex.Matcher matcher = pattern.matcher(ex.getMessage());
+        if (matcher.matches()) {
+            int line = Integer.parseInt(matcher.group(1));
+            int column = Integer.parseInt(matcher.group(2));
+            pos = new SqlParserPos(line, column, line, column);
+        }
+    } else if (ex instanceof CalciteContextException) {
+        // CalciteContextException is the standard wrapper for exceptions
+        // produced by the validator, but in the parser, the standard is
+        // SqlParseException; so, strip it away. In case you were wondering,
+        // the CalciteContextException appears because the parser
+        // occasionally calls into validator-style code such as
+        // SqlSpecialOperator.reduceExpr.
+        CalciteContextException ece =
+            (CalciteContextException) ex;
+        pos = new SqlParserPos(
+            ece.getPosLine(),
+            ece.getPosColumn(),
+            ece.getEndPosLine(),
+            ece.getEndPosColumn());
+        ex = ece.getCause();
+    }
+
+    return new SqlParseException(
+        ex.getMessage(), pos, expectedTokenSequences, tokenImage, ex);
+}
+
+/**
+ * Removes or transforms misleading information from a parse exception.
+ *
+ * @param e dirty excn
+ *
+ * @return clean excn
+ */
+JAVACODE ParseException cleanupParseException(ParseException ex)
+{
+    if (ex.expectedTokenSequences == null) {
+        return ex;
+    }
+    int iIdentifier = Arrays.asList(ex.tokenImage).indexOf("<IDENTIFIER>");
+
+    // Find all sequences in the error which contain identifier. For
+    // example,
+    //       {<IDENTIFIER>}
+    //       {A}
+    //       {B, C}
+    //       {D, <IDENTIFIER>}
+    //       {D, A}
+    //       {D, B}
+    //
+    // would yield
+    //       {}
+    //       {D}
+    final List<int[]> prefixList = new ArrayList<int[]>();
+    for (int i = 0; i < ex.expectedTokenSequences.length; ++i) {
+        int[] seq = ex.expectedTokenSequences[i];
+        int j = seq.length - 1;
+        int i1 = seq[j];
+        if (i1 == iIdentifier) {
+            int[] prefix = new int[j];
+            System.arraycopy(seq, 0, prefix, 0, j);
+            prefixList.add(prefix);
+        }
+    }
+
+    if (prefixList.isEmpty()) {
+        return ex;
+    }
+
+    int[][] prefixes = (int[][])
+        prefixList.toArray(new int[prefixList.size()][]);
+
+    // Since <IDENTIFIER> was one of the possible productions,
+    // we know that the parser will also have included all
+    // of the non-reserved keywords (which are treated as
+    // identifiers in non-keyword contexts).  So, now we need
+    // to clean those out, since they're totally irrelevant.
+
+    final List<int[]> list = new ArrayList<int[]>();
+    Metadata metadata = getMetadata();
+    for (int i = 0; i < ex.expectedTokenSequences.length; ++i) {
+        int [] seq = ex.expectedTokenSequences[i];
+        String tokenImage = ex.tokenImage[seq[seq.length - 1]];
+        String token = SqlParserUtil.getTokenVal(tokenImage);
+        if (token == null  || !metadata.isNonReservedKeyword(token)) {
+            list.add(seq);
+            continue;
+        }
+        boolean match = matchesPrefix(seq, prefixes);
+        if (!match) {
+            list.add(seq);
+        }
+    }
+
+    ex.expectedTokenSequences =
+        (int [][]) list.toArray(new int [list.size()][]);
+    return ex;
+}
+
+JAVACODE boolean matchesPrefix(int[] seq, int[][] prefixes)
+{
+    nextPrefix:
+    for (int[] prefix : prefixes) {
+        if (seq.length == prefix.length + 1) {
+            for (int k = 0; k < prefix.length; k++) {
+                if (prefix[k] != seq[k]) {
+                    continue nextPrefix;
+                }
+            }
+            return true;
+        }
+    }
+    return false;
+}
+
+/*****************************************
+ * Syntactical Descriptions              *
+ *****************************************/
+
+SqlNode ExprOrJoinOrOrderedQuery(ExprContext exprContext) :
+{
+    SqlNode e;
+    final List<Object> list = new ArrayList<Object>();
+}
+{
+    // Lookhead to distinguish between "TABLE emp" (which will be
+    // matched by ExplicitTable() via Query())
+    // and "TABLE fun(args)" (which will be matched by TableRef())
+    (
+        LOOKAHEAD(2)
+        e = Query(exprContext)
+        e = OrderByLimitOpt(e)
+        { return e; }
+    |
+        e = TableRef1(ExprContext.ACCEPT_QUERY_OR_JOIN)
+        ( e = JoinTable(e) )*
+        { list.add(e); }
+        ( AddSetOpQuery(list, exprContext) )*
+        { return SqlParserUtil.toTree(list); }
+    )
+}
+
+/**
+ * Parses either a row expression or a query expression with an optional
+ * ORDER BY.
+ *
+ * <p>Postgres syntax for limit:
+ *
+ * <blockquote><pre>
+ *    [ LIMIT { count | ALL } ]
+ *    [ OFFSET start ]</pre>
+ * </blockquote>
+ *
+ * <p>Trino syntax for limit:
+ *
+ * <blockquote><pre>
+ *    [ OFFSET start ]
+ *    [ LIMIT { count | ALL } ]</pre>
+ * </blockquote>
+ *
+ * <p>MySQL syntax for limit:
+ *
+ * <blockquote><pre>
+ *    [ LIMIT { count | start, count } ]</pre>
+ * </blockquote>
+ *
+ * <p>SQL:2008 syntax for limit:
+ *
+ * <blockquote><pre>
+ *    [ OFFSET start { ROW | ROWS } ]
+ *    [ FETCH { FIRST | NEXT } [ count ] { ROW | ROWS } ONLY ]</pre>
+ * </blockquote>
+ */
+SqlNode OrderedQueryOrExpr(ExprContext exprContext) :
+{
+    SqlNode e;
+}
+{
+    e = QueryOrExpr(exprContext)
+    e = OrderByLimitOpt(e)
+    { return e; }
+}
+
+/** Reads optional "ORDER BY", "LIMIT", "OFFSET", "FETCH" following a query,
+ * {@code e}. If any of them are present, adds them to the query;
+ * otherwise returns the query unchanged.
+ * Throws if they are present and {@code e} is not a query. */
+SqlNode OrderByLimitOpt(SqlNode e) :
+{
+    final SqlNodeList orderBy;
+    final Span s = Span.of();
+    SqlNode[] offsetFetch = {null, null};
+}
+{
+    (
+        // use the syntactic type of the expression we just parsed
+        // to decide whether ORDER BY makes sense
+        orderBy = OrderBy(e.isA(SqlKind.QUERY))
+    |   { orderBy = null; }
+    )
+    [
+        LimitClause(s, offsetFetch)
+        [ OffsetClause(s, offsetFetch) ]
+    |
+        OffsetClause(s, offsetFetch)
+        [
+            LimitClause(s, offsetFetch) {
+                if (!this.conformance.isOffsetLimitAllowed()) {
+                    throw SqlUtil.newContextException(s.end(this),
+                        RESOURCE.offsetLimitNotAllowed());
+                }
+            }
+        |
+            FetchClause(offsetFetch)
+        ]
+    |
+        FetchClause(offsetFetch)
+    ]
+    {
+        if (orderBy != null || offsetFetch[0] != null || offsetFetch[1] != null) {
+            return new SqlOrderBy(getPos(), e,
+                Util.first(orderBy, SqlNodeList.EMPTY),
+                offsetFetch[0], offsetFetch[1]);
+        }
+        return e;
+    }
+}
+
+/**
+ * Parses an OFFSET clause in an ORDER BY expression.
+ */
+void OffsetClause(Span s, SqlNode[] offsetFetch) :
+{
+}
+{
+    // ROW or ROWS is required in SQL:2008 but we make it optional
+    // because it is not present in Postgres-style syntax.
+    <OFFSET> { s.add(this); }
+    offsetFetch[0] = UnsignedNumericLiteralOrParam()
+    [ <ROW> | <ROWS> ]
+}
+
+/**
+ * Parses a FETCH clause in an ORDER BY expression.
+ */
+void FetchClause(SqlNode[] offsetFetch) :
+{
+}
+{
+    // SQL:2008-style syntax. "OFFSET ... FETCH ...".
+    // If you specify both LIMIT and FETCH, FETCH wins.
+    <FETCH> ( <FIRST> | <NEXT> ) offsetFetch[1] = UnsignedNumericLiteralOrParam()
+    ( <ROW> | <ROWS> ) <ONLY>
+}
+
+/**
+ * Parses a LIMIT clause in an ORDER BY expression.
+ */
+void LimitClause(Span s, SqlNode[] offsetFetch) :
+{
+}
+{
+    // Postgres-style syntax. "LIMIT ... OFFSET ..."
+    <LIMIT> { s.add(this); }
+    (
+        // MySQL-style syntax. "LIMIT start, count"
+        LOOKAHEAD(2)
+        offsetFetch[0] = UnsignedNumericLiteralOrParam()
+        <COMMA> offsetFetch[1] = UnsignedNumericLiteralOrParam() {
+            if (!this.conformance.isLimitStartCountAllowed()) {
+                throw SqlUtil.newContextException(s.end(this),
+                    RESOURCE.limitStartCountNotAllowed());
+            }
+        }
+    |
+        offsetFetch[1] = UnsignedNumericLiteralOrParam()
+    |
+        <ALL>
+    )
+}
+
+/**
+ * Parses a leaf in a query expression (SELECT, VALUES or TABLE).
+ */
+SqlNode LeafQuery(ExprContext exprContext) :
+{
+    SqlNode e;
+}
+{
+    {
+        // ensure a query is legal in this context
+        checkQueryExpression(exprContext);
+    }
+    e = SqlSelect() { return e; }
+|
+    e = TableConstructor() { return e; }
+|
+    e = ExplicitTable(getPos()) { return e; }
+}
+
+/**
+ * Parses a parenthesized query or single row expression.
+ * Depending on {@code exprContext}, may also accept a join.
+ */
+SqlNode ParenthesizedExpression(ExprContext exprContext) :
+{
+    SqlNode e;
+}
+{
+    <LPAREN>
+    {
+        // we've now seen left paren, so queries inside should
+        // be allowed as sub-queries
+        switch (exprContext) {
+        case ACCEPT_SUB_QUERY:
+            exprContext = ExprContext.ACCEPT_NONCURSOR;
+            break;
+        case ACCEPT_CURSOR:
+            exprContext = ExprContext.ACCEPT_ALL;
+            break;
+        }
+    }
+    e = ExprOrJoinOrOrderedQuery(exprContext)
+    <RPAREN>
+    {
+        exprContext.throwIfNotCompatible(e);
+        return e;
+    }
+}
+
+/**
+ * Parses a parenthesized query or comma-list of row expressions.
+ *
+ * <p>REVIEW jvs 8-Feb-2004: There's a small hole in this production.  It can be
+ * used to construct something like
+ *
+ * <blockquote><pre>
+ * WHERE x IN (select count(*) from t where c=d,5)</pre>
+ * </blockquote>
+ *
+ * <p>which should be illegal.  The above is interpreted as equivalent to
+ *
+ * <blockquote><pre>
+ * WHERE x IN ((select count(*) from t where c=d),5)</pre>
+ * </blockquote>
+ *
+ * <p>which is a legal use of a sub-query.  The only way to fix the hole is to
+ * be able to remember whether a subexpression was parenthesized or not, which
+ * means preserving parentheses in the SqlNode tree.  This is probably
+ * desirable anyway for use in purely syntactic parsing applications (e.g. SQL
+ * pretty-printer).  However, if this is done, it's important to also make
+ * isA() on the paren node call down to its operand so that we can
+ * always correctly discriminate a query from a row expression.
+ */
+SqlNodeList ParenthesizedQueryOrCommaList(
+    ExprContext exprContext) :
+{
+    SqlNode e;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    ExprContext firstExprContext = exprContext;
+    final Span s;
+}
+{
+    <LPAREN>
+    {
+        // we've now seen left paren, so a query by itself should
+        // be interpreted as a sub-query
+        s = span();
+        switch (exprContext) {
+        case ACCEPT_SUB_QUERY:
+            firstExprContext = ExprContext.ACCEPT_NONCURSOR;
+            break;
+        case ACCEPT_CURSOR:
+            firstExprContext = ExprContext.ACCEPT_ALL;
+            break;
+        }
+    }
+    e = OrderedQueryOrExpr(firstExprContext) { list.add(e); }
+    (
+        <COMMA>
+        {
+            // a comma-list can't appear where only a query is expected
+            checkNonQueryExpression(exprContext);
+        }
+        AddExpression(list, exprContext)
+    )*
+    <RPAREN>
+    {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+/** As ParenthesizedQueryOrCommaList, but allows DEFAULT
+ * in place of any of the expressions. For example,
+ * {@code (x, DEFAULT, null, DEFAULT)}. */
+SqlNodeList ParenthesizedQueryOrCommaListWithDefault(
+    ExprContext exprContext) :
+{
+    SqlNode e;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    ExprContext firstExprContext = exprContext;
+    final Span s;
+}
+{
+    <LPAREN>
+    {
+        // we've now seen left paren, so a query by itself should
+        // be interpreted as a sub-query
+        s = span();
+        switch (exprContext) {
+        case ACCEPT_SUB_QUERY:
+            firstExprContext = ExprContext.ACCEPT_NONCURSOR;
+            break;
+        case ACCEPT_CURSOR:
+            firstExprContext = ExprContext.ACCEPT_ALL;
+            break;
+        }
+    }
+    (
+        e = OrderedQueryOrExpr(firstExprContext) { list.add(e); }
+    |
+        e = Default() { list.add(e); }
+    )
+    (
+        <COMMA>
+        {
+            // a comma-list can't appear where only a query is expected
+            checkNonQueryExpression(exprContext);
+        }
+        (
+            e = Expression(exprContext) { list.add(e); }
+        |
+            e = Default() { list.add(e); }
+        )
+    )*
+    <RPAREN>
+    {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+/**
+ * Parses function parameter lists.
+ * If the list starts with DISTINCT or ALL, it is discarded.
+ */
+List<SqlNode> UnquantifiedFunctionParameterList(ExprContext exprContext) :
+{
+    final List<SqlNode> args;
+}
+{
+    args = FunctionParameterList(exprContext) {
+        args.remove(0); // remove DISTINCT or ALL, if present
+        return args;
+    }
+}
+
+/**
+ * Parses function parameter lists including DISTINCT keyword recognition,
+ * DEFAULT, and named argument assignment.
+ */
+List<SqlNode> FunctionParameterList(ExprContext exprContext) :
+{
+    final SqlLiteral qualifier;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    <LPAREN>
+    (
+        qualifier = AllOrDistinct() { list.add(qualifier); }
+    |
+        { list.add(null); }
+    )
+    AddArg0(list, exprContext)
+    (
+        <COMMA> {
+            // a comma-list can't appear where only a query is expected
+            checkNonQueryExpression(exprContext);
+        }
+        AddArg(list, exprContext)
+    )*
+    <RPAREN>
+    {
+        return list;
+    }
+}
+
+SqlLiteral AllOrDistinct() :
+{
+}
+{
+    <DISTINCT> { return SqlSelectKeyword.DISTINCT.symbol(getPos()); }
+|
+    <ALL> { return SqlSelectKeyword.ALL.symbol(getPos()); }
+}
+
+void AddArg0(List<SqlNode> list, ExprContext exprContext) :
+{
+    final SqlIdentifier name;
+    SqlNode e;
+    final ExprContext firstExprContext;
+    {
+        // we've now seen left paren, so queries inside should
+        // be allowed as sub-queries
+        switch (exprContext) {
+        case ACCEPT_SUB_QUERY:
+            firstExprContext = ExprContext.ACCEPT_NONCURSOR;
+            break;
+        case ACCEPT_CURSOR:
+            firstExprContext = ExprContext.ACCEPT_ALL;
+            break;
+        default:
+            firstExprContext = exprContext;
+            break;
+        }
+    }
+}
+{
+    (
+        LOOKAHEAD(2) name = SimpleIdentifier() <NAMED_ARGUMENT_ASSIGNMENT>
+    |   { name = null; }
+    )
+    (
+        e = Default()
+    |
+        LOOKAHEAD(3)
+        e = TableParam()
+    |
+        e = PartitionedQueryOrQueryOrExpr(firstExprContext)
+    )
+    {
+        if (name != null) {
+            e = SqlStdOperatorTable.ARGUMENT_ASSIGNMENT.createCall(
+                Span.of(name, e).pos(), e, name);
+        }
+        list.add(e);
+    }
+}
+
+void AddArg(List<SqlNode> list, ExprContext exprContext) :
+{
+    final SqlIdentifier name;
+    SqlNode e;
+}
+{
+    (
+        LOOKAHEAD(2) name = SimpleIdentifier() <NAMED_ARGUMENT_ASSIGNMENT>
+    |   { name = null; }
+    )
+    (
+        e = Default()
+    |
+        e = Expression(exprContext)
+    |
+        e = TableParam()
+    )
+    {
+        if (name != null) {
+            e = SqlStdOperatorTable.ARGUMENT_ASSIGNMENT.createCall(
+                Span.of(name, e).pos(), e, name);
+        }
+        list.add(e);
+    }
+}
+
+SqlNode Default() : {}
+{
+    <DEFAULT_> {
+        return SqlStdOperatorTable.DEFAULT.createCall(getPos());
+    }
+}
+
+/**
+ * Parses a query (SELECT, UNION, INTERSECT, EXCEPT, VALUES, TABLE) followed by
+ * the end-of-file symbol.
+ */
+SqlNode SqlQueryEof() :
+{
+    SqlNode query;
+}
+{
+    query = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY)
+    <EOF>
+    { return query; }
+}
+
+/**
+ * Parses a list of SQL statements separated by semicolon.
+ * The semicolon is required between statements, but is
+ * optional at the end.
+ */
+SqlNodeList SqlStmtList() :
+{
+    final List<SqlNode> stmtList = new ArrayList<SqlNode>();
+    SqlNode stmt;
+}
+{
+    stmt = SqlStmt() {
+        stmtList.add(stmt);
+    }
+    (
+        <SEMICOLON>
+        [
+            stmt = SqlStmt() {
+                stmtList.add(stmt);
+            }
+        ]
+    )*
+    <EOF>
+    {
+        return new SqlNodeList(stmtList, Span.of(stmtList).pos());
+    }
+}
+
+/**
+ * Parses an SQL statement.
+ */
+SqlNode SqlStmt() :
+{
+    SqlNode stmt;
+}
+{
+    (
+<#-- Add methods to parse additional statements here -->
+<#list (parser.statementParserMethods!default.parser.statementParserMethods) as method>
+        LOOKAHEAD(2) stmt = ${method}
+    |
+</#list>
+        stmt = SqlSetOption(Span.of(), null)
+    |
+        stmt = SqlAlter()
+    |
+<#if (parser.createStatementParserMethods!default.parser.createStatementParserMethods)?size != 0>
+        stmt = SqlCreate()
+    |
+</#if>
+<#if (parser.dropStatementParserMethods!default.parser.dropStatementParserMethods)?size != 0>
+        stmt = SqlDrop()
+    |
+</#if>
+        stmt = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY)
+    |
+        stmt = SqlExplain()
+    |
+        stmt = SqlDescribe()
+    |
+        stmt = SqlInsert()
+    |
+        stmt = SqlDelete()
+    |
+        stmt = SqlUpdate()
+    |
+        stmt = SqlMerge()
+    |
+        stmt = SqlProcedureCall()
+    )
+    {
+        return stmt;
+    }
+}
+
+/**
+ * Parses an SQL statement followed by the end-of-file symbol.
+ */
+SqlNode SqlStmtEof() :
+{
+    SqlNode stmt;
+}
+{
+    stmt = SqlStmt() <EOF>
+    {
+        return stmt;
+    }
+}
+
+<#-- Add implementations of additional parser statement calls here -->
+<#list (parser.implementationFiles!default.parser.implementationFiles) as file>
+    <#include "/@includes/"+file />
+</#list>
+
+SqlNodeList ParenthesizedKeyValueOptionCommaList() :
+{
+    final Span s;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    { s = span(); }
+    <LPAREN>
+    AddKeyValueOption(list)
+    (
+        <COMMA>
+        AddKeyValueOption(list)
+    )*
+    <RPAREN> {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+/**
+* Parses an option with format key=val whose key is a simple identifier or string literal
+* and value is a string literal.
+*/
+void AddKeyValueOption(List<SqlNode> list) :
+{
+    final SqlNode key;
+    final SqlNode value;
+}
+{
+    (
+        key = SimpleIdentifier()
+    |
+        key = StringLiteral()
+    )
+    <EQ>
+    value = StringLiteral() {
+        list.add(key);
+        list.add(value);
+    }
+}
+
+/** Parses an option value (either a string or a numeric) and adds to a list. */
+void AddOptionValue(List<SqlNode> list) :
+{
+    final SqlNode value;
+}
+{
+    (
+        value = NumericLiteral() { list.add(value); }
+    |
+        value = StringLiteral() { list.add(value); }
+    )
+}
+
+/**
+ * Parses a literal list separated by comma. The literal is either a string or a numeric.
+ */
+SqlNodeList ParenthesizedLiteralOptionCommaList() :
+{
+    final Span s;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    { s = span(); }
+    <LPAREN>
+    AddOptionValue(list) ( <COMMA> AddOptionValue(list) )*
+    <RPAREN> {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+void AddHint(List<SqlNode> hints) :
+{
+    final SqlIdentifier hintName;
+    final SqlNodeList hintOptions;
+    final SqlHint.HintOptionFormat optionFormat;
+}
+{
+    hintName = SimpleIdentifier()
+    (
+       LOOKAHEAD(5)
+        hintOptions = ParenthesizedKeyValueOptionCommaList() {
+            optionFormat = SqlHint.HintOptionFormat.KV_LIST;
+        }
+    |
+        LOOKAHEAD(3)
+        hintOptions = ParenthesizedSimpleIdentifierList() {
+            optionFormat = SqlHint.HintOptionFormat.ID_LIST;
+        }
+    |
+        LOOKAHEAD(3)
+        hintOptions = ParenthesizedLiteralOptionCommaList() {
+            optionFormat = SqlHint.HintOptionFormat.LITERAL_LIST;
+        }
+    |
+        LOOKAHEAD(2)
+        [<LPAREN> <RPAREN>]
+        {
+            hintOptions = SqlNodeList.EMPTY;
+            optionFormat = SqlHint.HintOptionFormat.EMPTY;
+        }
+    )
+    {
+        hints.add(
+           new SqlHint(Span.of(hintOptions).end(this), hintName, hintOptions,
+               optionFormat));
+    }
+}
+
+/** Parses hints following a table reference,
+ * and returns the wrapped table reference. */
+SqlNode TableHints(SqlIdentifier tableName) :
+{
+    final List<SqlNode> hints = new ArrayList<SqlNode>();
+}
+{
+    <HINT_BEG> AddHint(hints) ( <COMMA> AddHint(hints) )* <COMMENT_END> {
+        final SqlParserPos pos = Span.of(tableName).addAll(hints).end(this);
+        final SqlNodeList hintList = new SqlNodeList(hints, pos);
+        return new SqlTableRef(pos, tableName, hintList);
+    }
+}
+
+/**
+ * Parses a leaf SELECT expression without ORDER BY.
+ */
+SqlSelect SqlSelect() :
+{
+    final List<SqlLiteral> keywords = new ArrayList<SqlLiteral>();
+    final SqlLiteral keyword;
+    final SqlNodeList keywordList;
+    final List<SqlNode> selectList = new ArrayList<SqlNode>();
+    final SqlNode fromClause;
+    final SqlNode where;
+    final SqlNodeList groupBy;
+    final SqlNode having;
+    final SqlNodeList windowDecls;
+    final SqlNode qualify;
+    final List<SqlNode> hints = new ArrayList<SqlNode>();
+    final Span s;
+}
+{
+    <SELECT> { s = span(); }
+    [ <HINT_BEG> AddHint(hints) ( <COMMA> AddHint(hints) )* <COMMENT_END> ]
+    SqlSelectKeywords(keywords)
+    (
+        <STREAM> {
+            keywords.add(SqlSelectKeyword.STREAM.symbol(getPos()));
+        }
+    )?
+    (
+        keyword = AllOrDistinct() { keywords.add(keyword); }
+    )?
+    {
+        keywordList = new SqlNodeList(keywords, s.addAll(keywords).pos());
+    }
+    AddSelectItem(selectList)
+    ( <COMMA> AddSelectItem(selectList) )*
+    (
+        <FROM> fromClause = FromClause()
+        ( where = Where() | { where = null; } )
+        ( groupBy = GroupBy() | { groupBy = null; } )
+        ( having = Having() | { having = null; } )
+        ( windowDecls = Window() | { windowDecls = null; } )
+        ( qualify = Qualify() | { qualify = null; } )
+    |
+        E() {
+            fromClause = null;
+            where = null;
+            groupBy = null;
+            having = null;
+            windowDecls = null;
+            qualify = null;
+        }
+    )
+    {
+        return new SqlSelect(s.end(this), keywordList,
+            new SqlNodeList(selectList, Span.of(selectList).pos()),
+            fromClause, where, groupBy, having, windowDecls, qualify,
+            null, null, null, new SqlNodeList(hints, getPos()));
+    }
+}
+
+/*
+ * Abstract production:
+ *
+ *    void SqlSelectKeywords(List keywords)
+ *
+ * Parses dialect-specific keywords immediately following the SELECT keyword.
+ */
+
+/**
+ * Parses an EXPLAIN PLAN statement.
+ */
+SqlNode SqlExplain() :
+{
+    SqlNode stmt;
+    SqlExplainLevel detailLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES;
+    SqlExplain.Depth depth;
+    final SqlExplainFormat format;
+}
+{
+    <EXPLAIN> <PLAN>
+    [ detailLevel = ExplainDetailLevel() ]
+    depth = ExplainDepth()
+    (
+        LOOKAHEAD(2)
+        <AS> <XML> { format = SqlExplainFormat.XML; }
+    |
+        LOOKAHEAD(2)
+        <AS> <JSON> { format = SqlExplainFormat.JSON; }
+    |
+        <AS> <DOT_FORMAT> { format = SqlExplainFormat.DOT; }
+    |
+        { format = SqlExplainFormat.TEXT; }
+    )
+    <FOR> stmt = SqlQueryOrDml() {
+        return new SqlExplain(getPos(),
+            stmt,
+            detailLevel.symbol(SqlParserPos.ZERO),
+            depth.symbol(SqlParserPos.ZERO),
+            format.symbol(SqlParserPos.ZERO),
+            nDynamicParams);
+    }
+}
+
+/** Parses a query (SELECT or VALUES)
+ * or DML statement (INSERT, UPDATE, DELETE, MERGE). */
+SqlNode SqlQueryOrDml() :
+{
+    SqlNode stmt;
+}
+{
+    (
+        stmt = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY)
+    |
+        stmt = SqlInsert()
+    |
+        stmt = SqlDelete()
+    |
+        stmt = SqlUpdate()
+    |
+        stmt = SqlMerge()
+    ) { return stmt; }
+}
+
+/**
+ * Parses WITH TYPE | WITH IMPLEMENTATION | WITHOUT IMPLEMENTATION modifier for
+ * EXPLAIN PLAN.
+ */
+SqlExplain.Depth ExplainDepth() :
+{
+}
+{
+    (
+        LOOKAHEAD(2)
+        <WITH> <TYPE>
+        {
+            return SqlExplain.Depth.TYPE;
+        }
+        |
+        <WITH> <IMPLEMENTATION>
+        {
+            return SqlExplain.Depth.PHYSICAL;
+        }
+        |
+        <WITHOUT> <IMPLEMENTATION>
+        {
+            return SqlExplain.Depth.LOGICAL;
+        }
+        |
+        {
+            return SqlExplain.Depth.PHYSICAL;
+        }
+
+    )
+}
+
+/**
+ * Parses INCLUDING ALL ATTRIBUTES modifier for EXPLAIN PLAN.
+ */
+SqlExplainLevel ExplainDetailLevel() :
+{
+    SqlExplainLevel level = SqlExplainLevel.EXPPLAN_ATTRIBUTES;
+}
+{
+    (
+        <EXCLUDING> <ATTRIBUTES>
+        {
+            level = SqlExplainLevel.NO_ATTRIBUTES;
+        }
+        |
+        <INCLUDING>
+        [ <ALL> { level = SqlExplainLevel.ALL_ATTRIBUTES; } ]
+        <ATTRIBUTES>
+        {
+        }
+    )
+    {
+        return level;
+    }
+}
+
+/**
+ * Parses a DESCRIBE statement.
+ */
+SqlNode SqlDescribe() :
+{
+   final Span s;
+   final SqlIdentifier table;
+   final SqlIdentifier column;
+   final SqlIdentifier id;
+   final SqlNode stmt;
+}
+{
+    <DESCRIBE> { s = span(); }
+    (
+        LOOKAHEAD(2) (<DATABASE> | <CATALOG> | <SCHEMA>)
+        id = CompoundIdentifier() {
+            // DESCRIBE DATABASE and DESCRIBE CATALOG currently do the same as
+            // DESCRIBE SCHEMA but should be different. See
+            //   [CALCITE-1221] Implement DESCRIBE DATABASE, CATALOG, STATEMENT
+            return new SqlDescribeSchema(s.end(id), id);
+        }
+    |
+        // Use syntactic lookahead to determine whether a table name is coming.
+        // We do not allow SimpleIdentifier() because that includes <STATEMENT>.
+        LOOKAHEAD( <TABLE>
+           | <IDENTIFIER>
+           | <HYPHENATED_IDENTIFIER>
+           | <QUOTED_IDENTIFIER>
+           | <BACK_QUOTED_IDENTIFIER>
+           | <BIG_QUERY_BACK_QUOTED_IDENTIFIER>
+           | <BRACKET_QUOTED_IDENTIFIER> )
+        (<TABLE>)?
+        table = CompoundIdentifier()
+        ( column = SimpleIdentifier() | { column = null; } )
+        {
+            return new SqlDescribeTable(s.add(table).addIf(column).pos(),
+                table, column);
+        }
+    |
+        (LOOKAHEAD(1) <STATEMENT>)?
+        stmt = SqlQueryOrDml() {
+            // DESCRIBE STATEMENT currently does the same as EXPLAIN. See
+            //   [CALCITE-1221] Implement DESCRIBE DATABASE, CATALOG, STATEMENT
+            final SqlExplainLevel detailLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES;
+            final SqlExplain.Depth depth = SqlExplain.Depth.PHYSICAL;
+            final SqlExplainFormat format = SqlExplainFormat.TEXT;
+            return new SqlExplain(s.end(stmt),
+                stmt,
+                detailLevel.symbol(SqlParserPos.ZERO),
+                depth.symbol(SqlParserPos.ZERO),
+                format.symbol(SqlParserPos.ZERO),
+                nDynamicParams);
+        }
+    )
+}
+
+/**
+ * Parses a CALL statement.
+ */
+SqlNode SqlProcedureCall() :
+{
+    final Span s;
+    SqlNode routineCall;
+}
+{
+    <CALL> {
+        s = span();
+    }
+    routineCall = NamedRoutineCall(
+        SqlFunctionCategory.USER_DEFINED_PROCEDURE,
+        ExprContext.ACCEPT_SUB_QUERY)
+    {
+        return SqlStdOperatorTable.PROCEDURE_CALL.createCall(
+            s.end(routineCall), routineCall);
+    }
+}
+
+SqlNode NamedRoutineCall(
+    SqlFunctionCategory routineType,
+    ExprContext exprContext) :
+{
+    final SqlIdentifier name;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    final Span s;
+}
+{
+    name = CompoundIdentifier() {
+        s = span();
+    }
+    <LPAREN>
+    [
+        AddArg0(list, exprContext)
+        (
+            <COMMA> {
+                // a comma-list can't appear where only a query is expected
+                checkNonQueryExpression(exprContext);
+            }
+            AddArg(list, exprContext)
+        )*
+    ]
+    <RPAREN>
+    {
+        return createCall(name, s.end(this), routineType, null, list);
+    }
+}
+
+/**
+ * Table parameter of a table function.
+ * The input table with set semantics may be partitioned/ordered on one or more columns.
+ */
+SqlNode TableParam() :
+{
+    final Span s;
+    final SqlNodeList partitionList;
+    final SqlNodeList orderList;
+    SqlNode tableRef;
+}
+{
+    { s = span(); }
+    tableRef = ExplicitTable(getPos())
+    (
+        <PARTITION> <BY>
+        partitionList = SimpleIdentifierOrList()
+    |   { partitionList = SqlNodeList.EMPTY; }
+    )
+    (
+        orderList = OrderByOfSetSemanticsTable()
+     |  { orderList = SqlNodeList.EMPTY; }
+    )
+    { return CreateSetSemanticsTableIfNeeded(s, tableRef, partitionList, orderList); }
+}
+
+SqlNode PartitionedQueryOrQueryOrExpr(ExprContext exprContext) :
+{
+    SqlNode e;
+}
+{
+    e = OrderedQueryOrExpr(exprContext)
+    e = PartitionedByAndOrderBy(e)
+
+    { return e; }
+}
+
+SqlNode PartitionedByAndOrderBy(SqlNode e) :
+{
+    final Span s;
+    final SqlNodeList partitionList;
+    final SqlNodeList orderList;
+}
+{
+    { s = span(); }
+    (
+        <PARTITION> <BY>
+        partitionList = SimpleIdentifierOrList()
+    |   { partitionList = SqlNodeList.EMPTY; }
+    )
+    (
+        orderList = OrderByOfSetSemanticsTable()
+     |  { orderList = SqlNodeList.EMPTY; }
+    )
+    { return CreateSetSemanticsTableIfNeeded(s, e, partitionList, orderList); }
+}
+
+SqlNodeList OrderByOfSetSemanticsTable() :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    final Span s;
+}
+{
+  <ORDER>
+  { s = span(); }
+  <BY>
+  (
+      LOOKAHEAD(2)
+      <LPAREN> AddOrderItem(list)
+      (
+        // NOTE jvs 6-Feb-2004:  See comments at top of file for why
+        // hint is necessary here.
+        LOOKAHEAD(2) <COMMA> AddOrderItem(list)
+      )*
+      <RPAREN> {
+          return new SqlNodeList(list, s.addAll(list).pos());
+      }
+  |
+      AddOrderItem(list)
+      {
+          return new SqlNodeList(list, s.addAll(list).pos());
+      }
+  )
+}
+
+SqlNode CreateSetSemanticsTableIfNeeded(
+    final Span s,
+    final SqlNode e,
+    final SqlNodeList partitionList,
+    final SqlNodeList orderList) :
+{
+
+}
+{
+
+    {
+        if (partitionList.isEmpty() && orderList.isEmpty()) {
+            return e;
+        } else {
+            return SqlStdOperatorTable.SET_SEMANTICS_TABLE.createCall(
+                s.pos(), e, partitionList, orderList);
+        }
+    }
+}
+
+/**
+ * Parses an INSERT statement.
+ */
+SqlNode SqlInsert() :
+{
+    final List<SqlLiteral> keywords = new ArrayList<SqlLiteral>();
+    final SqlNodeList keywordList;
+    final SqlIdentifier tableName;
+    SqlNode tableRef;
+    SqlNode source;
+    final SqlNodeList columnList;
+    final Span s;
+    final Pair<SqlNodeList, SqlNodeList> p;
+}
+{
+    (
+        <INSERT>
+    |
+        <UPSERT> { keywords.add(SqlInsertKeyword.UPSERT.symbol(getPos())); }
+    )
+    { s = span(); }
+    SqlInsertKeywords(keywords) {
+        keywordList = new SqlNodeList(keywords, s.addAll(keywords).pos());
+    }
+    <INTO> tableName = CompoundTableIdentifier()
+    ( tableRef = TableHints(tableName) | { tableRef = tableName; } )
+    [ LOOKAHEAD(5) tableRef = ExtendTable(tableRef) ]
+    (
+        LOOKAHEAD(2)
+        p = ParenthesizedCompoundIdentifierList() {
+            if (p.right.size() > 0) {
+                tableRef = extend(tableRef, p.right);
+            }
+            if (p.left.size() > 0) {
+                columnList = p.left;
+            } else {
+                columnList = null;
+            }
+        }
+    |   { columnList = null; }
+    )
+    source = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) {
+        return new SqlInsert(s.end(source), keywordList, tableRef, source,
+            columnList);
+    }
+}
+
+/*
+ * Abstract production:
+ *
+ *    void SqlInsertKeywords(List keywords)
+ *
+ * Parses dialect-specific keywords immediately following the INSERT keyword.
+ */
+
+/**
+ * Parses a DELETE statement.
+ */
+SqlNode SqlDelete() :
+{
+    final SqlIdentifier tableName;
+    SqlNode tableRef;
+    final SqlIdentifier alias;
+    final SqlNode where;
+    final Span s;
+}
+{
+    <DELETE> {
+        s = span();
+    }
+    <FROM> tableName = CompoundTableIdentifier()
+    ( tableRef = TableHints(tableName) | { tableRef = tableName; } )
+    [ tableRef = ExtendTable(tableRef) ]
+    ( [ <AS> ] alias = SimpleIdentifier() | { alias = null; } )
+    ( where = Where() | { where = null; } )
+    {
+        return new SqlDelete(s.add(tableRef).addIf(alias).addIf(where).pos(),
+            tableRef, where, null, alias);
+    }
+}
+
+/**
+ * Parses an UPDATE statement.
+ */
+SqlNode SqlUpdate() :
+{
+    final SqlIdentifier tableName;
+    SqlNode tableRef;
+    final SqlIdentifier alias;
+    final SqlNode where;
+    final SqlNodeList sourceExpressionList;
+    final SqlNodeList targetColumnList;
+    SqlIdentifier id;
+    final Span s;
+}
+{
+    <UPDATE> {
+        s = span();
+        targetColumnList = new SqlNodeList(s.pos());
+        sourceExpressionList = new SqlNodeList(s.pos());
+    }
+    tableName = CompoundTableIdentifier()
+    ( tableRef = TableHints(tableName) | { tableRef = tableName; } )
+    [ tableRef = ExtendTable(tableRef) ]
+    ( [ <AS> ] alias = SimpleIdentifier() | { alias = null; } )
+    <SET> id = SimpleIdentifier() {
+        targetColumnList.add(id);
+    }
+    // TODO:  support DEFAULT also
+    <EQ> AddExpression(sourceExpressionList, ExprContext.ACCEPT_SUB_QUERY)
+    (
+        <COMMA>
+        id = SimpleIdentifier() { targetColumnList.add(id); }
+        <EQ> AddExpression(sourceExpressionList, ExprContext.ACCEPT_SUB_QUERY)
+    )*
+    ( where = Where() | { where = null; } )
+    {
+        final SqlParserPos pos = s.addAll(targetColumnList)
+            .addAll(sourceExpressionList).addIf(where).pos();
+        return new SqlUpdate(pos, tableRef, targetColumnList,
+            sourceExpressionList, where, null, alias);
+    }
+}
+
+/**
+ * Parses a MERGE statement.
+ */
+SqlNode SqlMerge() :
+{
+    final SqlIdentifier tableName;
+    SqlNode tableRef;
+    final SqlIdentifier alias;
+    final SqlNode sourceTableRef;
+    final SqlNode condition;
+    final SqlUpdate updateCall;
+    final SqlInsert insertCall;
+    final Span s;
+}
+{
+    <MERGE> { s = span(); } <INTO> tableName = CompoundTableIdentifier()
+    ( tableRef = TableHints(tableName) | { tableRef = tableName; } )
+    [ tableRef = ExtendTable(tableRef) ]
+    ( [ <AS> ] alias = SimpleIdentifier() | { alias = null; } )
+    <USING> sourceTableRef = TableRef()
+    <ON> condition = Expression(ExprContext.ACCEPT_SUB_QUERY)
+    (
+        LOOKAHEAD(2)
+        updateCall = WhenMatchedClause(tableRef, alias)
+        ( insertCall = WhenNotMatchedClause(tableRef) | { insertCall = null; } )
+    |
+        { updateCall = null; }
+        insertCall = WhenNotMatchedClause(tableRef)
+    )
+    {
+        final SqlParserPos pos = s.addIf(updateCall).addIf(insertCall).pos();
+        return new SqlMerge(pos, tableRef, condition, sourceTableRef,
+            updateCall, insertCall, null, alias);
+    }
+}
+
+SqlUpdate WhenMatchedClause(SqlNode table, SqlIdentifier alias) :
+{
+    SqlIdentifier id;
+    final Span s;
+    final SqlNodeList updateColumnList = new SqlNodeList(SqlParserPos.ZERO);
+    final SqlNodeList updateExprList = new SqlNodeList(SqlParserPos.ZERO);
+}
+{
+    <WHEN> { s = span(); } <MATCHED> <THEN>
+    <UPDATE> <SET> id = CompoundIdentifier() {
+        updateColumnList.add(id);
+    }
+    <EQ> AddExpression(updateExprList, ExprContext.ACCEPT_SUB_QUERY)
+    (
+        <COMMA>
+        id = CompoundIdentifier() {
+            updateColumnList.add(id);
+        }
+        <EQ> AddExpression(updateExprList, ExprContext.ACCEPT_SUB_QUERY)
+    )*
+    {
+        return new SqlUpdate(s.addAll(updateExprList).pos(), table,
+            updateColumnList, updateExprList, null, null, alias);
+    }
+}
+
+SqlInsert WhenNotMatchedClause(SqlNode table) :
+{
+    final Span insertSpan, valuesSpan;
+    final List<SqlLiteral> keywords = new ArrayList<SqlLiteral>();
+    final SqlNodeList keywordList;
+    final SqlNodeList insertColumnList;
+    SqlNode rowConstructor;
+    SqlNode insertValues;
+}
+{
+    <WHEN> <NOT> <MATCHED> <THEN> <INSERT> {
+        insertSpan = span();
+    }
+    SqlInsertKeywords(keywords) {
+        keywordList = new SqlNodeList(keywords, insertSpan.end(this));
+    }
+    (
+        LOOKAHEAD(2)
+        insertColumnList = ParenthesizedSimpleIdentifierList()
+    |   { insertColumnList = null; }
+    )
+    (
+        <LPAREN>
+        <VALUES> { valuesSpan = span(); } rowConstructor = RowConstructor()
+        <RPAREN>
+    |
+        <VALUES> { valuesSpan = span(); } rowConstructor = RowConstructor()
+    )
+    {
+        // TODO zfong 5/26/06: note that extra parentheses are accepted above
+        // around the VALUES clause as a hack for unparse, but this is
+        // actually invalid SQL; should fix unparse
+        insertValues = SqlStdOperatorTable.VALUES.createCall(
+            valuesSpan.end(this), rowConstructor);
+        return new SqlInsert(insertSpan.end(this), keywordList,
+            table, insertValues, insertColumnList);
+    }
+}
+
+/**
+ * Parses one item in a select list.
+ */
+void AddSelectItem(List<SqlNode> list) :
+{
+    final SqlNode e;
+    final SqlIdentifier id;
+}
+{
+    e = SelectExpression()
+    (
+        [ <AS> ]
+        (
+            id = SimpleIdentifier()
+        |
+            // Mute the warning about ambiguity between alias and continued
+            // string literal.
+            LOOKAHEAD(1)
+            id = SimpleIdentifierFromStringLiteral()
+        )
+        { list.add(SqlStdOperatorTable.AS.createCall(span().end(e), e, id)); }
+    |   { list.add(e); }
+    )
+}
+
+/**
+ * Parses one unaliased expression in a select list.
+ */
+SqlNode SelectExpression() :
+{
+    SqlNode e;
+}
+{
+    <STAR> {
+        return SqlIdentifier.star(getPos());
+    }
+|
+    e = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+        return e;
+    }
+}
+
+SqlLiteral Natural() :
+{
+}
+{
+    <NATURAL> { return SqlLiteral.createBoolean(true, getPos()); }
+|
+    { return SqlLiteral.createBoolean(false, getPos()); }
+}
+
+SqlLiteral JoinType() :
+{
+    JoinType joinType;
+}
+{
+    (
+    LOOKAHEAD(3) // required for "LEFT SEMI JOIN" in Babel
+<#list (parser.joinTypes!default.parser.joinTypes) as method>
+        joinType = ${method}()
+    |
+</#list>
+        <JOIN> { joinType = JoinType.INNER; }
+    |
+        <INNER> <JOIN> { joinType = JoinType.INNER; }
+    |
+        <LEFT> [ <OUTER> ] <JOIN> { joinType = JoinType.LEFT; }
+    |
+        <RIGHT> [ <OUTER> ] <JOIN> { joinType = JoinType.RIGHT; }
+    |
+        <FULL> [ <OUTER> ] <JOIN> { joinType = JoinType.FULL; }
+    |
+        <CROSS> <JOIN> { joinType = JoinType.CROSS; }
+    )
+    {
+        return joinType.symbol(getPos());
+    }
+}
+
+/**
+ * Parses the FROM clause for a SELECT.
+ *
+ * <p>FROM is mandatory in standard SQL, optional in dialects such as MySQL,
+ * PostgreSQL. The parser allows SELECT without FROM, but the validator fails
+ * if conformance is, say, STRICT_2003.
+ */
+SqlNode FromClause() :
+{
+    SqlNode e, e2;
+    SqlLiteral joinType;
+}
+{
+    e = Join()
+    (
+        // Comma joins should only occur at top-level in the FROM clause.
+        // Valid:
+        //  * FROM a, b
+        //  * FROM (a CROSS JOIN b), c
+        // Not valid:
+        //  * FROM a CROSS JOIN (b, c)
+        LOOKAHEAD(1)
+        <COMMA> { joinType = JoinType.COMMA.symbol(getPos()); }
+        e2 = Join() {
+            e = new SqlJoin(joinType.getParserPosition(),
+                e,
+                SqlLiteral.createBoolean(false, joinType.getParserPosition()),
+                joinType,
+                e2,
+                JoinConditionType.NONE.symbol(SqlParserPos.ZERO),
+                null);
+        }
+    )*
+    { return e; }
+}
+
+SqlNode Join() :
+{
+    SqlNode e;
+}
+{
+    e = TableRef1(ExprContext.ACCEPT_QUERY_OR_JOIN)
+    (
+        LOOKAHEAD(2)
+        e = JoinTable(e)
+    )*
+    {
+        return e;
+    }
+}
+
+/** Matches "LEFT JOIN t ON ...", "RIGHT JOIN t USING ...", "JOIN t". */
+SqlNode JoinTable(SqlNode e) :
+{
+    SqlNode e2, condition;
+    final SqlLiteral natural, joinType, on, using;
+    SqlNodeList list;
+}
+{
+    // LOOKAHEAD(3) is needed here rather than a LOOKAHEAD(2) because JavaCC
+    // calculates minimum lookahead count incorrectly for choice that contains
+    // zero size child. For instance, with the generated code,
+    // "LOOKAHEAD(2, Natural(), JoinType())"
+    // returns true immediately if it sees a single "<CROSS>" token. Where we
+    // expect the lookahead succeeds after "<CROSS> <APPLY>".
+    //
+    // For more information about the issue,
+    // see https://github.com/javacc/javacc/issues/86
+    //
+    // We allow CROSS JOIN (joinType = CROSS_JOIN) to have a join condition,
+    // even though that is not valid SQL; the validator will catch it.
+    LOOKAHEAD(3)
+    natural = Natural()
+    joinType = JoinType()
+    e2 = TableRef1(ExprContext.ACCEPT_QUERY_OR_JOIN)
+    (
+        <ON> { on = JoinConditionType.ON.symbol(getPos()); }
+        condition = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+            return new SqlJoin(joinType.getParserPosition(),
+                e,
+                natural,
+                joinType,
+                e2,
+                on,
+                condition);
+        }
+    |
+        <USING> { using = JoinConditionType.USING.symbol(getPos()); }
+        list = ParenthesizedSimpleIdentifierList() {
+            return new SqlJoin(joinType.getParserPosition(),
+                e,
+                natural,
+                joinType,
+                e2,
+                using,
+                new SqlNodeList(list, Span.of(using).end(this)));
+        }
+    |
+        {
+            return new SqlJoin(joinType.getParserPosition(),
+                e,
+                natural,
+                joinType,
+                e2,
+                JoinConditionType.NONE.symbol(joinType.getParserPosition()),
+                null);
+        }
+    )
+|
+    <CROSS> { joinType = JoinType.CROSS.symbol(getPos()); } <APPLY>
+    e2 = TableRef2(true) {
+        if (!this.conformance.isApplyAllowed()) {
+            throw SqlUtil.newContextException(getPos(), RESOURCE.applyNotAllowed());
+        }
+        return new SqlJoin(joinType.getParserPosition(),
+            e,
+            SqlLiteral.createBoolean(false, joinType.getParserPosition()),
+            joinType,
+            e2,
+            JoinConditionType.NONE.symbol(SqlParserPos.ZERO),
+            null);
+    }
+|
+    <OUTER> { joinType = JoinType.LEFT.symbol(getPos()); } <APPLY>
+    e2 = TableRef2(true) {
+        if (!this.conformance.isApplyAllowed()) {
+            throw SqlUtil.newContextException(getPos(), RESOURCE.applyNotAllowed());
+        }
+        return new SqlJoin(joinType.getParserPosition(),
+            e,
+            SqlLiteral.createBoolean(false, joinType.getParserPosition()),
+            joinType,
+            e2,
+            JoinConditionType.ON.symbol(SqlParserPos.ZERO),
+            SqlLiteral.createBoolean(true, joinType.getParserPosition()));
+    }
+}
+
+/**
+ * Parses a table reference in a FROM clause, not lateral unless LATERAL
+ * is explicitly specified.
+ */
+SqlNode TableRef() :
+{
+    final SqlNode e;
+}
+{
+    e = TableRef3(ExprContext.ACCEPT_QUERY, false) { return e; }
+}
+
+SqlNode TableRef1(ExprContext exprContext) :
+{
+    final SqlNode e;
+}
+{
+    e = TableRef3(exprContext, false) { return e; }
+}
+
+/**
+ * Parses a table reference in a FROM clause.
+ */
+SqlNode TableRef2(boolean lateral) :
+{
+    final SqlNode e;
+}
+{
+    e = TableRef3(ExprContext.ACCEPT_QUERY, lateral) { return e; }
+}
+
+SqlNode TableRef3(ExprContext exprContext, boolean lateral) :
+{
+    final SqlIdentifier tableName;
+    SqlNode tableRef;
+    final SqlIdentifier alias;
+    final Span s;
+    SqlNodeList args;
+    final SqlNodeList columnAliasList;
+    SqlUnnestOperator unnestOp = SqlStdOperatorTable.UNNEST;
+}
+{
+    (
+        LOOKAHEAD(2)
+        tableName = CompoundTableIdentifier()
+        ( tableRef = TableHints(tableName) | { tableRef = tableName; } )
+        [ tableRef = ExtendTable(tableRef) ]
+        tableRef = Over(tableRef)
+        [ tableRef = Snapshot(tableRef) ]
+        [ tableRef = MatchRecognize(tableRef) ]
+    |
+        LOOKAHEAD(2)
+        [ <LATERAL> { lateral = true; } ]
+        tableRef = ParenthesizedExpression(exprContext)
+        tableRef = Over(tableRef)
+        tableRef = addLateral(tableRef, lateral)
+        [ tableRef = MatchRecognize(tableRef) ]
+    |
+        <UNNEST> { s = span(); }
+        args = ParenthesizedQueryOrCommaList(ExprContext.ACCEPT_SUB_QUERY)
+        [
+            <WITH> <ORDINALITY> {
+                unnestOp = SqlStdOperatorTable.UNNEST_WITH_ORDINALITY;
+            }
+        ]
+        {
+            tableRef = unnestOp.createCall(s.end(this), (List<SqlNode>) args);
+        }
+    |
+        [ <LATERAL> { lateral = true; } ]
+        tableRef = TableFunctionCall()
+        tableRef = addLateral(tableRef, lateral)
+    |
+        tableRef = ExtendedTableRef()
+    )
+    [
+        LOOKAHEAD(2)
+        tableRef = Pivot(tableRef)
+    ]
+    [
+        LOOKAHEAD(2)
+        tableRef = Unpivot(tableRef)
+    ]
+    [
+        [ <AS> ] alias = SimpleIdentifier()
+        (
+            columnAliasList = ParenthesizedSimpleIdentifierList()
+        |   { columnAliasList = null; }
+        )
+        {
+            // Standard SQL (and Postgres) allow applying "AS alias" to a JOIN,
+            // e.g. "FROM (a CROSS JOIN b) AS c". The new alias obscures the
+            // internal aliases, and columns cannot be referenced if they are
+            // not unique. TODO: Support this behavior; see
+            // [CALCITE-5168] Allow AS after parenthesized JOIN
+            checkNotJoin(tableRef);
+            if (columnAliasList == null) {
+                tableRef = SqlStdOperatorTable.AS.createCall(
+                    Span.of(tableRef).end(this), tableRef, alias);
+            } else {
+                List<SqlNode> idList = new ArrayList<SqlNode>();
+                idList.add(tableRef);
+                idList.add(alias);
+                idList.addAll(columnAliasList.getList());
+                tableRef = SqlStdOperatorTable.AS.createCall(
+                    Span.of(tableRef).end(this), idList);
+            }
+        }
+    ]
+    [ tableRef = Tablesample(tableRef) ]
+    { return tableRef; }
+}
+
+SqlNode Tablesample(SqlNode tableRef) :
+{
+    final Span s;
+    final SqlNode sample;
+    final boolean isBernoulli;
+    final SqlNumericLiteral samplePercentage;
+    boolean isRepeatable = false;
+    int repeatableSeed = 0;
+}
+{
+    <TABLESAMPLE> { s = span(); checkNotJoin(tableRef); }
+    (
+        <SUBSTITUTE> <LPAREN> sample = StringLiteral() <RPAREN>
+        {
+            String sampleName =
+                SqlLiteral.unchain(sample).getValueAs(String.class);
+            SqlSampleSpec sampleSpec = SqlSampleSpec.createNamed(sampleName);
+            final SqlLiteral sampleLiteral =
+                SqlLiteral.createSample(sampleSpec, s.end(this));
+            tableRef = SqlStdOperatorTable.TABLESAMPLE.createCall(
+                s.add(tableRef).end(this), tableRef, sampleLiteral);
+        }
+    |
+        (
+            <BERNOULLI> { isBernoulli = true; }
+        |
+            <SYSTEM> { isBernoulli = false; }
+        )
+        <LPAREN> samplePercentage = UnsignedNumericLiteral() <RPAREN>
+        [
+            <REPEATABLE> <LPAREN> repeatableSeed = IntLiteral() <RPAREN>
+            {
+                isRepeatable = true;
+            }
+        ]
+        {
+            final BigDecimal ONE_HUNDRED = BigDecimal.valueOf(100L);
+            BigDecimal rate = samplePercentage.bigDecimalValue();
+            if (rate.compareTo(BigDecimal.ZERO) < 0
+                || rate.compareTo(ONE_HUNDRED) > 0)
+            {
+                throw SqlUtil.newContextException(getPos(), RESOURCE.invalidSampleSize());
+            }
+
+            // Treat TABLESAMPLE(0) and TABLESAMPLE(100) as no table
+            // sampling at all.  Not strictly correct: TABLESAMPLE(0)
+            // should produce no output, but it simplifies implementation
+            // to know that some amount of sampling will occur.
+            // In practice values less than ~1E-43% are treated as 0.0 and
+            // values greater than ~99.999997% are treated as 1.0
+            float fRate = rate.divide(ONE_HUNDRED).floatValue();
+            if (fRate > 0.0f && fRate < 1.0f) {
+                SqlSampleSpec tableSampleSpec =
+                    isRepeatable
+                        ? SqlSampleSpec.createTableSample(
+                            isBernoulli, fRate, repeatableSeed)
+                        : SqlSampleSpec.createTableSample(isBernoulli, fRate);
+
+                SqlLiteral tableSampleLiteral =
+                    SqlLiteral.createSample(tableSampleSpec, s.end(this));
+                tableRef = SqlStdOperatorTable.TABLESAMPLE.createCall(
+                    s.end(this), tableRef, tableSampleLiteral);
+            }
+        }
+    )
+    { return tableRef; }
+}
+
+/** Wraps a table reference in a call to EXTEND if an optional "EXTEND" clause
+ * is present. */
+SqlNode ExtendTable(SqlNode tableRef) :
+{
+    final SqlNodeList extendList;
+}
+{
+    [ <EXTEND> ]
+    extendList = ExtendList() {
+        return extend(tableRef, extendList);
+    }
+}
+
+SqlNodeList ExtendList() :
+{
+    final Span s;
+    List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    <LPAREN> { s = span(); }
+    AddColumnType(list)
+    (
+        <COMMA> AddColumnType(list)
+    )*
+    <RPAREN> {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+void AddColumnType(List<SqlNode> list) :
+{
+    final SqlIdentifier name;
+    final SqlDataTypeSpec type;
+    final boolean nullable;
+}
+{
+    name = CompoundIdentifier()
+    type = DataType()
+    nullable = NotNullOpt()
+    {
+        list.add(name);
+        list.add(type.withNullable(nullable, getPos()));
+    }
+}
+
+/**
+ * Parses a compound identifier with optional type.
+ */
+void AddCompoundIdentifierType(List<SqlNode> list, List<SqlNode> extendList) :
+{
+    final SqlIdentifier name;
+    final SqlDataTypeSpec type;
+    final boolean nullable;
+}
+{
+    name = CompoundIdentifier()
+    (
+        type = DataType()
+        nullable = NotNullOpt()
+    |
+        { type = null; nullable = true; }
+    )
+    {
+        if (type != null) {
+            if (!this.conformance.allowExtend()) {
+                throw SqlUtil.newContextException(type.getParserPosition(),
+                    RESOURCE.extendNotAllowed());
+            }
+            extendList.add(name);
+            extendList.add(type.withNullable(nullable, getPos()));
+        }
+        list.add(name);
+    }
+}
+
+SqlNode TableFunctionCall() :
+{
+    final Span s;
+    final SqlNode call;
+    SqlFunctionCategory funcType = SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION;
+}
+{
+    <TABLE> { s = span(); } <LPAREN>
+    [
+        <SPECIFIC>
+        {
+            funcType = SqlFunctionCategory.USER_DEFINED_TABLE_SPECIFIC_FUNCTION;
+        }
+    ]
+    call = NamedRoutineCall(funcType, ExprContext.ACCEPT_CURSOR)
+    <RPAREN>
+    {
+        return SqlStdOperatorTable.COLLECTION_TABLE.createCall(s.end(this), call);
+    }
+}
+
+/**
+ * Abstract production:
+ *    SqlNode ExtendedTableRef()
+ *
+ * <p>Allows parser to be extended with new types of table references.  The
+ * default implementation of this production is empty.
+ */
+
+/*
+ * Abstract production:
+ *
+ *    SqlNode TableOverOpt()
+ *
+ * Allows an OVER clause following a table expression as an extension to
+ * standard SQL syntax. The default implementation of this production is empty.
+ */
+
+/**
+ * Parses an explicit TABLE t reference.
+ */
+SqlNode ExplicitTable(SqlParserPos pos) :
+{
+    SqlNode tableRef;
+}
+{
+    <TABLE> tableRef = CompoundIdentifier()
+    {
+        return SqlStdOperatorTable.EXPLICIT_TABLE.createCall(pos, tableRef);
+    }
+}
+
+/**
+ * Parses a VALUES leaf query expression.
+ */
+SqlNode TableConstructor() :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    final Span s;
+}
+{
+    (
+        <VALUES> { s = span(); }
+    |
+        <VALUE>
+        {
+            s = span();
+            if (!this.conformance.isValueAllowed()) {
+                throw SqlUtil.newContextException(getPos(), RESOURCE.valueNotAllowed());
+            }
+        }
+    )
+    AddRowConstructor(list)
+    (
+        LOOKAHEAD(2)
+        <COMMA> AddRowConstructor(list)
+    )*
+    {
+        return SqlStdOperatorTable.VALUES.createCall(s.end(this), list);
+    }
+}
+
+/** Parses a row constructor and adds it to a list. */
+void AddRowConstructor(List<SqlNode> list) :
+{
+    SqlNode e;
+}
+{
+    e = RowConstructor() { list.add(e); }
+}
+
+/**
+ * Parses a row constructor in the context of a VALUES expression.
+ */
+SqlNode RowConstructor() :
+{
+    final SqlNodeList valueList;
+    final SqlNode value;
+    final Span s;
+}
+{
+    // hints are necessary here due to common LPAREN prefixes
+    (
+        // TODO jvs 8-Feb-2004: extra parentheses are accepted here as a hack
+        // for unparse, but this is actually invalid SQL; should
+        // fix unparse
+        LOOKAHEAD(3)
+        <LPAREN> { s = span(); }
+        <ROW>
+        valueList = ParenthesizedQueryOrCommaListWithDefault(ExprContext.ACCEPT_NONCURSOR)
+        <RPAREN> { s.add(this); }
+    |
+        LOOKAHEAD(3)
+        (
+            <ROW> { s = span(); }
+        |
+            { s = Span.of(); }
+        )
+        valueList = ParenthesizedQueryOrCommaListWithDefault(ExprContext.ACCEPT_NONCURSOR)
+    |
+        value = Expression(ExprContext.ACCEPT_NONCURSOR)
+        {
+            // NOTE: A bare value here is standard SQL syntax, believe it or
+            // not.  Taken together with multi-row table constructors, it leads
+            // to very easy mistakes if you forget the parentheses on a
+            // single-row constructor.  This is also the reason for the
+            // LOOKAHEAD in TableConstructor().  It would be so much more
+            // reasonable to require parentheses.  Sigh.
+            s = Span.of(value);
+            valueList = new SqlNodeList(ImmutableList.of(value),
+                value.getParserPosition());
+        }
+    )
+    {
+        // REVIEW jvs 8-Feb-2004: Should we discriminate between scalar
+        // sub-queries inside of ROW and row sub-queries?  The standard does,
+        // but the distinction seems to be purely syntactic.
+        return SqlStdOperatorTable.ROW.createCall(s.end(valueList),
+            (List<SqlNode>) valueList);
+    }
+}
+
+/** Parses a WHERE clause for SELECT, DELETE, and UPDATE. */
+SqlNode Where() :
+{
+    SqlNode condition;
+}
+{
+    <WHERE> condition = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+        return condition;
+    }
+}
+
+/** Parses a GROUP BY clause for SELECT. */
+SqlNodeList GroupBy() :
+{
+    final List<SqlNode> list;
+    final boolean distinct;
+    final Span s;
+}
+{
+    <GROUP> { s = span(); }
+    <BY>
+    (
+        <DISTINCT> { distinct = true; }
+    |   <ALL> { distinct = false; }
+    |   { distinct = false; }
+    )
+    list = GroupingElementList() {
+        final SqlParserPos pos = s.end(this);
+        final List<SqlNode> list2 = distinct
+            ? ImmutableList.of(
+                SqlInternalOperators.GROUP_BY_DISTINCT.createCall(pos, list))
+            : list;
+        return new SqlNodeList(list2, pos);
+    }
+}
+
+List<SqlNode> GroupingElementList() :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    AddGroupingElement(list)
+    ( LOOKAHEAD(2) <COMMA> AddGroupingElement(list) )*
+    { return list; }
+}
+
+void AddGroupingElement(List<SqlNode> list) :
+{
+    final List<SqlNode> subList;
+    final SqlNodeList nodes;
+    final Span s;
+}
+{
+    LOOKAHEAD(2)
+    <GROUPING> { s = span(); }
+    <SETS> <LPAREN> subList = GroupingElementList() <RPAREN> {
+        list.add(
+            SqlStdOperatorTable.GROUPING_SETS.createCall(s.end(this), subList));
+    }
+|   <ROLLUP> { s = span(); }
+    <LPAREN> nodes = ExpressionCommaList(s, ExprContext.ACCEPT_SUB_QUERY)
+    <RPAREN> {
+        list.add(
+            SqlStdOperatorTable.ROLLUP.createCall(s.end(this), nodes.getList()));
+    }
+|   <CUBE> { s = span(); }
+    <LPAREN> nodes = ExpressionCommaList(s, ExprContext.ACCEPT_SUB_QUERY)
+    <RPAREN> {
+        list.add(
+            SqlStdOperatorTable.CUBE.createCall(s.end(this), nodes.getList()));
+    }
+|   LOOKAHEAD(3)
+    <LPAREN> { s = span(); } <RPAREN> {
+        list.add(new SqlNodeList(s.end(this)));
+    }
+|   AddExpression(list, ExprContext.ACCEPT_SUB_QUERY)
+}
+
+/**
+ * Parses a list of expressions separated by commas.
+ */
+SqlNodeList ExpressionCommaList(
+    final Span s,
+    ExprContext exprContext) :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    AddExpressions(list, exprContext) {
+        return new SqlNodeList(list, s.addAll(list).pos());
+    }
+}
+
+/**
+ * Parses a list of expressions separated by commas,
+ * appending expressions to a given list.
+ */
+void AddExpressions(List<SqlNode> list, ExprContext exprContext) :
+{
+}
+{
+    AddExpression(list, exprContext)
+    (
+        // NOTE jvs 6-Feb-2004:  See comments at top of file for why
+        // hint is necessary here.
+        LOOKAHEAD(2)
+        <COMMA> AddExpression(list, ExprContext.ACCEPT_SUB_QUERY)
+    )*
+}
+
+/** Parses a HAVING clause for SELECT. */
+SqlNode Having() :
+{
+    SqlNode e;
+}
+{
+    <HAVING> e = Expression(ExprContext.ACCEPT_SUB_QUERY) { return e; }
+}
+
+/** Parses a WINDOW clause for SELECT. */
+SqlNodeList Window() :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    final Span s;
+}
+{
+    <WINDOW> { s = span(); }
+    AddWindowSpec(list)
+    (
+        LOOKAHEAD(2)
+        <COMMA> AddWindowSpec(list)
+    )*
+    {
+        return new SqlNodeList(list, s.addAll(list).pos());
+    }
+}
+
+void AddWindowSpec(List<SqlNode> list) :
+{
+    final SqlIdentifier id;
+    final SqlWindow e;
+}
+{
+    id = SimpleIdentifier() <AS> e = WindowSpecification() {
+        e.setDeclName(id);
+        list.add(e);
+    }
+}
+
+/**
+ * Parses a window specification.
+ */
+SqlWindow WindowSpecification() :
+{
+    final SqlIdentifier id;
+    final SqlNodeList partitionList;
+    final SqlNodeList orderList;
+    final SqlLiteral isRows;
+    final SqlNode lowerBound, upperBound;
+    final Span s, s1, s2;
+    final SqlLiteral allowPartial;
+}
+{
+    <LPAREN> { s = span(); }
+    (
+        id = SimpleIdentifier()
+    |   { id = null; }
+    )
+    (
+        <PARTITION> { s1 = span(); }
+        <BY>
+        partitionList = ExpressionCommaList(s1, ExprContext.ACCEPT_NON_QUERY)
+    |   { partitionList = SqlNodeList.EMPTY; }
+    )
+    (
+        orderList = OrderBy(true)
+    |   { orderList = SqlNodeList.EMPTY; }
+    )
+    (
+        (
+            <ROWS> { isRows = SqlLiteral.createBoolean(true, getPos()); }
+        |
+            <RANGE> { isRows = SqlLiteral.createBoolean(false, getPos()); }
+        )
+        (
+            <BETWEEN> lowerBound = WindowRange()
+            <AND> upperBound = WindowRange()
+        |
+            lowerBound = WindowRange()
+            { upperBound = null; }
+        )
+    |
+        {
+            isRows = SqlLiteral.createBoolean(false, SqlParserPos.ZERO);
+            lowerBound = upperBound = null;
+        }
+    )
+    (
+        <ALLOW> { s2 = span(); } <PARTIAL> {
+            allowPartial = SqlLiteral.createBoolean(true, s2.end(this));
+        }
+    |
+        <DISALLOW> { s2 = span(); } <PARTIAL> {
+            allowPartial = SqlLiteral.createBoolean(false, s2.end(this));
+        }
+    |   { allowPartial = null; }
+    )
+    <RPAREN>
+    {
+        return SqlWindow.create(null, id, partitionList, orderList,
+            isRows, lowerBound, upperBound, allowPartial, s.end(this));
+    }
+}
+
+SqlNode WindowRange() :
+{
+    final SqlNode e;
+    final Span s;
+}
+{
+    LOOKAHEAD(2)
+    <CURRENT> { s = span(); } <ROW> {
+        return SqlWindow.createCurrentRow(s.end(this));
+    }
+|
+    LOOKAHEAD(2)
+    <UNBOUNDED> { s = span(); }
+    (
+        <PRECEDING> {
+            return SqlWindow.createUnboundedPreceding(s.end(this));
+        }
+    |
+        <FOLLOWING> {
+            return SqlWindow.createUnboundedFollowing(s.end(this));
+        }
+    )
+|
+    e = Expression(ExprContext.ACCEPT_NON_QUERY)
+    (
+        <PRECEDING> {
+            return SqlWindow.createPreceding(e, getPos());
+        }
+    |
+        <FOLLOWING> {
+            return SqlWindow.createFollowing(e, getPos());
+        }
+    )
+}
+
+/** Parses a QUALIFY clause for SELECT. */
+SqlNode Qualify() :
+{
+    SqlNode e;
+}
+{
+    <QUALIFY> e = Expression(ExprContext.ACCEPT_SUB_QUERY) { return e; }
+}
+
+/**
+ * Parses an ORDER BY clause.
+ */
+SqlNodeList OrderBy(boolean accept) :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    final Span s;
+}
+{
+    <ORDER> {
+        s = span();
+        if (!accept) {
+            // Someone told us ORDER BY wasn't allowed here.  So why
+            // did they bother calling us?  To get the correct
+            // parser position for error reporting.
+            throw SqlUtil.newContextException(s.pos(), RESOURCE.illegalOrderBy());
+        }
+    }
+    <BY> AddOrderItem(list)
+    (
+        // NOTE jvs 6-Feb-2004:  See comments at top of file for why
+        // hint is necessary here.
+        LOOKAHEAD(2) <COMMA> AddOrderItem(list)
+    )*
+    {
+        return new SqlNodeList(list, s.addAll(list).pos());
+    }
+}
+
+/**
+ * Parses one item in an ORDER BY clause, and adds it to a list.
+ */
+void AddOrderItem(List<SqlNode> list) :
+{
+    SqlNode e;
+}
+{
+    e = Expression(ExprContext.ACCEPT_SUB_QUERY)
+    (
+        <ASC>
+    |   <DESC> {
+            e = SqlStdOperatorTable.DESC.createCall(getPos(), e);
+        }
+    )?
+    (
+        LOOKAHEAD(2)
+        <NULLS> <FIRST> {
+            e = SqlStdOperatorTable.NULLS_FIRST.createCall(getPos(), e);
+        }
+    |
+        <NULLS> <LAST> {
+            e = SqlStdOperatorTable.NULLS_LAST.createCall(getPos(), e);
+        }
+    )?
+    {
+        list.add(e);
+    }
+}
+
+/** Wraps a table reference in a call to OVER if an optional "OVER" clause
+ * is present (if the dialect supports OVER for table expressions). */
+SqlNode Over(SqlNode tableRef) :
+{
+    final SqlNode over;
+}
+{
+    over = TableOverOpt() {
+        if (over != null) {
+            return SqlStdOperatorTable.OVER.createCall(
+                getPos(), checkNotJoin(tableRef), over);
+        } else {
+            return tableRef;
+        }
+    }
+}
+
+/** Wraps a table reference in a call to LATERAL if {@code lateral} is true. */
+JAVACODE SqlNode addLateral(SqlNode tableRef, boolean lateral) {
+    return lateral
+        ? SqlStdOperatorTable.LATERAL.createCall(getPos(),
+            checkNotJoin(tableRef))
+        : tableRef;
+}
+
+/**
+ * Parses a FOR SYSTEM_TIME clause following a table expression.
+ */
+SqlSnapshot Snapshot(SqlNode tableRef) :
+{
+    final Span s;
+    final SqlNode e;
+}
+{
+    { s = span(); } <FOR> <SYSTEM_TIME> <AS> <OF>
+    // Syntax for temporal table in
+    // standard SQL 2011 IWD 9075-2:201?(E) 7.6 <table reference>
+    // supports grammar as following:
+    // 1. datetime literal
+    // 2. datetime value function, i.e. CURRENT_TIMESTAMP
+    // 3. datetime term in 1 or 2 +(or -) interval term
+
+    // We extend to support column reference, use Expression
+    // to simplify the parsing code.
+    e = Expression(ExprContext.ACCEPT_NON_QUERY) {
+        return new SqlSnapshot(s.end(this), tableRef, e);
+    }
+}
+
+/** Parses a PIVOT clause following a table expression. */
+SqlNode Pivot(SqlNode tableRef) :
+{
+    final Span s;
+    final Span s2;
+    final List<SqlNode> aggList = new ArrayList<SqlNode>();
+    final List<SqlNode> valueList = new ArrayList<SqlNode>();
+    final SqlNodeList axisList;
+    final SqlNodeList inList;
+}
+{
+    <PIVOT> { s = span(); checkNotJoin(tableRef); }
+    <LPAREN>
+    AddPivotAgg(aggList) ( <COMMA> AddPivotAgg(aggList) )*
+    <FOR> axisList = SimpleIdentifierOrList()
+    <IN> <LPAREN> { s2 = span(); }
+    [ AddPivotValue(valueList) ( <COMMA> AddPivotValue(valueList) )* ]
+    <RPAREN> {
+        inList = new SqlNodeList(valueList, s2.end(this));
+    }
+    <RPAREN>
+    {
+        return new SqlPivot(s.end(this), tableRef,
+            new SqlNodeList(aggList, SqlParserPos.sum(aggList)),
+            axisList, inList);
+    }
+}
+
+void AddPivotAgg(List<SqlNode> list) :
+{
+    final SqlNode e;
+    final SqlIdentifier alias;
+}
+{
+    e = NamedFunctionCall()
+    (
+        // Because babel put FOR into non-reserved keyword set.
+        LOOKAHEAD({getToken(1).kind != COMMA && getToken(1).kind != FOR})
+        [ <AS> ] alias = SimpleIdentifier() {
+            list.add(
+                SqlStdOperatorTable.AS.createCall(Span.of(e).end(this), e,
+                    alias));
+        }
+    |
+        { list.add(e); }
+    )
+}
+
+void AddPivotValue(List<SqlNode> list) :
+{
+    final SqlNode e;
+    final SqlNodeList tuple;
+    final SqlIdentifier alias;
+}
+{
+    e = RowConstructor() { tuple = SqlParserUtil.stripRow(e); }
+    (
+        [ <AS> ] alias = SimpleIdentifier() {
+            list.add(
+                SqlStdOperatorTable.AS.createCall(Span.of(tuple).end(this),
+                    tuple, alias));
+        }
+    |
+        { list.add(tuple); }
+    )
+}
+
+/** Parses an UNPIVOT clause following a table expression. */
+SqlNode Unpivot(SqlNode tableRef) :
+{
+    final Span s;
+    final boolean includeNulls;
+    final SqlNodeList measureList;
+    final SqlNodeList axisList;
+    final Span s2;
+    final List<SqlNode> values = new ArrayList<SqlNode>();
+    final SqlNodeList inList;
+}
+{
+    <UNPIVOT> { s = span(); checkNotJoin(tableRef); }
+    (
+        <INCLUDE> <NULLS> { includeNulls = true; }
+    |   <EXCLUDE> <NULLS> { includeNulls = false; }
+    |   { includeNulls = false; }
+    )
+    <LPAREN>
+    measureList = SimpleIdentifierOrList()
+    <FOR> axisList = SimpleIdentifierOrList()
+    <IN>
+    <LPAREN> { s2 = span(); }
+    AddUnpivotValue(values) ( <COMMA> AddUnpivotValue(values) )*
+    <RPAREN>
+    { inList = new SqlNodeList(values, s2.end(this)); }
+    <RPAREN> {
+        return new SqlUnpivot(s.end(this), tableRef, includeNulls, measureList,
+            axisList, inList);
+    }
+}
+
+void AddUnpivotValue(List<SqlNode> list) :
+{
+    final SqlNodeList columnList;
+    final SqlNode values;
+}
+{
+    columnList = SimpleIdentifierOrList()
+    (
+        <AS> values = RowConstructor() {
+            final SqlNodeList valueList = SqlParserUtil.stripRow(values);
+            list.add(
+                SqlStdOperatorTable.AS.createCall(Span.of(columnList).end(this),
+                    columnList, valueList));
+        }
+    |
+        { list.add(columnList); }
+    )
+}
+
+/**
+ * Parses a MATCH_RECOGNIZE clause following a table expression.
+ */
+SqlMatchRecognize MatchRecognize(SqlNode tableRef) :
+{
+    final Span s, s0, s1, s2;
+    final SqlNodeList measureList;
+    final SqlNodeList partitionList;
+    final SqlNodeList orderList;
+    final SqlNode pattern;
+    final SqlLiteral interval;
+    final SqlNodeList patternDefList;
+    final SqlNode after;
+    final SqlNode var;
+    final SqlLiteral rowsPerMatch;
+    final SqlNodeList subsetList;
+    final SqlLiteral isStrictStarts;
+    final SqlLiteral isStrictEnds;
+}
+{
+    <MATCH_RECOGNIZE> { s = span(); checkNotJoin(tableRef); } <LPAREN>
+    (
+        <PARTITION> { s2 = span(); } <BY>
+        partitionList = ExpressionCommaList(s2, ExprContext.ACCEPT_NON_QUERY)
+    |
+        { partitionList = SqlNodeList.EMPTY; }
+    )
+    (
+        orderList = OrderBy(true)
+    |
+        { orderList = SqlNodeList.EMPTY; }
+    )
+    (
+        <MEASURES>
+        measureList = MeasureColumnCommaList(span())
+    |
+        { measureList = SqlNodeList.EMPTY; }
+    )
+    (
+        <ONE> { s0 = span(); } <ROW> <PER> <MATCH> {
+            rowsPerMatch = SqlMatchRecognize.RowsPerMatchOption.ONE_ROW.symbol(s0.end(this));
+        }
+    |
+        <ALL> { s0 = span(); } <ROWS> <PER> <MATCH> {
+            rowsPerMatch = SqlMatchRecognize.RowsPerMatchOption.ALL_ROWS.symbol(s0.end(this));
+        }
+    |   { rowsPerMatch = null; }
+    )
+    (
+        <AFTER> { s1 = span(); } <MATCH> <SKIP_>
+        (
+            <TO>
+            (
+                LOOKAHEAD(2)
+                <NEXT> <ROW> {
+                    after = SqlMatchRecognize.AfterOption.SKIP_TO_NEXT_ROW
+                        .symbol(s1.end(this));
+                }
+            |
+                LOOKAHEAD(2)
+                <FIRST> var = SimpleIdentifier() {
+                    after = SqlMatchRecognize.SKIP_TO_FIRST.createCall(
+                        s1.end(var), var);
+                }
+            |
+                // This "LOOKAHEAD({true})" is a workaround for Babel.
+                // Because of babel parser uses option "LOOKAHEAD=2" globally,
+                // JavaCC generates something like "LOOKAHEAD(2, [<LAST>] SimpleIdentifier())"
+                // here. But the correct LOOKAHEAD should be
+                // "LOOKAHEAD(2, [ LOOKAHEAD(2, <LAST> SimpleIdentifier()) <LAST> ]
+                // SimpleIdentifier())" which have the syntactic lookahead for <LAST> considered.
+                //
+                // Overall LOOKAHEAD({true}) is even better as this is the last branch in the
+                // choice.
+                LOOKAHEAD({true})
+                [ LOOKAHEAD(2, <LAST> SimpleIdentifier()) <LAST> ] var = SimpleIdentifier() {
+                    after = SqlMatchRecognize.SKIP_TO_LAST.createCall(
+                        s1.end(var), var);
+                }
+            )
+        |
+            <PAST> <LAST> <ROW> {
+                 after = SqlMatchRecognize.AfterOption.SKIP_PAST_LAST_ROW
+                     .symbol(s1.end(this));
+            }
+        )
+    |   { after = null; }
+    )
+    <PATTERN>
+    <LPAREN>
+    (
+        <CARET> { isStrictStarts = SqlLiteral.createBoolean(true, getPos()); }
+    |   { isStrictStarts = SqlLiteral.createBoolean(false, getPos()); }
+    )
+    pattern = PatternExpression()
+    (
+        <DOLLAR> { isStrictEnds = SqlLiteral.createBoolean(true, getPos()); }
+    |   { isStrictEnds = SqlLiteral.createBoolean(false, getPos()); }
+    )
+    <RPAREN>
+    (
+        <WITHIN> interval = IntervalLiteral()
+    |   { interval = null; }
+    )
+    (
+        <SUBSET> subsetList = SubsetDefinitionCommaList(span())
+    |   { subsetList = SqlNodeList.EMPTY; }
+    )
+    <DEFINE>
+    patternDefList = PatternDefinitionCommaList(span())
+    <RPAREN> {
+        return new SqlMatchRecognize(s.end(this), tableRef,
+            pattern, isStrictStarts, isStrictEnds, patternDefList, measureList,
+            after, subsetList, rowsPerMatch, partitionList, orderList, interval);
+    }
+}
+
+SqlNodeList MeasureColumnCommaList(Span s) :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    AddMeasureColumn(list)
+    ( <COMMA> AddMeasureColumn(list) )*
+    { return new SqlNodeList(list, s.addAll(list).pos()); }
+}
+
+void AddMeasureColumn(List<SqlNode> list) :
+{
+    final SqlNode e;
+    final SqlIdentifier alias;
+}
+{
+    e = Expression(ExprContext.ACCEPT_NON_QUERY)
+    <AS>
+    alias = SimpleIdentifier() {
+        list.add(SqlStdOperatorTable.AS.createCall(Span.of(e).end(this), e, alias));
+    }
+}
+
+SqlNode PatternExpression() :
+{
+    SqlNode left;
+    SqlNode right;
+}
+{
+    left = PatternTerm()
+    (
+        <VERTICAL_BAR>
+        right = PatternTerm() {
+            left = SqlStdOperatorTable.PATTERN_ALTER.createCall(
+                Span.of(left).end(right), left, right);
+        }
+    )*
+    {
+        return left;
+    }
+}
+
+SqlNode PatternTerm() :
+{
+    SqlNode left;
+    SqlNode right;
+}
+{
+    left = PatternFactor()
+    (
+        right = PatternFactor() {
+            left = SqlStdOperatorTable.PATTERN_CONCAT.createCall(
+                Span.of(left).end(right), left, right);
+        }
+    )*
+    {
+        return left;
+    }
+}
+
+SqlNode PatternFactor() :
+{
+    final SqlNode e;
+    final SqlNode extra;
+    final SqlLiteral startNum;
+    final SqlLiteral endNum;
+    final SqlLiteral reluctant;
+}
+{
+    e = PatternPrimary()
+    (
+        LOOKAHEAD(1)
+        (
+            <STAR> {
+                startNum = LITERAL_ZERO;
+                endNum = LITERAL_MINUS_ONE;
+            }
+        |
+            <PLUS> {
+                startNum = LITERAL_ONE;
+                endNum = LITERAL_MINUS_ONE;
+            }
+        |
+            <HOOK> {
+                startNum = LITERAL_ZERO;
+                endNum = LITERAL_ONE;
+            }
+        |
+            <LBRACE>
+            (
+                startNum = UnsignedNumericLiteral()
+                (
+                    <COMMA>
+                    (
+                        endNum = UnsignedNumericLiteral()
+                    |
+                        { endNum = LITERAL_MINUS_ONE; }
+                    )
+                |
+                     { endNum = startNum; }
+                )
+                <RBRACE>
+            |
+                <COMMA>
+                endNum = UnsignedNumericLiteral()
+                <RBRACE>
+                { startNum = LITERAL_MINUS_ONE; }
+            |
+                <MINUS> extra = PatternExpression() <MINUS> <RBRACE> {
+                    return SqlStdOperatorTable.PATTERN_CONCAT.createCall(
+                        Span.of(e).end(this), e,
+                        SqlStdOperatorTable.PATTERN_EXCLUDE.createCall(
+                            Span.of(extra).end(this), extra));
+                }
+            )
+        )
+        (
+            <HOOK> {
+                reluctant = SqlLiteral.createBoolean(
+                    startNum.intValue(true) != endNum.intValue(true),
+                    SqlParserPos.ZERO);
+            }
+        |
+            { reluctant = SqlLiteral.createBoolean(false, SqlParserPos.ZERO); }
+        )
+    |
+        { return e; }
+    )
+    {
+        return SqlStdOperatorTable.PATTERN_QUANTIFIER.createCall(
+            span().end(e), e, startNum, endNum, reluctant);
+    }
+}
+
+SqlNode PatternPrimary() :
+{
+    final Span s;
+    SqlNode e;
+    final List<SqlNode> list;
+}
+{
+    e = SimpleIdentifier() { return e; }
+|
+    <LPAREN> e = PatternExpression() <RPAREN> { return e; }
+|
+    <LBRACE> { s = span(); }
+    <MINUS> e = PatternExpression()
+    <MINUS> <RBRACE> {
+        return SqlStdOperatorTable.PATTERN_EXCLUDE.createCall(s.end(this), e);
+    }
+|
+    (
+        <PERMUTE> { s = span(); list = new ArrayList<SqlNode>(); }
+        <LPAREN>
+        e = PatternExpression() { list.add(e); }
+        ( <COMMA> e = PatternExpression() { list.add(e); } )*
+        <RPAREN> {
+            return SqlStdOperatorTable.PATTERN_PERMUTE.createCall(
+                s.end(this), list);
+        }
+    )
+}
+
+SqlNodeList SubsetDefinitionCommaList(Span s) :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    AddSubsetDefinition(list)
+    ( <COMMA> AddSubsetDefinition(list) )*
+    { return new SqlNodeList(list, s.addAll(list).pos()); }
+}
+
+void AddSubsetDefinition(List<SqlNode> list) :
+{
+    final SqlNode var;
+    final SqlNodeList varList;
+}
+{
+    var = SimpleIdentifier()
+    <EQ>
+    <LPAREN>
+    varList = ExpressionCommaList(span(), ExprContext.ACCEPT_NON_QUERY)
+    <RPAREN> {
+        list.add(
+            SqlStdOperatorTable.EQUALS.createCall(span().end(var), var,
+                varList));
+    }
+}
+
+SqlNodeList PatternDefinitionCommaList(Span s) :
+{
+    SqlNode e;
+    final List<SqlNode> eList = new ArrayList<SqlNode>();
+}
+{
+    e = PatternDefinition() {
+        eList.add(e);
+    }
+    (
+        <COMMA>
+        e = PatternDefinition() {
+            eList.add(e);
+        }
+    )*
+    {
+        return new SqlNodeList(eList, s.addAll(eList).pos());
+    }
+}
+
+SqlNode PatternDefinition() :
+{
+    final SqlNode var;
+    final SqlNode e;
+}
+{
+    var = SimpleIdentifier()
+    <AS>
+    e = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+        return SqlStdOperatorTable.AS.createCall(Span.of(var, e).pos(), e, var);
+    }
+}
+
+// ----------------------------------------------------------------------------
+// Expressions
+
+/**
+ * Parses a SQL expression (such as might occur in a WHERE clause) followed by
+ * the end-of-file symbol.
+ */
+SqlNode SqlExpressionEof() :
+{
+    SqlNode e;
+}
+{
+    e = Expression(ExprContext.ACCEPT_SUB_QUERY) (<EOF>)
+    {
+        return e;
+    }
+}
+
+/**
+ * Parses either a row expression or a query expression without ORDER BY.
+ *
+ * <p>Examples of valid queries:
+ * <ul>
+ * <li>{@code SELECT c FROM t}
+ * <li>{@code SELECT c} (valid in some dialects)
+ * <li>{@code SELECT c FROM t UNION SELECT c2 FROM t2}
+ * <li>{@code WITH q AS (SELECT 1) SELECT * FROM q}
+ * <li>{@code VALUES (1, 2)}
+ * <li>{@code TABLE t}
+ * </ul>
+ *
+ * <p>Non-examples:
+ * <ul>
+ * <li>{@code emp CROSS JOIN dept}
+ * <li>{@code SELECT c FROM t ORDER BY c}
+ * <li>{@code (SELECT c FROM t)}
+ * </ul>
+ */
+SqlNode QueryOrExpr(ExprContext exprContext) :
+{
+    SqlNodeList withList = null;
+    final SqlNode e;
+    final List<Object> list = new ArrayList<Object>();
+}
+{
+    [ withList = WithList() ]
+    e = LeafQueryOrExpr(exprContext) { list.add(e); }
+    ( AddSetOpQuery(list, exprContext) )*
+    { return addWith(withList, SqlParserUtil.toTree(list)); }
+}
+
+SqlNode Query(ExprContext exprContext) :
+{
+    SqlNodeList withList = null;
+    final SqlNode e;
+    final List<Object> list = new ArrayList<Object>();
+}
+{
+    [ withList = WithList() ]
+    e = LeafQuery(exprContext) { list.add(e); }
+    ( AddSetOpQuery(list, exprContext) )*
+    { return addWith(withList, SqlParserUtil.toTree(list)); }
+}
+
+JAVACODE SqlNode addWith(SqlNodeList withList, SqlNode e) {
+    return withList == null
+        ? e
+        : new SqlWith(withList.getParserPosition(), withList, e);
+}
+
+/** Parses a set operator (e.g. UNION or INTERSECT)
+ * followed by a query or expression,
+ * and adds both to {@code list}. */
+void AddSetOpQueryOrExpr(List<Object> list, ExprContext exprContext) :
+{
+    final SqlOperator op;
+    final SqlParserPos pos;
+    final SqlNode e;
+}
+{
+    {
+        if (list.size() == 1 && !((SqlNode) list.get(0)).isA(SqlKind.QUERY)) {
+            // whoops, expression we just parsed wasn't a query,
+            // but we're about to see something like UNION, so
+            // force an exception retroactively
+            checkNonQueryExpression(ExprContext.ACCEPT_QUERY);
+        }
+    }
+    op = BinaryQueryOperator() {
+        // ensure a query is legal in this context
+        pos = getPos();
+        checkQueryExpression(exprContext);
+    }
+    e = LeafQueryOrExpr(ExprContext.ACCEPT_QUERY) {
+        list.add(new SqlParserUtil.ToTreeListItem(op, pos));
+        list.add(e);
+    }
+}
+
+/** Parses a set operator (e.g. UNION or INTERSECT)
+ * followed by a query,
+ * and adds both to {@code list}. */
+void AddSetOpQuery(List<Object> list, ExprContext exprContext) :
+{
+    final SqlOperator op;
+    final SqlParserPos pos;
+    final SqlNode e;
+}
+{
+    {
+        if (list.size() == 1 && !((SqlNode) list.get(0)).isA(SqlKind.QUERY)) {
+            // whoops, expression we just parsed wasn't a query,
+            // but we're about to see something like UNION, so
+            // force an exception retroactively
+            checkNonQueryExpression(ExprContext.ACCEPT_QUERY);
+        }
+    }
+    op = BinaryQueryOperator() {
+        // ensure a query is legal in this context
+        pos = getPos();
+        checkQueryExpression(exprContext);
+    }
+    e = LeafQueryOrExpr(ExprContext.ACCEPT_QUERY) {
+        list.add(new SqlParserUtil.ToTreeListItem(op, pos));
+        list.add(e);
+    }
+}
+
+SqlNodeList WithList() :
+{
+    final Span s;
+    final List<SqlWithItem> list = new ArrayList<SqlWithItem>();
+}
+{
+    <WITH> { s = span(); }
+    AddWithItem(list) ( <COMMA> AddWithItem(list) )*
+    { return new SqlNodeList(list, s.end(this)); }
+}
+
+void AddWithItem(List<SqlWithItem> list) :
+{
+    final SqlIdentifier id;
+    final SqlNodeList columnList;
+    final SqlNode definition;
+}
+{
+    id = SimpleIdentifier()
+    ( columnList = ParenthesizedSimpleIdentifierList() | { columnList = null; } )
+    <AS>
+    definition = ParenthesizedExpression(ExprContext.ACCEPT_QUERY)
+    { list.add(new SqlWithItem(id.getParserPosition(), id, columnList, definition)); }
+}
+
+/**
+ * Parses either a row expression, a leaf query expression, or
+ * a parenthesized expression of any kind.
+ */
+SqlNode LeafQueryOrExpr(ExprContext exprContext) :
+{
+    SqlNode e;
+}
+{
+    e = LeafQuery(exprContext) { return e; }
+|
+    e = Expression(exprContext) { return e; }
+}
+
+/** As {@link #Expression} but appends to a list. */
+void AddExpression(List<SqlNode> list, ExprContext exprContext) :
+{
+    final SqlNode e;
+}
+{
+    e = Expression(exprContext) { list.add(e); }
+}
+
+/**
+ * Parses a row expression or a parenthesized expression of any kind.
+ */
+SqlNode Expression(ExprContext exprContext) :
+{
+    final List<Object> list;
+}
+{
+    list = Expression2(exprContext) { return SqlParserUtil.toTree(list); }
+}
+
+void AddExpression2b(List<Object> list, ExprContext exprContext) :
+{
+    SqlNode e;
+    SqlOperator op;
+    SqlNode ext;
+}
+{
+    (
+        LOOKAHEAD(1)
+        op = PrefixRowOperator() {
+            checkNonQueryExpression(exprContext);
+            list.add(new SqlParserUtil.ToTreeListItem(op, getPos()));
+        }
+    )*
+    e = Expression3(exprContext) {
+        list.add(e);
+    }
+    (
+        LOOKAHEAD(2) <DOT>
+        ext = RowExpressionExtension() {
+            list.add(
+                new SqlParserUtil.ToTreeListItem(
+                    SqlStdOperatorTable.DOT, getPos()));
+            list.add(ext);
+        }
+    )*
+}
+
+/**
+ * Parses a binary row expression, or a parenthesized expression of any
+ * kind.
+ *
+ * <p>The result is as a flat list of operators and operands. The top-level
+ * call to get an expression should call {@link #Expression}, but lower-level
+ * calls should call this, to give the parser the opportunity to associate
+ * operator calls.
+ *
+ * <p>For example 'a = b like c = d' should come out '((a = b) like c) = d'
+ * because LIKE and '=' have the same precedence, but tends to come out as '(a
+ * = b) like (c = d)' because (a = b) and (c = d) are parsed as separate
+ * expressions.
+ */
+List<Object> Expression2(ExprContext exprContext) :
+{
+    final List<Object> list = new ArrayList();
+    List<Object> list2;
+    final List<Object> list3 = new ArrayList();
+    SqlNodeList nodeList;
+    SqlNode e;
+    SqlOperator op;
+    SqlIdentifier p;
+    final Span s = span();
+}
+{
+    AddExpression2b(list, exprContext)
+    (
+        LOOKAHEAD(2)
+        (
+            LOOKAHEAD(2)
+            (
+                // Special case for "IN", because RHS of "IN" is the only place
+                // that an expression-list is allowed ("exp IN (exp1, exp2)").
+                LOOKAHEAD(2) {
+                    checkNonQueryExpression(exprContext);
+                }
+                (
+                    <NOT> <IN> { op = SqlStdOperatorTable.NOT_IN; }
+                |
+                    <IN> { op = SqlStdOperatorTable.IN; }
+                |
+                    { final SqlKind k; }
+                    k = comp()
+                    (
+                        <SOME> { op = SqlStdOperatorTable.some(k); }
+                    |
+                        <ANY> { op = SqlStdOperatorTable.some(k); }
+                    |
+                        <ALL> { op = SqlStdOperatorTable.all(k); }
+                    )
+                )
+                { s.clear().add(this); }
+                nodeList = ParenthesizedQueryOrCommaList(ExprContext.ACCEPT_NONCURSOR)
+                {
+                    list.add(new SqlParserUtil.ToTreeListItem(op, s.pos()));
+                    s.add(nodeList);
+                    // special case for stuff like IN (s1 UNION s2)
+                    if (nodeList.size() == 1) {
+                        SqlNode item = nodeList.get(0);
+                        if (item.isA(SqlKind.QUERY)) {
+                            list.add(item);
+                        } else {
+                            list.add(nodeList);
+                        }
+                    } else {
+                        list.add(nodeList);
+                    }
+                }
+            |
+                LOOKAHEAD(2) {
+                    checkNonQueryExpression(exprContext);
+                }
+                (
+                    <NOT> <BETWEEN> {
+                        op = SqlStdOperatorTable.NOT_BETWEEN;
+                        s.clear().add(this);
+                    }
+                    [
+                        <SYMMETRIC> { op = SqlStdOperatorTable.SYMMETRIC_NOT_BETWEEN; }
+                    |
+                        <ASYMMETRIC>
+                    ]
+                |
+                    <BETWEEN>
+                    {
+                        op = SqlStdOperatorTable.BETWEEN;
+                        s.clear().add(this);
+                    }
+                    [
+                        <SYMMETRIC> { op = SqlStdOperatorTable.SYMMETRIC_BETWEEN; }
+                    |
+                        <ASYMMETRIC>
+                    ]
+                )
+                AddExpression2b(list3, ExprContext.ACCEPT_SUB_QUERY) {
+                    list.add(new SqlParserUtil.ToTreeListItem(op, s.pos()));
+                    list.addAll(list3);
+                    list3.clear();
+                }
+            |
+                LOOKAHEAD(2) {
+                    checkNonQueryExpression(exprContext);
+                    s.clear().add(this);
+                }
+                (
+                    (
+                        <NOT>
+                        (
+                            <LIKE> { op = SqlStdOperatorTable.NOT_LIKE; }
+                        |
+                            <ILIKE> { op = SqlLibraryOperators.NOT_ILIKE; }
+                        |
+                            <RLIKE> { op = SqlLibraryOperators.NOT_RLIKE; }
+                        |
+                            <SIMILAR> <TO> { op = SqlStdOperatorTable.NOT_SIMILAR_TO; }
+                        )
+                    |
+                        <LIKE> { op = SqlStdOperatorTable.LIKE; }
+                    |
+                        <ILIKE> { op = SqlLibraryOperators.ILIKE; }
+                    |
+                        <RLIKE> { op = SqlLibraryOperators.RLIKE; }
+                    |
+                        <SIMILAR> <TO> { op = SqlStdOperatorTable.SIMILAR_TO; }
+                    )
+                <#if (parser.includePosixOperators!default.parser.includePosixOperators)>
+                |
+                    <NEGATE> <TILDE> { op = SqlStdOperatorTable.NEGATED_POSIX_REGEX_CASE_SENSITIVE; }
+                    [ <STAR> { op = SqlStdOperatorTable.NEGATED_POSIX_REGEX_CASE_INSENSITIVE; } ]
+                |
+                    <TILDE> { op = SqlStdOperatorTable.POSIX_REGEX_CASE_SENSITIVE; }
+                    [ <STAR> { op = SqlStdOperatorTable.POSIX_REGEX_CASE_INSENSITIVE; } ]
+                </#if>
+                )
+                list2 = Expression2(ExprContext.ACCEPT_SUB_QUERY) {
+                    list.add(new SqlParserUtil.ToTreeListItem(op, s.pos()));
+                    list.addAll(list2);
+                }
+                [
+                    LOOKAHEAD(2)
+                    <ESCAPE> e = Expression3(ExprContext.ACCEPT_SUB_QUERY) {
+                        s.clear().add(this);
+                        list.add(
+                            new SqlParserUtil.ToTreeListItem(
+                                SqlStdOperatorTable.ESCAPE, s.pos()));
+                        list.add(e);
+                    }
+                ]
+            |
+            <#list (parser.extraBinaryExpressions!default.parser.extraBinaryExpressions) as extra >
+                ${extra}(list, exprContext, s)
+            |
+            </#list>
+                LOOKAHEAD(3) op = BinaryRowOperator() {
+                    checkNonQueryExpression(exprContext);
+                    list.add(new SqlParserUtil.ToTreeListItem(op, getPos()));
+                }
+                AddExpression2b(list, ExprContext.ACCEPT_SUB_QUERY)
+            |
+                <LBRACKET>
+                e = Expression(ExprContext.ACCEPT_SUB_QUERY)
+                <RBRACKET> {
+                    list.add(
+                        new SqlParserUtil.ToTreeListItem(
+                            SqlStdOperatorTable.ITEM, getPos()));
+                    list.add(e);
+                }
+                (
+                    LOOKAHEAD(2) <DOT>
+                    p = SimpleIdentifier() {
+                        list.add(
+                            new SqlParserUtil.ToTreeListItem(
+                                SqlStdOperatorTable.DOT, getPos()));
+                        list.add(p);
+                    }
+                )*
+            |
+                {
+                    checkNonQueryExpression(exprContext);
+                }
+                op = PostfixRowOperator() {
+                    list.add(new SqlParserUtil.ToTreeListItem(op, getPos()));
+                }
+            )
+        )+
+        {
+            return list;
+        }
+    |
+        {
+            return list;
+        }
+    )
+}
+
+/** Parses a comparison operator inside a SOME / ALL predicate. */
+SqlKind comp() :
+{
+}
+{
+    <LT> { return SqlKind.LESS_THAN; }
+|
+    <LE> { return SqlKind.LESS_THAN_OR_EQUAL; }
+|
+    <GT> { return SqlKind.GREATER_THAN; }
+|
+    <GE> { return SqlKind.GREATER_THAN_OR_EQUAL; }
+|
+    <EQ> { return SqlKind.EQUALS; }
+|
+    <NE> { return SqlKind.NOT_EQUALS; }
+|
+    <NE2> {
+        if (!this.conformance.isBangEqualAllowed()) {
+            throw SqlUtil.newContextException(getPos(), RESOURCE.bangEqualNotAllowed());
+        }
+        return SqlKind.NOT_EQUALS;
+    }
+}
+
+/**
+ * Parses a unary row expression, or a parenthesized expression of any
+ * kind.
+ */
+SqlNode Expression3(ExprContext exprContext) :
+{
+    final SqlNode e;
+    final SqlNodeList list;
+    final SqlNodeList list1;
+    final Span s;
+    final Span rowSpan;
+}
+{
+    LOOKAHEAD(2)
+    e = AtomicRowExpression()
+    {
+        checkNonQueryExpression(exprContext);
+        return e;
+    }
+|
+    e = CursorExpression(exprContext) { return e; }
+|
+    LOOKAHEAD(3)
+    <ROW> {
+        s = span();
+    }
+    list = ParenthesizedQueryOrCommaList(exprContext) {
+        if (exprContext != ExprContext.ACCEPT_ALL
+            && exprContext != ExprContext.ACCEPT_CURSOR
+            && !this.conformance.allowExplicitRowValueConstructor())
+        {
+            throw SqlUtil.newContextException(s.end(list),
+                RESOURCE.illegalRowExpression());
+        }
+        return SqlStdOperatorTable.ROW.createCall(list);
+    }
+|
+    (
+        <ROW> { rowSpan = span(); }
+    |   { rowSpan = null; }
+    )
+    list1 = ParenthesizedQueryOrCommaList(exprContext) {
+        if (rowSpan != null) {
+            // interpret as row constructor
+            return SqlStdOperatorTable.ROW.createCall(rowSpan.end(list1),
+                (List<SqlNode>) list1);
+        }
+    }
+    [
+        LOOKAHEAD(2)
+        /* TODO:
+        (
+            op = periodOperator()
+            list2 = ParenthesizedQueryOrCommaList(exprContext)
+            {
+                if (list1.size() != 2 || list2.size() != 2) {
+                    throw SqlUtil.newContextException(
+                        list1.getParserPosition().plus(
+                            list2.getParserPosition()),
+                        RESOURCE.illegalOverlaps());
+                }
+                for (SqlNode node : list2) {
+                    list1.add(node);
+                }
+                return op.createCall(
+                    list1.getParserPosition().plus(list2.getParserPosition()),
+                    list1.toArray());
+            }
+        )
+    |
+        */
+        (
+            e = IntervalQualifier()
+            {
+                if ((list1.size() == 1)
+                    && list1.get(0) instanceof SqlCall)
+                {
+                    final SqlCall call = (SqlCall) list1.get(0);
+                    if (call.getKind() == SqlKind.MINUS
+                            && call.operandCount() == 2) {
+                        return SqlStdOperatorTable.MINUS_DATE.createCall(
+                            Span.of(list1).end(this), call.operand(0),
+                            call.operand(1), e);
+                     }
+                }
+                throw SqlUtil.newContextException(span().end(list1),
+                    RESOURCE.illegalMinusDate());
+            }
+        )
+    ]
+    {
+        if (list1.size() == 1) {
+            // interpret as single value or query
+            return list1.get(0);
+        } else {
+            // interpret as row constructor
+            return SqlStdOperatorTable.ROW.createCall(span().end(list1),
+                (List<SqlNode>) list1);
+        }
+    }
+}
+
+SqlOperator periodOperator() :
+{
+}
+{
+     <OVERLAPS> { return SqlStdOperatorTable.OVERLAPS; }
+|
+     LOOKAHEAD(2)
+     <IMMEDIATELY> <PRECEDES> { return SqlStdOperatorTable.IMMEDIATELY_PRECEDES; }
+|
+     <PRECEDES> { return SqlStdOperatorTable.PRECEDES; }
+|
+     <IMMEDIATELY> <SUCCEEDS> { return SqlStdOperatorTable.IMMEDIATELY_SUCCEEDS; }
+|
+     <SUCCEEDS> { return SqlStdOperatorTable.SUCCEEDS; }
+|
+     <EQUALS> { return SqlStdOperatorTable.PERIOD_EQUALS; }
+}
+
+/**
+ * Parses a COLLATE clause
+ */
+SqlCollation CollateClause() :
+{
+}
+{
+    <COLLATE> <COLLATION_ID>
+    {
+        return new SqlCollation(
+            getToken(0).image, SqlCollation.Coercibility.EXPLICIT);
+    }
+}
+
+/**
+ * Numeric literal or parameter; used in LIMIT, OFFSET and FETCH clauses.
+ */
+SqlNode UnsignedNumericLiteralOrParam() :
+{
+    final SqlNode e;
+}
+{
+    (
+        e = UnsignedNumericLiteral()
+    |
+        e = DynamicParam()
+    )
+    { return e; }
+}
+
+/**
+ * Parses a row expression extension, it can be either an identifier,
+ * or a call to a named function.
+ */
+SqlNode RowExpressionExtension() :
+{
+    final SqlFunctionCategory funcType = SqlFunctionCategory.USER_DEFINED_FUNCTION;
+    final SqlIdentifier p;
+    final Span s;
+    final List<SqlNode> args;
+    final SqlLiteral quantifier;
+}
+{
+    p = SimpleIdentifier()
+    (
+        LOOKAHEAD( <LPAREN> ) { s = span(); }
+        (
+            LOOKAHEAD(2) <LPAREN> <STAR> {
+                quantifier = null;
+                args = ImmutableList.of(SqlIdentifier.star(getPos()));
+            }
+            <RPAREN>
+        |
+            LOOKAHEAD(2) <LPAREN> <RPAREN> {
+                quantifier = null;
+                args = ImmutableList.of();
+            }
+        |
+            args = FunctionParameterList(ExprContext.ACCEPT_SUB_QUERY) {
+                quantifier = (SqlLiteral) args.get(0);
+                args.remove(0);
+            }
+        )
+        { return createCall(p, s.end(this), funcType, quantifier, args); }
+    |
+        { return p; }
+    )
+}
+
+/**
+ * Parses a call to the STRING_AGG aggregate function (or to an aggregate
+ * function with similar syntax: ARRAY_AGG, ARRAY_CONCAT_AGG, GROUP_CONCAT).
+ */
+SqlCall StringAggFunctionCall() :
+{
+    final Span s, s2;
+    final SqlOperator op;
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final SqlLiteral qualifier;
+    final SqlNodeList orderBy;
+    final Pair<SqlParserPos, SqlOperator> nullTreatment;
+    final SqlNode separator;
+}
+{
+    (
+        <ARRAY_AGG> { s = span(); op = SqlLibraryOperators.ARRAY_AGG; }
+    |   <ARRAY_CONCAT_AGG> { s = span(); op = SqlLibraryOperators.ARRAY_CONCAT_AGG; }
+    |   <GROUP_CONCAT> { s = span(); op = SqlLibraryOperators.GROUP_CONCAT; }
+    |   <STRING_AGG> { s = span(); op = SqlLibraryOperators.STRING_AGG; }
+    )
+    <LPAREN>
+    (
+        qualifier = AllOrDistinct()
+    |   { qualifier = null; }
+    )
+    AddArg(args, ExprContext.ACCEPT_SUB_QUERY)
+    (
+        <COMMA> {
+            // a comma-list can't appear where only a query is expected
+            // TODO: the following line is a no-op; remove it?
+            checkNonQueryExpression(ExprContext.ACCEPT_SUB_QUERY);
+        }
+        AddArg(args, ExprContext.ACCEPT_SUB_QUERY)
+    )*
+    (
+        nullTreatment = NullTreatment()
+    |   { nullTreatment = null; }
+    )
+    [
+        orderBy = OrderBy(true) {
+            args.add(orderBy);
+        }
+    ]
+    [
+        <SEPARATOR> { s2 = span(); } separator = StringLiteral() {
+            args.add(SqlInternalOperators.SEPARATOR.createCall(s2.end(this), separator));
+        }
+    ]
+    <RPAREN>
+    {
+        SqlCall call = op.createCall(qualifier, s.end(this), args);
+        if (nullTreatment != null) {
+            // Wrap in RESPECT_NULLS or IGNORE_NULLS.
+            call = nullTreatment.right.createCall(nullTreatment.left, call);
+        }
+        return call;
+    }
+}
+
+/**
+ * Parses an atomic row expression.
+ */
+SqlNode AtomicRowExpression() :
+{
+    final SqlNode e;
+}
+{
+    (
+        LOOKAHEAD(2)
+        e = LiteralOrIntervalExpression()
+    |
+        e = DynamicParam()
+    |
+        LOOKAHEAD(2)
+        e = BuiltinFunctionCall()
+    |
+        e = JdbcFunctionCall()
+    |
+        e = MultisetConstructor()
+    |
+        e = ArrayConstructor()
+    |
+        LOOKAHEAD(3)
+        e = MapConstructor()
+    |
+        e = PeriodConstructor()
+    |
+        // NOTE jvs 18-Jan-2005:  use syntactic lookahead to discriminate
+        // compound identifiers from function calls in which the function
+        // name is a compound identifier
+        LOOKAHEAD( [<SPECIFIC>] FunctionName() <LPAREN>)
+        e = NamedFunctionCall()
+    |
+        e = ContextVariable()
+    |
+        e = CompoundIdentifier()
+    |
+        e = NewSpecification()
+    |
+        e = CaseExpression()
+    |
+        e = SequenceExpression()
+    )
+    { return e; }
+}
+
+SqlNode CaseExpression() :
+{
+    final Span whenSpan = Span.of();
+    final Span thenSpan = Span.of();
+    final Span s;
+    SqlNode e;
+    final SqlNode caseIdentifier;
+    final SqlNode elseClause;
+    final List<SqlNode> whenList = new ArrayList<SqlNode>();
+    final List<SqlNode> thenList = new ArrayList<SqlNode>();
+}
+{
+    <CASE> { s = span(); }
+    (
+        caseIdentifier = Expression(ExprContext.ACCEPT_SUB_QUERY)
+    |   { caseIdentifier = null; }
+    )
+    (
+        <WHEN> { whenSpan.add(this); }
+        e = ExpressionCommaList(s, ExprContext.ACCEPT_SUB_QUERY) {
+            if (((SqlNodeList) e).size() == 1) {
+                e = ((SqlNodeList) e).get(0);
+            }
+            whenList.add(e);
+        }
+        <THEN> { thenSpan.add(this); }
+        e = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+            thenList.add(e);
+        }
+    )+
+    (
+        <ELSE> elseClause = Expression(ExprContext.ACCEPT_SUB_QUERY)
+    |   { elseClause = null; }
+    )
+    <END> {
+        return SqlCase.createSwitched(s.end(this), caseIdentifier,
+            new SqlNodeList(whenList, whenSpan.addAll(whenList).pos()),
+            new SqlNodeList(thenList, thenSpan.addAll(thenList).pos()),
+            elseClause);
+    }
+}
+
+SqlCall SequenceExpression() :
+{
+    final Span s;
+    final SqlOperator f;
+    final SqlNode sequenceRef;
+}
+{
+    (
+        <NEXT> { f = SqlStdOperatorTable.NEXT_VALUE; s = span(); }
+    |
+        LOOKAHEAD(3)
+        <CURRENT> { f = SqlStdOperatorTable.CURRENT_VALUE; s = span(); }
+    )
+    <VALUE> <FOR> sequenceRef = CompoundIdentifier() {
+        return f.createCall(s.end(sequenceRef), sequenceRef);
+    }
+}
+
+/**
+ * Parses "SET &lt;NAME&gt; = VALUE" or "RESET &lt;NAME&gt;", without a leading
+ * "ALTER &lt;SCOPE&gt;".
+ */
+SqlSetOption SqlSetOption(Span s, String scope) :
+{
+    SqlIdentifier name;
+    final SqlNode val;
+}
+{
+    (
+        <SET> {
+            s.add(this);
+        }
+        name = CompoundIdentifier()
+        <EQ>
+        (
+            val = Literal()
+        |
+            val = SimpleIdentifier()
+        |
+            <ON> {
+                // OFF is handled by SimpleIdentifier, ON handled here.
+                val = new SqlIdentifier(token.image.toUpperCase(Locale.ROOT),
+                    getPos());
+            }
+        )
+        {
+            return new SqlSetOption(s.end(val), scope, name, val);
+        }
+    |
+        <RESET> {
+            s.add(this);
+        }
+        (
+            name = CompoundIdentifier()
+        |
+            <ALL> {
+                name = new SqlIdentifier(token.image.toUpperCase(Locale.ROOT),
+                    getPos());
+            }
+        )
+        {
+            return new SqlSetOption(s.end(name), scope, name, null);
+        }
+    )
+}
+
+/**
+ * Parses an expression for setting or resetting an option in SQL, such as QUOTED_IDENTIFIERS,
+ * or explain plan level (physical/logical).
+ */
+SqlAlter SqlAlter() :
+{
+    final Span s;
+    final String scope;
+    final SqlAlter alterNode;
+}
+{
+    <ALTER> { s = span(); }
+    scope = Scope()
+    (
+<#-- additional literal parser methods are included here -->
+<#list (parser.alterStatementParserMethods!default.parser.alterStatementParserMethods) as method>
+        alterNode = ${method}(s, scope)
+    |
+</#list>
+
+        alterNode = SqlSetOption(s, scope)
+    )
+    {
+        return alterNode;
+    }
+}
+
+String Scope() :
+{
+}
+{
+    ( <SYSTEM> | <SESSION> ) { return token.image.toUpperCase(Locale.ROOT); }
+}
+
+<#if (parser.createStatementParserMethods!default.parser.createStatementParserMethods)?size != 0>
+/**
+ * Parses a CREATE statement.
+ */
+SqlCreate SqlCreate() :
+{
+    final Span s;
+    boolean replace = false;
+    final SqlCreate create;
+}
+{
+    <CREATE> { s = span(); }
+    [
+        <OR> <REPLACE> {
+            replace = true;
+        }
+    ]
+    (
+<#-- additional literal parser methods are included here -->
+<#list (parser.createStatementParserMethods!default.parser.createStatementParserMethods) as method>
+        create = ${method}(s, replace)
+        <#sep>| LOOKAHEAD(2) </#sep>
+</#list>
+    )
+    {
+        return create;
+    }
+}
+</#if>
+
+<#if (parser.dropStatementParserMethods!default.parser.dropStatementParserMethods)?size != 0>
+/**
+ * Parses a DROP statement.
+ */
+SqlDrop SqlDrop() :
+{
+    final Span s;
+    boolean replace = false;
+    final SqlDrop drop;
+}
+{
+    <DROP> { s = span(); }
+    (
+<#-- additional literal parser methods are included here -->
+<#list (parser.dropStatementParserMethods!default.parser.dropStatementParserMethods) as method>
+        drop = ${method}(s, replace)
+        <#sep>|</#sep>
+</#list>
+    )
+    {
+        return drop;
+    }
+}
+</#if>
+
+/**
+ * Parses a literal expression, allowing continued string literals.
+ * Usually returns an SqlLiteral, but a continued string literal
+ * is an SqlCall expression, which concatenates 2 or more string
+ * literals; the validator reduces this.
+ *
+ * <p>If the context allows both literals and expressions,
+ * use {@link #LiteralOrIntervalExpression}, which requires less
+ * lookahead.
+ */
+SqlNode Literal() :
+{
+    SqlNode e;
+}
+{
+    (
+        e = NonIntervalLiteral()
+    |
+        e = IntervalLiteral()
+    )
+    { return e; }
+}
+
+/** Parses a literal that is not an interval literal. */
+SqlNode NonIntervalLiteral() :
+{
+    final SqlNode e;
+}
+{
+    (
+        e = NumericLiteral()
+    |
+        e = StringLiteral()
+    |
+        e = SpecialLiteral()
+    |
+        e = DateTimeLiteral()
+<#-- additional literal parser methods are included here -->
+<#list (parser.literalParserMethods!default.parser.literalParserMethods) as method>
+    |
+        e = ${method}
+</#list>
+    )
+    {
+        return e;
+    }
+}
+
+/** Parses a literal or an interval expression.
+ *
+ * <p>We include them in the same production because it is difficult to
+ * distinguish interval literals from interval expression (both of which
+ * start with the {@code INTERVAL} keyword); this way, we can use less
+ * LOOKAHEAD. */
+SqlNode LiteralOrIntervalExpression() :
+{
+    final SqlNode e;
+}
+{
+    (
+        e = IntervalLiteralOrExpression()
+    |
+        e = NonIntervalLiteral()
+    )
+    { return e; }
+}
+
+/** Parses a unsigned numeric literal */
+SqlNumericLiteral UnsignedNumericLiteral() :
+{
+}
+{
+    <UNSIGNED_INTEGER_LITERAL> {
+        return SqlLiteral.createExactNumeric(token.image, getPos());
+    }
+|
+    <DECIMAL_NUMERIC_LITERAL> {
+        return SqlLiteral.createExactNumeric(token.image, getPos());
+    }
+|
+    <APPROX_NUMERIC_LITERAL> {
+        return SqlLiteral.createApproxNumeric(token.image, getPos());
+    }
+}
+
+/** Parses a numeric literal (can be signed) */
+SqlLiteral NumericLiteral() :
+{
+    final SqlNumericLiteral num;
+    final Span s;
+}
+{
+    <PLUS> num = UnsignedNumericLiteral() {
+        return num;
+    }
+|
+    <MINUS> { s = span(); } num = UnsignedNumericLiteral() {
+        return SqlLiteral.createNegative(num, s.end(this));
+    }
+|
+    num = UnsignedNumericLiteral() {
+        return num;
+    }
+}
+
+/** Parse a special literal keyword */
+SqlLiteral SpecialLiteral() :
+{
+}
+{
+    <TRUE> { return SqlLiteral.createBoolean(true, getPos()); }
+|
+    <FALSE> { return SqlLiteral.createBoolean(false, getPos()); }
+|
+    <UNKNOWN> { return SqlLiteral.createUnknown(getPos()); }
+|
+    <NULL> { return SqlLiteral.createNull(getPos()); }
+}
+
+/**
+ * Parses a string literal. The literal may be continued onto several
+ * lines.  For a simple literal, the result is an SqlLiteral.  For a continued
+ * literal, the result is an SqlCall expression, which concatenates 2 or more
+ * string literals; the validator reduces this.
+ *
+ * @see SqlLiteral#unchain(SqlNode)
+ * @see SqlLiteral#stringValue(SqlNode)
+ *
+ * @return a literal expression
+ */
+SqlNode StringLiteral() :
+{
+    String p;
+    final List<SqlLiteral> frags;
+    char unicodeEscapeChar = 0;
+    String charSet = null;
+    SqlCharStringLiteral literal;
+}
+{
+    // A continued string literal consists of a head fragment and one or more
+    // tail fragments. Since comments may occur between the fragments, and
+    // comments are special tokens, each fragment is a token. But since spaces
+    // or comments may not occur between the prefix and the first quote, the
+    // head fragment, with any prefix, is one token.
+
+    <BINARY_STRING_LITERAL>
+    {
+        frags = new ArrayList<SqlLiteral>();
+        try {
+            p = SqlParserUtil.trim(token.image, "xX'");
+            frags.add(SqlLiteral.createBinaryString(p, getPos()));
+        } catch (NumberFormatException ex) {
+            throw SqlUtil.newContextException(getPos(),
+                RESOURCE.illegalBinaryString(token.image));
+        }
+    }
+    (
+        // The grammar is ambiguous when a continued literals and a character
+        // string alias are both possible. For example, in
+        //   SELECT x'01'\n'ab'
+        // we prefer that 'ab' continues the literal, and is not an alias.
+        // The following LOOKAHEAD mutes the warning about ambiguity.
+        LOOKAHEAD(1)
+        <QUOTED_STRING>
+        {
+            try {
+                p = SqlParserUtil.trim(token.image, "'"); // no embedded quotes
+                frags.add(SqlLiteral.createBinaryString(p, getPos()));
+            } catch (NumberFormatException ex) {
+                throw SqlUtil.newContextException(getPos(),
+                    RESOURCE.illegalBinaryString(token.image));
+            }
+        }
+    )*
+    {
+        assert !frags.isEmpty();
+        if (frags.size() == 1) {
+            return frags.get(0); // just the head fragment
+        } else {
+            SqlParserPos pos2 = SqlParserPos.sum(frags);
+            return SqlStdOperatorTable.LITERAL_CHAIN.createCall(pos2, frags);
+        }
+    }
+|
+    (
+        <PREFIXED_STRING_LITERAL>
+        { charSet = SqlParserUtil.getCharacterSet(token.image); }
+    |   <QUOTED_STRING>
+    |   <UNICODE_STRING_LITERAL> {
+            // TODO jvs 2-Feb-2009:  support the explicit specification of
+            // a character set for Unicode string literals, per SQL:2003
+            unicodeEscapeChar = BACKSLASH;
+            charSet = "UTF16";
+        }
+    )
+    {
+        frags = new ArrayList<SqlLiteral>();
+        p = SqlParserUtil.parseString(token.image);
+        try {
+            literal = SqlLiteral.createCharString(p, charSet, getPos());
+            frags.add(literal);
+        } catch (java.nio.charset.UnsupportedCharsetException e) {
+            throw SqlUtil.newContextException(getPos(),
+                RESOURCE.unknownCharacterSet(charSet));
+        }
+    }
+    (
+        // The grammar is ambiguous when a continued literals and a character
+        // string alias are both possible. For example, in
+        //   SELECT 'taxi'\n'cab'
+        // we prefer that 'cab' continues the literal, and is not an alias.
+        // The following LOOKAHEAD mutes the warning about ambiguity.
+        LOOKAHEAD(1)
+        <QUOTED_STRING>
+        {
+            p = SqlParserUtil.parseString(token.image);
+            try {
+                literal = SqlLiteral.createCharString(p, charSet, getPos());
+                frags.add(literal);
+            } catch (java.nio.charset.UnsupportedCharsetException e) {
+                throw SqlUtil.newContextException(getPos(),
+                    RESOURCE.unknownCharacterSet(charSet));
+            }
+        }
+    )*
+    [
+        <UESCAPE> <QUOTED_STRING>
+        {
+            if (unicodeEscapeChar == 0) {
+                throw SqlUtil.newContextException(getPos(),
+                    RESOURCE.unicodeEscapeUnexpected());
+            }
+            String s = SqlParserUtil.parseString(token.image);
+            unicodeEscapeChar = SqlParserUtil.checkUnicodeEscapeChar(s);
+        }
+    ]
+    {
+        assert !frags.isEmpty();
+        if (frags.size() == 1) {
+            // just the head fragment
+            SqlLiteral lit = (SqlLiteral) frags.get(0);
+            return lit.unescapeUnicode(unicodeEscapeChar);
+        } else {
+            SqlNode[] rands = (SqlNode[]) frags.toArray(new SqlNode[0]);
+            for (int i = 0; i < rands.length; ++i) {
+                rands[i] = ((SqlLiteral) rands[i]).unescapeUnicode(
+                    unicodeEscapeChar);
+            }
+            SqlParserPos pos2 = SqlParserPos.sum(rands);
+            return SqlStdOperatorTable.LITERAL_CHAIN.createCall(pos2, rands);
+        }
+    }
+|
+    <C_STYLE_ESCAPED_STRING_LITERAL>
+    {
+        try {
+            p = SqlParserUtil.parseCString(getToken(0).image);
+        } catch (SqlParserUtil.MalformedUnicodeEscape e) {
+            throw SqlUtil.newContextException(getPos(),
+                RESOURCE.unicodeEscapeMalformed(e.i));
+       }
+       return SqlLiteral.createCharString(p, "UTF16", getPos());
+    }
+|
+    <BIG_QUERY_DOUBLE_QUOTED_STRING>
+    {
+        p = SqlParserUtil.stripQuotes(getToken(0).image, DQ, DQ, "\\\"",
+            Casing.UNCHANGED);
+        try {
+            return SqlLiteral.createCharString(p, charSet, getPos());
+        } catch (java.nio.charset.UnsupportedCharsetException e) {
+            throw SqlUtil.newContextException(getPos(),
+                RESOURCE.unknownCharacterSet(charSet));
+        }
+    }
+|
+    <BIG_QUERY_QUOTED_STRING>
+    {
+        p = SqlParserUtil.stripQuotes(getToken(0).image, "'", "'", "\\'",
+            Casing.UNCHANGED);
+        try {
+            return SqlLiteral.createCharString(p, charSet, getPos());
+        } catch (java.nio.charset.UnsupportedCharsetException e) {
+            throw SqlUtil.newContextException(getPos(),
+                RESOURCE.unknownCharacterSet(charSet));
+        }
+    }
+}
+
+/** Parses a character literal.
+ * Matches a single-quoted string, such as 'foo';
+ * on BigQuery also matches a double-quoted string, such as "foo".
+ * Returns the value of the string with quotes removed. */
+String SimpleStringLiteral() :
+{
+}
+{
+    <QUOTED_STRING> {
+        return SqlParserUtil.parseString(token.image);
+    }
+|
+    <BIG_QUERY_QUOTED_STRING> {
+        return SqlParserUtil.stripQuotes(token.image, "'", "'", "\\'", Casing.UNCHANGED);
+    }
+|
+    <BIG_QUERY_DOUBLE_QUOTED_STRING> {
+        return SqlParserUtil.stripQuotes(token.image, DQ, DQ, "\\\"", Casing.UNCHANGED);
+    }
+}
+
+/**
+ * Parses a date/time literal.
+ */
+SqlLiteral DateTimeLiteral() :
+{
+    final String p;
+    final Span s;
+}
+{
+    <LBRACE_D> <QUOTED_STRING> {
+        p = SqlParserUtil.parseString(token.image);
+    }
+    <RBRACE> {
+        return SqlParserUtil.parseDateLiteral(p, getPos());
+    }
+|
+    <LBRACE_T> <QUOTED_STRING> {
+        p = SqlParserUtil.parseString(token.image);
+    }
+    <RBRACE> {
+        return SqlParserUtil.parseTimeLiteral(p, getPos());
+    }
+|
+    <LBRACE_TS> { s = span(); } <QUOTED_STRING> {
+        p = SqlParserUtil.parseString(token.image);
+    }
+    <RBRACE> {
+        return SqlParserUtil.parseTimestampLiteral(p, s.end(this));
+    }
+|
+    <DATE> { s = span(); } p = SimpleStringLiteral() {
+      return SqlLiteral.createUnknown("DATE", p, s.end(this));
+    }
+|
+    <DATETIME> { s = span(); } p = SimpleStringLiteral() {
+        return SqlLiteral.createUnknown("DATETIME", p, s.end(this));
+    }
+|
+    <TIME> { s = span(); } p = SimpleStringLiteral() {
+      return SqlLiteral.createUnknown("TIME", p, s.end(this));
+    }
+|
+    LOOKAHEAD(2)
+    <TIMESTAMP> { s = span(); } p = SimpleStringLiteral() {
+        return SqlLiteral.createUnknown("TIMESTAMP", p, s.end(this));
+    }
+|
+    <TIMESTAMP> { s = span(); } <WITH> <LOCAL> <TIME> <ZONE> p = SimpleStringLiteral() {
+        return SqlLiteral.createUnknown("TIMESTAMP WITH LOCAL TIME ZONE", p, s.end(this));
+    }
+}
+
+/** Parses a Date/Time constructor function, for example "DATE(1969, 7, 21)"
+ * or "DATETIME(d, t)". Enabled in some libraries (e.g. BigQuery). */
+SqlNode DateTimeConstructorCall() :
+{
+    final SqlFunctionCategory funcType = SqlFunctionCategory.TIMEDATE;
+    final SqlIdentifier qualifiedName;
+    final Span s;
+    final SqlLiteral quantifier;
+    final List<? extends SqlNode> args;
+}
+{
+    (<DATE> | <TIME> | <DATETIME> | <TIMESTAMP>) {
+        s = span();
+        qualifiedName = new SqlIdentifier(unquotedIdentifier(), getPos());
+    }
+    args = FunctionParameterList(ExprContext.ACCEPT_SUB_QUERY) {
+        quantifier = (SqlLiteral) args.get(0);
+        args.remove(0);
+        return createCall(qualifiedName, s.end(this), funcType, quantifier, args);
+    }
+}
+
+/** Parses a MULTISET constructor */
+SqlNode MultisetConstructor() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    SqlNode e;
+    final Span s;
+}
+{
+    <MULTISET> { s = span(); }
+    (
+        LOOKAHEAD(2)
+        <LPAREN>
+        // by sub query "MULTISET(SELECT * FROM T)"
+        e = LeafQueryOrExpr(ExprContext.ACCEPT_QUERY)
+        <RPAREN> {
+            return SqlStdOperatorTable.MULTISET_QUERY.createCall(
+                s.end(this), e);
+        }
+    |
+        // by enumeration "MULTISET[e0, e1, ..., eN]"
+        <LBRACKET> // TODO: do trigraph as well ??( ??)
+        AddExpression(args, ExprContext.ACCEPT_NON_QUERY)
+        ( <COMMA> AddExpression(args, ExprContext.ACCEPT_NON_QUERY) )*
+        <RBRACKET>
+        {
+            return SqlStdOperatorTable.MULTISET_VALUE.createCall(
+                s.end(this), args);
+        }
+    )
+}
+
+/** Parses an ARRAY constructor */
+SqlNode ArrayConstructor() :
+{
+    SqlNodeList args;
+    SqlNode e;
+    final Span s;
+    final String p;
+}
+{
+    <ARRAY> { s = span(); }
+    (
+        LOOKAHEAD(1)
+        <LPAREN>
+        // by sub query "MULTISET(SELECT * FROM T)"
+        e = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY)
+        <RPAREN>
+        {
+            return SqlStdOperatorTable.ARRAY_QUERY.createCall(
+                s.end(this), e);
+        }
+    |
+        // by enumeration "ARRAY[e0, e1, ..., eN]"
+        <LBRACKET> // TODO: do trigraph as well ??( ??)
+        (
+            args = ExpressionCommaList(s, ExprContext.ACCEPT_NON_QUERY)
+        |
+            { args = SqlNodeList.EMPTY; }
+        )
+        <RBRACKET>
+        {
+            return SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR.createCall(
+                s.end(this), args.getList());
+        }
+<#if (parser.includeParsingStringLiteralAsArrayLiteral!default.parser.includeParsingStringLiteralAsArrayLiteral) >
+    |
+        p = SimpleStringLiteral() {
+            try {
+                return SqlParserUtil.parseArrayLiteral(p);
+            } catch (SqlParseException ex) {
+                throw SqlUtil.newContextException(getPos(),
+                    RESOURCE.illegalArrayExpression(p));
+            }
+        }
+</#if>
+    )
+}
+
+SqlCall ArrayLiteral() :
+{
+    final List<SqlNode> list;
+    SqlNode e;
+    final Span s;
+}
+{
+    <LBRACE> { s = span(); }
+    (
+        e = Literal() { list = startList(e); }
+        ( <COMMA> e = Literal() { list.add(e); } )*
+    |
+        e = ArrayLiteral() { list = startList(e); }
+        ( <COMMA> e = ArrayLiteral() { list.add(e); } )*
+    |
+        { list = Collections.emptyList(); }
+    )
+    <RBRACE> {
+       return SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR.createCall(s.end(this), list);
+    }
+}
+
+/** Parses a MAP constructor */
+SqlNode MapConstructor() :
+{
+    SqlNodeList args;
+    SqlNode e;
+    final Span s;
+}
+{
+    <MAP> { s = span(); }
+    (
+        LOOKAHEAD(1)
+        <LPAREN>
+        // by sub query "MAP (SELECT empno, deptno FROM emp)"
+        e = LeafQueryOrExpr(ExprContext.ACCEPT_QUERY)
+        <RPAREN>
+        {
+            return SqlStdOperatorTable.MAP_QUERY.createCall(
+                s.end(this), e);
+        }
+    |
+        // by enumeration "MAP[k0, v0, ..., kN, vN]"
+        <LBRACKET> // TODO: do trigraph as well ??( ??)
+        (
+            args = ExpressionCommaList(s, ExprContext.ACCEPT_NON_QUERY)
+        |
+            { args = SqlNodeList.EMPTY; }
+        )
+        <RBRACKET>
+        {
+            return SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR.createCall(
+                s.end(this), args.getList());
+        }
+    )
+}
+
+/** Parses a PERIOD constructor */
+SqlNode PeriodConstructor() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final Span s;
+}
+{
+    <PERIOD> { s = span(); }
+    <LPAREN>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <RPAREN> {
+        return SqlStdOperatorTable.ROW.createCall(s.end(this), args);
+    }
+}
+
+/**
+ * Parses an interval literal.
+ */
+SqlLiteral IntervalLiteral() :
+{
+    final String p;
+    final SqlIntervalQualifier intervalQualifier;
+    int sign = 1;
+    final Span s;
+}
+{
+    <INTERVAL> { s = span(); }
+    [
+        <MINUS> { sign = -1; }
+    |
+        <PLUS> { sign = 1; }
+    ]
+    p = SimpleStringLiteral()
+    intervalQualifier = IntervalQualifier() {
+        return SqlParserUtil.parseIntervalLiteral(s.end(intervalQualifier),
+            sign, p, intervalQualifier);
+    }
+}
+
+/** Parses an interval literal (e.g. {@code INTERVAL '2:3' HOUR TO MINUTE})
+ * or an interval expression (e.g. {@code INTERVAL emp.empno MINUTE}
+ * or {@code INTERVAL 3 MONTHS}). */
+SqlNode IntervalLiteralOrExpression() :
+{
+    final String p;
+    final SqlIntervalQualifier intervalQualifier;
+    int sign = 1;
+    final Span s;
+    SqlNode e;
+}
+{
+    <INTERVAL> { s = span(); }
+    [
+        <MINUS> { sign = -1; }
+    |
+        <PLUS> { sign = 1; }
+    ]
+    (
+        // literal (with quoted string)
+        p = SimpleStringLiteral()
+        intervalQualifier = IntervalQualifier() {
+            return SqlParserUtil.parseIntervalLiteral(s.end(intervalQualifier),
+                sign, p, intervalQualifier);
+        }
+    |
+        // To keep parsing simple, any expressions besides numeric literal and
+        // identifiers must be enclosed in parentheses.
+        (
+            <LPAREN>
+            e = Expression(ExprContext.ACCEPT_SUB_QUERY)
+            <RPAREN>
+        |
+            e = UnsignedNumericLiteral()
+        |
+            e = CompoundIdentifier()
+        )
+        intervalQualifier = IntervalQualifierStart() {
+            if (sign == -1) {
+                e = SqlStdOperatorTable.UNARY_MINUS.createCall(e.getParserPosition(), e);
+            }
+            return SqlStdOperatorTable.INTERVAL.createCall(s.end(this), e,
+                intervalQualifier);
+        }
+    )
+}
+
+TimeUnit Year() :
+{
+}
+{
+    <YEAR> { return TimeUnit.YEAR; }
+|
+    <YEARS> { return warn(TimeUnit.YEAR); }
+}
+
+TimeUnit Quarter() :
+{
+}
+{
+    <QUARTER> { return TimeUnit.QUARTER; }
+|
+    <QUARTERS> { return warn(TimeUnit.QUARTER); }
+}
+
+TimeUnit Month() :
+{
+}
+{
+    <MONTH> { return TimeUnit.MONTH; }
+|
+    <MONTHS> { return warn(TimeUnit.MONTH); }
+}
+
+TimeUnit Week() :
+{
+}
+{
+    <WEEK> { return TimeUnit.WEEK; }
+|
+    <WEEKS> { return warn(TimeUnit.WEEK); }
+}
+
+TimeUnit Day() :
+{
+}
+{
+    <DAY> { return TimeUnit.DAY; }
+|
+    <DAYS> { return warn(TimeUnit.DAY); }
+}
+
+TimeUnit Hour() :
+{
+}
+{
+    <HOUR> { return TimeUnit.HOUR; }
+|
+    <HOURS> { return warn(TimeUnit.HOUR); }
+}
+
+TimeUnit Minute() :
+{
+}
+{
+    <MINUTE> { return TimeUnit.MINUTE; }
+|
+    <MINUTES> { return warn(TimeUnit.MINUTE); }
+}
+
+TimeUnit Second() :
+{
+}
+{
+    <SECOND> { return TimeUnit.SECOND; }
+|
+    <SECONDS> { return warn(TimeUnit.SECOND); }
+}
+
+SqlIntervalQualifier IntervalQualifier() :
+{
+    final Span s;
+    final TimeUnit start;
+    final TimeUnit end;
+    final int startPrec;
+    int secondFracPrec = RelDataType.PRECISION_NOT_SPECIFIED;
+}
+{
+    (
+        start = Year() { s = span(); } startPrec = PrecisionOpt()
+        (
+            LOOKAHEAD(2) <TO> end = Month()
+        |   { end = null; }
+        )
+    |
+        start = Quarter() { s = span(); } startPrec = PrecisionOpt()
+        { end = null; }
+    |
+        start = Month() { s = span(); } startPrec = PrecisionOpt()
+        { end = null; }
+    |
+        start = Week() { s = span(); } startPrec = PrecisionOpt()
+        { end = null; }
+    |
+        start = Day() { s = span(); } startPrec = PrecisionOpt()
+        (
+            LOOKAHEAD(2) <TO>
+            (
+                end = Hour()
+            |
+                end = Minute()
+            |
+                end = Second() secondFracPrec = PrecisionOpt()
+            )
+        |   { end = null; }
+        )
+    |
+        start = Hour() { s = span(); } startPrec = PrecisionOpt()
+        (
+            LOOKAHEAD(2) <TO>
+            (
+                end = Minute()
+            |
+                end = Second()
+                [ <LPAREN> secondFracPrec = UnsignedIntLiteral() <RPAREN> ]
+            )
+        |   { end = null; }
+        )
+    |
+        start = Minute() { s = span(); } startPrec = PrecisionOpt()
+        (
+            LOOKAHEAD(2) <TO>
+            end = Second()
+            [ <LPAREN> secondFracPrec = UnsignedIntLiteral() <RPAREN> ]
+        |   { end = null; }
+        )
+    |
+        start = Second() { s = span(); }
+        (
+            <LPAREN> startPrec = UnsignedIntLiteral()
+            [ <COMMA> secondFracPrec = UnsignedIntLiteral() ]
+            <RPAREN>
+        |   { startPrec = -1; }
+        )
+        { end = null; }
+    )
+    {
+        return new SqlIntervalQualifier(start, startPrec, end, secondFracPrec,
+            s.end(this));
+    }
+}
+
+/** Interval qualifier without 'TO unit'. */
+SqlIntervalQualifier IntervalQualifierStart() :
+{
+    final Span s;
+    final TimeUnit start;
+    int startPrec = RelDataType.PRECISION_NOT_SPECIFIED;
+    int secondFracPrec = RelDataType.PRECISION_NOT_SPECIFIED;
+}
+{
+    (
+        (
+            start = Year()
+        |   start = Quarter()
+        |   start = Month()
+        |   start = Week()
+        |   start = Day()
+        |   start = Hour()
+        |   start = Minute()
+        )
+        { s = span(); }
+        startPrec = PrecisionOpt()
+    |
+        start = Second() { s = span(); }
+        [   <LPAREN> startPrec = UnsignedIntLiteral()
+            [ <COMMA> secondFracPrec = UnsignedIntLiteral() ]
+            <RPAREN>
+        ]
+    )
+    {
+        return new SqlIntervalQualifier(start, startPrec, null, secondFracPrec,
+            s.end(this));
+    }
+}
+
+/** Parses a built-in time unit (e.g. "YEAR")
+ * or user-defined time frame (e.g. "MINUTE15")
+ * and in each case returns a {@link SqlIntervalQualifier}.
+ *
+ * <p>The units are used in several functions, incuding CEIL, FLOOR, EXTRACT.
+ * Includes NANOSECOND, MILLISECOND, which were previously allowed in EXTRACT
+ * but not CEIL, FLOOR.
+ *
+ * <p>Includes {@code WEEK} and {@code WEEK(SUNDAY)} through
+  {@code WEEK(SATURDAY)}.
+ *
+ * <p>Does not include SQL_TSI_DAY, SQL_TSI_FRAC_SECOND etc. These will be
+ * parsed as identifiers and can be resolved in the validator if they are
+ * registered as abbreviations in your time frame set.
+ */
+SqlIntervalQualifier TimeUnitOrName() : {
+    final SqlIdentifier unitName;
+    final SqlIntervalQualifier intervalQualifier;
+}
+{
+    // When we see a time unit that is also a non-reserved keyword, such as
+    // NANOSECOND, there is a choice between using the TimeUnit enum
+    // (TimeUnit.NANOSECOND) or the name. The following LOOKAHEAD directive
+    // tells the parser that we prefer the former.
+    //
+    // Reserved keywords, such as SECOND, cannot be identifiers, and are
+    // therefore not ambiguous.
+    LOOKAHEAD(2)
+    intervalQualifier = TimeUnit() {
+        return intervalQualifier;
+    }
+|   unitName = SimpleIdentifier() {
+        return new SqlIntervalQualifier(unitName.getSimple(),
+            unitName.getParserPosition());
+    }
+}
+
+/** Parses a built-in time unit (e.g. "YEAR")
+ * and returns a {@link SqlIntervalQualifier}.
+ *
+ * <p>Includes {@code WEEK} and {@code WEEK(SUNDAY)} through
+  {@code WEEK(SATURDAY)}.
+ *
+ * <p>Does not include SQL_TSI_DAY, SQL_TSI_FRAC_SECOND etc. These will be
+ * parsed as identifiers and can be resolved in the validator if they are
+ * registered as abbreviations in your time frame set.
+ */
+SqlIntervalQualifier TimeUnit() : {
+    final Span span;
+    final String w;
+}
+{
+    <NANOSECOND> { return new SqlIntervalQualifier(TimeUnit.NANOSECOND, null, getPos()); }
+|   <MICROSECOND> { return new SqlIntervalQualifier(TimeUnit.MICROSECOND, null, getPos()); }
+|   <MILLISECOND> { return new SqlIntervalQualifier(TimeUnit.MILLISECOND, null, getPos()); }
+|   <SECOND> { return new SqlIntervalQualifier(TimeUnit.SECOND, null, getPos()); }
+|   <MINUTE> { return new SqlIntervalQualifier(TimeUnit.MINUTE, null, getPos()); }
+|   <HOUR> { return new SqlIntervalQualifier(TimeUnit.HOUR, null, getPos()); }
+|   <DAY> { return new SqlIntervalQualifier(TimeUnit.DAY, null, getPos()); }
+|   <DOW> { return new SqlIntervalQualifier(TimeUnit.DOW, null, getPos()); }
+|   <DOY> { return new SqlIntervalQualifier(TimeUnit.DOY, null, getPos()); }
+|   <ISODOW> { return new SqlIntervalQualifier(TimeUnit.ISODOW, null, getPos()); }
+|   <ISOYEAR> { return new SqlIntervalQualifier(TimeUnit.ISOYEAR, null, getPos()); }
+|   <WEEK> { span = span(); }
+    (
+        // There is a choice between "WEEK(weekday)" and "WEEK". We prefer
+        // the former, and the parser will look ahead for '('.
+        LOOKAHEAD(2)
+        <LPAREN> w = weekdayName() <RPAREN> {
+            return new SqlIntervalQualifier(w, span.end(this));
+        }
+    |
+        { return new SqlIntervalQualifier(TimeUnit.WEEK, null, getPos()); }
+    )
+|   <MONTH> { return new SqlIntervalQualifier(TimeUnit.MONTH, null, getPos()); }
+|   <QUARTER> { return new SqlIntervalQualifier(TimeUnit.QUARTER, null, getPos()); }
+|   <YEAR> { return new SqlIntervalQualifier(TimeUnit.YEAR, null, getPos()); }
+|   <EPOCH> { return new SqlIntervalQualifier(TimeUnit.EPOCH, null, getPos()); }
+|   <DECADE> { return new SqlIntervalQualifier(TimeUnit.DECADE, null, getPos()); }
+|   <CENTURY> { return new SqlIntervalQualifier(TimeUnit.CENTURY, null, getPos()); }
+|   <MILLENNIUM> { return new SqlIntervalQualifier(TimeUnit.MILLENNIUM, null, getPos()); }
+}
+
+String weekdayName() :
+{
+}
+{
+    <SUNDAY> { return "WEEK_SUNDAY"; }
+|   <MONDAY> { return "WEEK_MONDAY"; }
+|   <TUESDAY> { return "WEEK_TUESDAY"; }
+|   <WEDNESDAY> { return "WEEK_WEDNESDAY"; }
+|   <THURSDAY> { return "WEEK_THURSDAY"; }
+|   <FRIDAY> { return "WEEK_FRIDAY"; }
+|   <SATURDAY> { return "WEEK_SATURDAY"; }
+}
+
+/**
+ * Parses a dynamic parameter marker.
+ */
+SqlDynamicParam DynamicParam() :
+{
+}
+{
+    <HOOK> {
+        return new SqlDynamicParam(nDynamicParams++, getPos());
+    }
+}
+
+/**
+ * Parses one segment of an identifier that may be composite.
+ *
+ * <p>Each time it reads an identifier it writes one element to each list;
+ * the entry in {@code positions} records its position and whether the
+ * segment was quoted.
+ */
+void AddIdentifierSegment(List<String> names, List<SqlParserPos> positions) :
+{
+    final String id;
+    char unicodeEscapeChar = BACKSLASH;
+    final SqlParserPos pos;
+    final Span span;
+}
+{
+    (
+        <IDENTIFIER> {
+            id = unquotedIdentifier();
+            pos = getPos();
+        }
+    |
+        <HYPHENATED_IDENTIFIER> {
+            id = unquotedIdentifier();
+            pos = getPos();
+        }
+    |
+        <QUOTED_IDENTIFIER> {
+            id = SqlParserUtil.stripQuotes(getToken(0).image, DQ, DQ, DQDQ,
+                quotedCasing);
+            pos = getPos().withQuoting(true);
+        }
+    |
+        <BACK_QUOTED_IDENTIFIER> {
+            id = SqlParserUtil.stripQuotes(getToken(0).image, "`", "`", "``",
+                quotedCasing);
+            pos = getPos().withQuoting(true);
+        }
+    |
+        <BIG_QUERY_BACK_QUOTED_IDENTIFIER> {
+            id = SqlParserUtil.stripQuotes(getToken(0).image, "`", "`", "\\`",
+                quotedCasing);
+            pos = getPos().withQuoting(true);
+        }
+    |
+        <BRACKET_QUOTED_IDENTIFIER> {
+            id = SqlParserUtil.stripQuotes(getToken(0).image, "[", "]", "]]",
+                quotedCasing);
+            pos = getPos().withQuoting(true);
+        }
+    |
+        <UNICODE_QUOTED_IDENTIFIER> {
+            span = span();
+            String image = getToken(0).image;
+            image = image.substring(image.indexOf('"'));
+            image = SqlParserUtil.stripQuotes(image, DQ, DQ, DQDQ, quotedCasing);
+        }
+        [
+            <UESCAPE> <QUOTED_STRING> {
+                String s = SqlParserUtil.parseString(token.image);
+                unicodeEscapeChar = SqlParserUtil.checkUnicodeEscapeChar(s);
+            }
+        ]
+        {
+            pos = span.end(this).withQuoting(true);
+            SqlLiteral lit = SqlLiteral.createCharString(image, "UTF16", pos);
+            lit = lit.unescapeUnicode(unicodeEscapeChar);
+            id = lit.toValue();
+        }
+    |
+        id = NonReservedKeyWord() {
+            pos = getPos();
+        }
+    )
+    {
+        if (id.length() > this.identifierMaxLength) {
+            throw SqlUtil.newContextException(pos,
+                RESOURCE.identifierTooLong(id, this.identifierMaxLength));
+        }
+        names.add(id);
+        if (positions != null) {
+            positions.add(pos);
+        }
+    }
+}
+
+/** As {@link #AddIdentifierSegment} but part of a table name (for example,
+ * following {@code FROM}, {@code INSERT} or {@code UPDATE}).
+ *
+ * <p>In some dialects the lexical rules for table names are different from
+ * for other identifiers. For example, in BigQuery, table names may contain
+ * hyphens. */
+void AddTableIdentifierSegment(List<String> names, List<SqlParserPos> positions) :
+{
+}
+{
+    AddIdentifierSegment(names, positions) {
+        final int n = names.size();
+        if (n > 0
+                && positions.size() == n
+                && names.get(n - 1).contains(".")
+                && positions.get(n - 1).isQuoted()
+                && this.conformance.splitQuotedTableName()) {
+            final String name = names.remove(n - 1);
+            final SqlParserPos pos = positions.remove(n - 1);
+            final String[] splitNames = name.split("\\.");
+            for (String splitName : splitNames) {
+                names.add(splitName);
+                positions.add(pos);
+            }
+        }
+    }
+}
+
+/**
+ * Parses a simple identifier as a String.
+ */
+String Identifier() :
+{
+    final List<String> names = new ArrayList<String>();
+}
+{
+    AddIdentifierSegment(names, null) {
+        return names.get(0);
+    }
+}
+
+/**
+ * Parses a simple identifier as an SqlIdentifier.
+ */
+SqlIdentifier SimpleIdentifier() :
+{
+    final List<String> names = new ArrayList<String>();
+    final List<SqlParserPos> positions = new ArrayList<SqlParserPos>();
+}
+{
+    AddIdentifierSegment(names, positions) {
+        return new SqlIdentifier(names.get(0), positions.get(0));
+    }
+}
+
+/**
+ * Parses a character literal as an SqlIdentifier.
+ * Only valid for column aliases in certain dialects.
+ */
+SqlIdentifier SimpleIdentifierFromStringLiteral() :
+{
+}
+{
+    <QUOTED_STRING> {
+        if (!this.conformance.allowCharLiteralAlias()) {
+            throw SqlUtil.newContextException(getPos(), RESOURCE.charLiteralAliasNotValid());
+        }
+        final String s = SqlParserUtil.parseString(token.image);
+        return new SqlIdentifier(s, getPos());
+    }
+}
+
+/**
+ * Parses a comma-separated list of simple identifiers.
+ */
+void AddSimpleIdentifiers(List<SqlNode> list) :
+{
+    SqlIdentifier id;
+}
+{
+    id = SimpleIdentifier() {list.add(id);}
+    (
+        <COMMA> id = SimpleIdentifier() {
+            list.add(id);
+        }
+    )*
+}
+
+/**
+  * List of simple identifiers in parentheses. The position extends from the
+  * open parenthesis to the close parenthesis.
+  */
+SqlNodeList ParenthesizedSimpleIdentifierList() :
+{
+    final Span s;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    <LPAREN> { s = span(); }
+    AddSimpleIdentifiers(list)
+    <RPAREN> {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+/** List of simple identifiers in parentheses or one simple identifier.
+ *
+ * <ul>Examples:
+ * <li>{@code DEPTNO}
+ * <li>{@code (EMPNO, DEPTNO)}
+ * </ul>
+ */
+SqlNodeList SimpleIdentifierOrList() :
+{
+    SqlIdentifier id;
+    SqlNodeList list;
+}
+{
+    id = SimpleIdentifier() {
+        return new SqlNodeList(Collections.singletonList(id), id.getParserPosition());
+    }
+|
+    list = ParenthesizedSimpleIdentifierList() {
+        return list;
+    }
+}
+
+<#if (parser.includeCompoundIdentifier!default.parser.includeCompoundIdentifier) >
+/**
+ * Parses a compound identifier.
+ */
+SqlIdentifier CompoundIdentifier() :
+{
+    final List<String> nameList = new ArrayList<String>();
+    final List<SqlParserPos> posList = new ArrayList<SqlParserPos>();
+    boolean star = false;
+}
+{
+    AddIdentifierSegment(nameList, posList)
+    (
+        LOOKAHEAD(2)
+        <DOT>
+        AddIdentifierSegment(nameList, posList)
+    )*
+    (
+        LOOKAHEAD(2)
+        <DOT>
+        <STAR> {
+            star = true;
+            nameList.add("");
+            posList.add(getPos());
+        }
+    )?
+    {
+        SqlParserPos pos = SqlParserPos.sum(posList);
+        if (star) {
+            return SqlIdentifier.star(nameList, pos, posList);
+        }
+        return new SqlIdentifier(nameList, null, pos, posList);
+    }
+}
+
+/**
+ * Parses a compound identifier in the FROM clause.
+ */
+SqlIdentifier CompoundTableIdentifier() :
+{
+    final List<String> nameList = new ArrayList<String>();
+    final List<SqlParserPos> posList = new ArrayList<SqlParserPos>();
+}
+{
+    AddTableIdentifierSegment(nameList, posList)
+    (
+        LOOKAHEAD(2)
+        <DOT>
+        AddTableIdentifierSegment(nameList, posList)
+    )*
+    {
+        SqlParserPos pos = SqlParserPos.sum(posList);
+        return new SqlIdentifier(nameList, null, pos, posList);
+    }
+}
+
+/**
+ * Parses a comma-separated list of compound identifiers.
+ */
+void AddCompoundIdentifierTypes(List<SqlNode> list, List<SqlNode> extendList) :
+{
+}
+{
+    AddCompoundIdentifierType(list, extendList)
+    (<COMMA> AddCompoundIdentifierType(list, extendList))*
+}
+
+/**
+ * List of compound identifiers in parentheses. The position extends from the
+ * open parenthesis to the close parenthesis.
+ */
+Pair<SqlNodeList, SqlNodeList> ParenthesizedCompoundIdentifierList() :
+{
+    final Span s;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    final List<SqlNode> extendList = new ArrayList<SqlNode>();
+}
+{
+    <LPAREN> { s = span(); }
+    AddCompoundIdentifierTypes(list, extendList)
+    <RPAREN> {
+        return Pair.of(new SqlNodeList(list, s.end(this)), new SqlNodeList(extendList, s.end(this)));
+    }
+}
+<#else>
+  <#include "/@includes/compoundIdentifier.ftl" />
+</#if>
+
+/**
+ * Parses a NEW UDT(...) expression.
+ */
+SqlNode NewSpecification() :
+{
+    final Span s;
+    final SqlNode routineCall;
+}
+{
+    <NEW> { s = span(); }
+    routineCall =
+        NamedRoutineCall(SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR,
+            ExprContext.ACCEPT_SUB_QUERY) {
+        return SqlStdOperatorTable.NEW.createCall(s.end(routineCall), routineCall);
+    }
+}
+
+//TODO: real parse errors.
+int UnsignedIntLiteral() :
+{
+    Token t;
+}
+{
+    t = <UNSIGNED_INTEGER_LITERAL>
+    {
+        try {
+            return Integer.parseInt(t.image);
+        } catch (NumberFormatException ex) {
+            throw SqlUtil.newContextException(getPos(),
+                RESOURCE.invalidLiteral(t.image, Integer.class.getCanonicalName()));
+        }
+    }
+}
+
+int IntLiteral() :
+{
+    Token t;
+}
+{
+    (
+        t = <UNSIGNED_INTEGER_LITERAL>
+    |
+        <PLUS> t = <UNSIGNED_INTEGER_LITERAL>
+    )
+    {
+        try {
+            return Integer.parseInt(t.image);
+        } catch (NumberFormatException ex) {
+            throw SqlUtil.newContextException(getPos(),
+                RESOURCE.invalidLiteral(t.image, Integer.class.getCanonicalName()));
+        }
+    }
+|
+    <MINUS> t = <UNSIGNED_INTEGER_LITERAL> {
+        try {
+            return -Integer.parseInt(t.image);
+        } catch (NumberFormatException ex) {
+            throw SqlUtil.newContextException(getPos(),
+                RESOURCE.invalidLiteral(t.image, Integer.class.getCanonicalName()));
+        }
+    }
+}
+
+// Type name with optional scale and precision.
+SqlDataTypeSpec DataType() :
+{
+    SqlTypeNameSpec typeName;
+    final Span s;
+}
+{
+    typeName = TypeName() {
+        s = Span.of(typeName.getParserPos());
+    }
+    (
+        typeName = CollectionsTypeName(typeName)
+    )*
+    {
+        return new SqlDataTypeSpec(typeName, s.add(typeName.getParserPos()).pos());
+    }
+}
+
+// Some SQL type names need special handling due to the fact that they have
+// spaces in them but are not quoted.
+SqlTypeNameSpec TypeName() :
+{
+    final SqlTypeNameSpec typeNameSpec;
+    final SqlIdentifier typeName;
+    final Span s = Span.of();
+}
+{
+    (
+<#-- additional types are included here -->
+<#-- put custom data types in front of Calcite core data types -->
+<#list (parser.dataTypeParserMethods!default.parser.dataTypeParserMethods) as method>
+        LOOKAHEAD(2)
+        typeNameSpec = ${method}
+    |
+</#list>
+        LOOKAHEAD(2)
+        typeNameSpec = SqlTypeName(s)
+    |
+        typeNameSpec = RowTypeName()
+    |
+        typeName = CompoundIdentifier() {
+            typeNameSpec = new SqlUserDefinedTypeNameSpec(typeName, s.end(this));
+        }
+    )
+    {
+        return typeNameSpec;
+    }
+}
+
+// Types used for JDBC and ODBC scalar conversion function
+SqlTypeNameSpec SqlTypeName(Span s) :
+{
+    final SqlTypeNameSpec sqlTypeNameSpec;
+}
+{
+    (
+        sqlTypeNameSpec = SqlTypeName1(s)
+    |
+        sqlTypeNameSpec = SqlTypeName2(s)
+    |
+        sqlTypeNameSpec = SqlTypeName3(s)
+    |
+        sqlTypeNameSpec = CharacterTypeName(s)
+    |
+        sqlTypeNameSpec = DateTimeTypeName()
+    )
+    {
+        return sqlTypeNameSpec;
+    }
+}
+
+// Parse sql type name that don't allow any extra specifications except the type name.
+// For extra specification, we mean precision, scale, charSet, etc.
+SqlTypeNameSpec SqlTypeName1(Span s) :
+{
+    final SqlTypeName sqlTypeName;
+}
+{
+    (
+        <GEOMETRY> {
+            if (!this.conformance.allowGeometry()) {
+                throw SqlUtil.newContextException(getPos(), RESOURCE.geometryDisabled());
+            }
+            s.add(this);
+            sqlTypeName = SqlTypeName.GEOMETRY;
+        }
+    |
+        <BOOLEAN> { s.add(this); sqlTypeName = SqlTypeName.BOOLEAN; }
+    |
+        ( <INTEGER> | <INT> ) { s.add(this); sqlTypeName = SqlTypeName.INTEGER; }
+    |
+        <TINYINT> { s.add(this); sqlTypeName = SqlTypeName.TINYINT; }
+    |
+        <SMALLINT> { s.add(this); sqlTypeName = SqlTypeName.SMALLINT; }
+    |
+        <BIGINT> { s.add(this); sqlTypeName = SqlTypeName.BIGINT; }
+    |
+        <REAL> { s.add(this); sqlTypeName = SqlTypeName.REAL; }
+    |
+        <DOUBLE> { s.add(this); }
+        [ <PRECISION> ] { sqlTypeName = SqlTypeName.DOUBLE; }
+    |
+        <FLOAT> { s.add(this); sqlTypeName = SqlTypeName.FLOAT; }
+    )
+    {
+        return new SqlBasicTypeNameSpec(sqlTypeName, s.end(this));
+    }
+}
+
+// Parse sql type name that allows precision specification.
+SqlTypeNameSpec SqlTypeName2(Span s) :
+{
+    final SqlTypeName sqlTypeName;
+    int precision = -1;
+}
+{
+    (
+        <BINARY> { s.add(this); }
+        (
+            <VARYING> { sqlTypeName = SqlTypeName.VARBINARY; }
+        |
+            { sqlTypeName = SqlTypeName.BINARY; }
+        )
+    |
+        <VARBINARY> { s.add(this); sqlTypeName = SqlTypeName.VARBINARY; }
+    )
+    precision = PrecisionOpt()
+    {
+        return new SqlBasicTypeNameSpec(sqlTypeName, precision, s.end(this));
+    }
+}
+
+// Parse sql type name that allows precision and scale specifications.
+SqlTypeNameSpec SqlTypeName3(Span s) :
+{
+    final SqlTypeName sqlTypeName;
+    int precision = -1;
+    int scale = -1;
+}
+{
+    (
+        (<DECIMAL> | <DEC> | <NUMERIC>) { s.add(this); sqlTypeName = SqlTypeName.DECIMAL; }
+    |
+        <ANY> { s.add(this); sqlTypeName = SqlTypeName.ANY; }
+    )
+    [
+        <LPAREN>
+        precision = UnsignedIntLiteral()
+        [
+            <COMMA>
+            scale = UnsignedIntLiteral()
+        ]
+        <RPAREN>
+    ]
+    {
+        return new SqlBasicTypeNameSpec(sqlTypeName, precision, scale, s.end(this));
+    }
+}
+
+// Types used for for JDBC and ODBC scalar conversion function
+SqlJdbcDataTypeName JdbcOdbcDataTypeName() :
+{
+}
+{
+    (<SQL_CHAR> | <CHAR>) { return SqlJdbcDataTypeName.SQL_CHAR; }
+|   (<SQL_VARCHAR> | <VARCHAR>) { return SqlJdbcDataTypeName.SQL_VARCHAR; }
+|   (<SQL_DATE> | <DATE>) { return SqlJdbcDataTypeName.SQL_DATE; }
+|   (<SQL_TIME> | <TIME>) { return SqlJdbcDataTypeName.SQL_TIME; }
+|   (<SQL_TIMESTAMP> | <TIMESTAMP>) { return SqlJdbcDataTypeName.SQL_TIMESTAMP; }
+|   (<SQL_DECIMAL> | <DECIMAL>) { return SqlJdbcDataTypeName.SQL_DECIMAL; }
+|   (<SQL_NUMERIC> | <NUMERIC>) { return SqlJdbcDataTypeName.SQL_NUMERIC; }
+|   (<SQL_BOOLEAN> | <BOOLEAN>) { return SqlJdbcDataTypeName.SQL_BOOLEAN; }
+|   (<SQL_INTEGER> | <INTEGER>) { return SqlJdbcDataTypeName.SQL_INTEGER; }
+|   (<SQL_BINARY> | <BINARY>) { return SqlJdbcDataTypeName.SQL_BINARY; }
+|   (<SQL_VARBINARY> | <VARBINARY>) { return SqlJdbcDataTypeName.SQL_VARBINARY; }
+|   (<SQL_TINYINT> | <TINYINT>) { return SqlJdbcDataTypeName.SQL_TINYINT; }
+|   (<SQL_SMALLINT> | <SMALLINT>) { return SqlJdbcDataTypeName.SQL_SMALLINT; }
+|   (<SQL_BIGINT> | <BIGINT>) { return SqlJdbcDataTypeName.SQL_BIGINT; }
+|   (<SQL_REAL>| <REAL>) { return SqlJdbcDataTypeName.SQL_REAL; }
+|   (<SQL_DOUBLE> | <DOUBLE>) { return SqlJdbcDataTypeName.SQL_DOUBLE; }
+|   (<SQL_FLOAT> | <FLOAT>) { return SqlJdbcDataTypeName.SQL_FLOAT; }
+|   <SQL_INTERVAL_YEAR> { return SqlJdbcDataTypeName.SQL_INTERVAL_YEAR; }
+|   <SQL_INTERVAL_YEAR_TO_MONTH> { return SqlJdbcDataTypeName.SQL_INTERVAL_YEAR_TO_MONTH; }
+|   <SQL_INTERVAL_MONTH> { return SqlJdbcDataTypeName.SQL_INTERVAL_MONTH; }
+|   <SQL_INTERVAL_DAY> { return SqlJdbcDataTypeName.SQL_INTERVAL_DAY; }
+|   <SQL_INTERVAL_DAY_TO_HOUR> { return SqlJdbcDataTypeName.SQL_INTERVAL_DAY_TO_HOUR; }
+|   <SQL_INTERVAL_DAY_TO_MINUTE> { return SqlJdbcDataTypeName.SQL_INTERVAL_DAY_TO_MINUTE; }
+|   <SQL_INTERVAL_DAY_TO_SECOND> { return SqlJdbcDataTypeName.SQL_INTERVAL_DAY_TO_SECOND; }
+|   <SQL_INTERVAL_HOUR> { return SqlJdbcDataTypeName.SQL_INTERVAL_HOUR; }
+|   <SQL_INTERVAL_HOUR_TO_MINUTE> { return SqlJdbcDataTypeName.SQL_INTERVAL_HOUR_TO_MINUTE; }
+|   <SQL_INTERVAL_HOUR_TO_SECOND> { return SqlJdbcDataTypeName.SQL_INTERVAL_HOUR_TO_SECOND; }
+|   <SQL_INTERVAL_MINUTE> { return SqlJdbcDataTypeName.SQL_INTERVAL_MINUTE; }
+|   <SQL_INTERVAL_MINUTE_TO_SECOND> { return SqlJdbcDataTypeName.SQL_INTERVAL_MINUTE_TO_SECOND; }
+|   <SQL_INTERVAL_SECOND> { return SqlJdbcDataTypeName.SQL_INTERVAL_SECOND; }
+}
+
+SqlLiteral JdbcOdbcDataType() :
+{
+    SqlJdbcDataTypeName typeName;
+}
+{
+    typeName = JdbcOdbcDataTypeName() {
+        return typeName.symbol(getPos());
+    }
+}
+
+/**
+* Parse a collection type name, the input element type name may
+* also be a collection type.
+*/
+SqlTypeNameSpec CollectionsTypeName(SqlTypeNameSpec elementTypeName) :
+{
+    final SqlTypeName collectionTypeName;
+}
+{
+    (
+        <MULTISET> { collectionTypeName = SqlTypeName.MULTISET; }
+    |
+        <ARRAY> { collectionTypeName = SqlTypeName.ARRAY; }
+    )
+    {
+        return new SqlCollectionTypeNameSpec(elementTypeName,
+                collectionTypeName, getPos());
+    }
+}
+
+/**
+* Parse a nullable option, default is true.
+*/
+boolean NullableOptDefaultTrue() :
+{
+}
+{
+    <NULL> { return true; }
+|
+    <NOT> <NULL> { return false; }
+|
+    { return true; }
+}
+
+/**
+* Parse a nullable option, default is false.
+*/
+boolean NullableOptDefaultFalse() :
+{
+}
+{
+    <NULL> { return true; }
+|
+    <NOT> <NULL> { return false; }
+|
+    { return false; }
+}
+
+/** Parses NOT NULL and returns false, or parses nothing and returns true. */
+boolean NotNullOpt() :
+{
+}
+{
+    <NOT> <NULL> { return false; }
+|
+    { return true; }
+}
+
+/**
+* Parse a "name1 type1 [NULL | NOT NULL], name2 type2 [NULL | NOT NULL] ..." list,
+* the field type default is not nullable.
+*/
+void AddFieldNameTypes(List<SqlIdentifier> fieldNames,
+    List<SqlDataTypeSpec> fieldTypes) :
+{
+}
+{
+    AddFieldNameType(fieldNames, fieldTypes)
+    ( <COMMA> AddFieldNameType(fieldNames, fieldTypes) )*
+}
+
+void AddFieldNameType(List<SqlIdentifier> fieldNames,
+    List<SqlDataTypeSpec> fieldTypes) :
+{
+    final SqlIdentifier fName;
+    final SqlDataTypeSpec fType;
+    final boolean nullable;
+}
+{
+    fName = SimpleIdentifier()
+    fType = DataType()
+    nullable = NullableOptDefaultFalse()
+    {
+        fieldNames.add(fName);
+        fieldTypes.add(fType.withNullable(nullable, getPos()));
+    }
+}
+
+/**
+* Parse Row type with format: Row(name1 type1, name2 type2).
+* Every field type can have suffix of `NULL` or `NOT NULL` to indicate if this type is nullable.
+* i.e. Row(f0 int not null, f1 varchar null).
+*/
+SqlTypeNameSpec RowTypeName() :
+{
+    List<SqlIdentifier> fieldNames = new ArrayList<SqlIdentifier>();
+    List<SqlDataTypeSpec> fieldTypes = new ArrayList<SqlDataTypeSpec>();
+}
+{
+    <ROW>
+    <LPAREN> AddFieldNameTypes(fieldNames, fieldTypes) <RPAREN>
+    {
+        return new SqlRowTypeNameSpec(getPos(), fieldNames, fieldTypes);
+    }
+}
+
+/**
+* Parse character types: char, varchar.
+*/
+SqlTypeNameSpec CharacterTypeName(Span s) :
+{
+    int precision = -1;
+    final SqlTypeName sqlTypeName;
+    String charSetName = null;
+}
+{
+    (
+        (<CHARACTER> | <CHAR>) { s.add(this); }
+        (
+            <VARYING> { sqlTypeName = SqlTypeName.VARCHAR; }
+        |
+            { sqlTypeName = SqlTypeName.CHAR; }
+        )
+    |
+        <VARCHAR> { s.add(this); sqlTypeName = SqlTypeName.VARCHAR; }
+    )
+    precision = PrecisionOpt()
+    [
+        <CHARACTER> <SET>
+        charSetName = Identifier()
+    ]
+    {
+        return new SqlBasicTypeNameSpec(sqlTypeName, precision, charSetName, s.end(this));
+    }
+}
+
+/**
+* Parse datetime types: date, time, timestamp.
+*/
+SqlTypeNameSpec DateTimeTypeName() :
+{
+    int precision = -1;
+    SqlTypeName typeName;
+    boolean withLocalTimeZone = false;
+    final Span s;
+}
+{
+    <DATE> {
+        typeName = SqlTypeName.DATE;
+        return new SqlBasicTypeNameSpec(typeName, getPos());
+    }
+|
+    LOOKAHEAD(2)
+    <TIME> { s = span(); }
+    precision = PrecisionOpt()
+    withLocalTimeZone = TimeZoneOpt()
+    {
+        if (withLocalTimeZone) {
+            typeName = SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE;
+        } else {
+            typeName = SqlTypeName.TIME;
+        }
+        return new SqlBasicTypeNameSpec(typeName, precision, s.end(this));
+    }
+|
+    <TIMESTAMP> { s = span(); }
+    precision = PrecisionOpt()
+    withLocalTimeZone = TimeZoneOpt()
+    {
+        if (withLocalTimeZone) {
+            typeName = SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE;
+        } else {
+            typeName = SqlTypeName.TIMESTAMP;
+        }
+        return new SqlBasicTypeNameSpec(typeName, precision, s.end(this));
+    }
+}
+
+// Parse an optional data type precision, default is -1.
+int PrecisionOpt() :
+{
+    int precision = -1;
+}
+{
+    <LPAREN>
+    precision = UnsignedIntLiteral()
+    <RPAREN>
+    { return precision; }
+|
+    { return -1; }
+}
+
+/**
+* Parse a time zone suffix for DateTime types. According to SQL-2011,
+* "with time zone" and "without time zone" belong to standard SQL but we
+* only implement the "without time zone".
+*
+* <p>We also support "with local time zone".
+*
+* @return true if this is "with local time zone".
+*/
+boolean TimeZoneOpt() :
+{
+}
+{
+    LOOKAHEAD(3)
+    <WITHOUT> <TIME> <ZONE> { return false; }
+|
+    <WITH> <LOCAL> <TIME> <ZONE> { return true; }
+|
+    { return false; }
+}
+
+/**
+ * Parses a CURSOR(query) expression.  The parser allows these
+ * anywhere, but the validator restricts them to appear only as
+ * arguments to table functions.
+ */
+SqlNode CursorExpression(ExprContext exprContext) :
+{
+    final SqlNode e;
+    final Span s;
+}
+{
+    <CURSOR> {
+        s = span();
+        if (exprContext != ExprContext.ACCEPT_ALL
+                && exprContext != ExprContext.ACCEPT_CURSOR) {
+            throw SqlUtil.newContextException(s.end(this),
+                RESOURCE.illegalCursorExpression());
+        }
+    }
+    e = Expression(ExprContext.ACCEPT_QUERY) {
+        return SqlStdOperatorTable.CURSOR.createCall(s.end(e), e);
+    }
+}
+
+/**
+ * Parses a call to a builtin function with special syntax.
+ */
+SqlNode BuiltinFunctionCall() :
+{
+    final SqlIdentifier name;
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    SqlNode e;
+    final Span s;
+    SqlDataTypeSpec dt;
+    final SqlIntervalQualifier unit;
+    final SqlNode node;
+}
+{
+    //~ FUNCTIONS WITH SPECIAL SYNTAX ---------------------------------------
+    (
+        <CAST> { s = span(); }
+        <LPAREN> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        <AS>
+        (
+            dt = DataType() { args.add(dt); }
+        |
+            <INTERVAL> e = IntervalQualifier() { args.add(e); }
+        )
+        <RPAREN> {
+            return SqlStdOperatorTable.CAST.createCall(s.end(this), args);
+        }
+    |
+        <EXTRACT> { s = span(); }
+        <LPAREN> unit = TimeUnitOrName() {
+            args.add(unit);
+        }
+        <FROM>
+        AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        <RPAREN> {
+            return SqlStdOperatorTable.EXTRACT.createCall(s.end(this), args);
+        }
+    |
+        <POSITION> { s = span(); }
+        <LPAREN>
+        // FIXME jvs 31-Aug-2006:  FRG-192:  This should be
+        // Expression(ExprContext.ACCEPT_SUB_QUERY), but that doesn't work
+        // because it matches the other kind of IN.
+        e = AtomicRowExpression() { args.add(e); }
+        <IN>
+        AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        [ <FROM> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY) ]
+        <RPAREN> {
+            return SqlStdOperatorTable.POSITION.createCall(s.end(this), args);
+        }
+    |
+        <CONVERT> { s = span(); }
+        <LPAREN>
+        AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        <USING> name = SimpleIdentifier() { args.add(name); }
+        <RPAREN> {
+            return SqlStdOperatorTable.CONVERT.createCall(s.end(this), args);
+        }
+    |
+        <TRANSLATE> { s = span(); }
+        <LPAREN>
+        AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        (
+            <USING> name = SimpleIdentifier() { args.add(name); }
+            <RPAREN> {
+                return SqlStdOperatorTable.TRANSLATE.createCall(s.end(this),
+                    args);
+            }
+        |
+            ( <COMMA> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY) )*
+            <RPAREN> {
+                return SqlLibraryOperators.TRANSLATE3.createCall(s.end(this),
+                    args);
+            }
+        )
+    |
+        <OVERLAY> { s = span(); }
+        <LPAREN> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        <PLACING> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        <FROM> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        [ <FOR> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY) ]
+        <RPAREN> {
+            return SqlStdOperatorTable.OVERLAY.createCall(s.end(this), args);
+        }
+    |
+        <FLOOR> { s = span(); }
+        e = FloorCeilOptions(s, true) {
+            return e;
+        }
+    |
+        ( <CEIL> | <CEILING>) { s = span(); }
+        e = FloorCeilOptions(s, false) {
+            return e;
+        }
+    |
+        <SUBSTRING> { s = span(); }
+        <LPAREN>
+        AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        ( <FROM> | <COMMA>)
+        AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        [
+            (<FOR> | <COMMA>)
+            AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+        ]
+        <RPAREN> {
+            return SqlStdOperatorTable.SUBSTRING.createCall(
+                s.end(this), args);
+        }
+    |
+        <TRIM> {
+            SqlLiteral flag = null;
+            SqlNode trimChars = null;
+            s = span();
+        }
+        <LPAREN>
+        [
+            LOOKAHEAD(2)
+            [
+                <BOTH> {
+                    s.add(this);
+                    flag = SqlTrimFunction.Flag.BOTH.symbol(getPos());
+                }
+            |
+                <TRAILING> {
+                    s.add(this);
+                    flag = SqlTrimFunction.Flag.TRAILING.symbol(getPos());
+                }
+            |
+                <LEADING> {
+                    s.add(this);
+                    flag = SqlTrimFunction.Flag.LEADING.symbol(getPos());
+                }
+            ]
+            [ trimChars = Expression(ExprContext.ACCEPT_SUB_QUERY) ]
+            (
+                <FROM> {
+                    if (null == flag && null == trimChars) {
+                        throw SqlUtil.newContextException(getPos(),
+                            RESOURCE.illegalFromEmpty());
+                    }
+                }
+            |
+                <RPAREN> {
+                    // This is to handle the case of TRIM(x)
+                    // (FRG-191).
+                    if (flag == null) {
+                        flag = SqlTrimFunction.Flag.BOTH.symbol(SqlParserPos.ZERO);
+                    }
+                    args.add(flag);
+                    args.add(null); // no trim chars
+                    args.add(trimChars); // reinterpret trimChars as source
+                    return SqlStdOperatorTable.TRIM.createCall(s.end(this),
+                        args);
+                }
+            )
+        ]
+        e = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+            if (flag == null) {
+                flag = SqlTrimFunction.Flag.BOTH.symbol(SqlParserPos.ZERO);
+            }
+            args.add(flag);
+            args.add(trimChars);
+            args.add(e);
+        }
+        <RPAREN> {
+            return SqlStdOperatorTable.TRIM.createCall(s.end(this), args);
+        }
+    |
+        node = DateTimeConstructorCall() { return node; }
+    |
+        node = DateTruncFunctionCall() { return node; }
+    |
+        node = TimestampAddFunctionCall() { return node; }
+    |
+        node = TimestampDiffFunctionCall() { return node; }
+    |
+        node = TimestampDiff3FunctionCall() { return node; }
+    |
+        node = TimestampTruncFunctionCall() { return node; }
+    |
+        node = TimeDiffFunctionCall() { return node; }
+    |
+        node = TimeTruncFunctionCall() { return node; }
+    |
+<#list (parser.builtinFunctionCallMethods!default.parser.builtinFunctionCallMethods) as method>
+        node = ${method} { return node; }
+    |
+</#list>
+        node = MatchRecognizeFunctionCall() { return node; }
+    |
+        node = JsonExistsFunctionCall() { return node; }
+    |
+        node = JsonValueFunctionCall() { return node; }
+    |
+        node = JsonQueryFunctionCall() { return node; }
+    |
+        node = JsonObjectFunctionCall() { return node; }
+    |
+        node = JsonObjectAggFunctionCall() { return node; }
+    |
+        node = JsonArrayFunctionCall() { return node; }
+    |
+        node = JsonArrayAggFunctionCall() { return node; }
+    |
+        node = GroupByWindowingCall() { return node; }
+    )
+}
+
+SqlJsonEncoding JsonRepresentation() :
+{
+}
+{
+    <JSON>
+    [
+        // Encoding is currently ignored.
+        LOOKAHEAD(2) <ENCODING>
+        (
+            <UTF8> { return SqlJsonEncoding.UTF8; }
+        |
+            <UTF16> { return SqlJsonEncoding.UTF16; }
+        |
+            <UTF32> { return SqlJsonEncoding.UTF32; }
+        )
+    ]
+    {
+        return SqlJsonEncoding.UTF8;
+    }
+}
+
+void JsonInputClause() :
+{
+}
+{
+    <FORMAT> JsonRepresentation()
+}
+
+SqlDataTypeSpec JsonReturningClause() :
+{
+    SqlDataTypeSpec dt;
+}
+{
+    <RETURNING> dt = DataType() { return dt; }
+}
+
+SqlDataTypeSpec JsonOutputClause() :
+{
+    SqlDataTypeSpec dataType;
+}
+{
+    dataType = JsonReturningClause()
+    [
+        <FORMAT> JsonRepresentation()
+    ]
+    {
+        return dataType;
+    }
+}
+
+SqlNode JsonPathSpec() :
+{
+    SqlNode e;
+}
+{
+    e = StringLiteral() {
+        return e;
+    }
+}
+
+List<SqlNode> JsonApiCommonSyntax() :
+{
+    SqlNode e;
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+}
+{
+    AddExpression(args, ExprContext.ACCEPT_NON_QUERY)
+    <COMMA>
+    AddExpression(args, ExprContext.ACCEPT_NON_QUERY)
+    [
+        // We currently don't support JSON passing clause, leave the java code blocks no-op
+        <PASSING> e = Expression(ExprContext.ACCEPT_NON_QUERY) {
+            // no-op
+        }
+        <AS> e = SimpleIdentifier() {
+            // no-op
+        }
+        (
+            <COMMA>
+            e = Expression(ExprContext.ACCEPT_NON_QUERY) {
+                // no-op
+            }
+            <AS> e = SimpleIdentifier() {
+                // no-op
+            }
+        )*
+    ]
+    {
+        return args;
+    }
+}
+
+SqlJsonExistsErrorBehavior JsonExistsErrorBehavior() :
+{
+
+}
+{
+    <TRUE> { return SqlJsonExistsErrorBehavior.TRUE; }
+    |
+    <FALSE> { return SqlJsonExistsErrorBehavior.FALSE; }
+    |
+    <UNKNOWN> { return SqlJsonExistsErrorBehavior.UNKNOWN; }
+    |
+    <ERROR> { return SqlJsonExistsErrorBehavior.ERROR; }
+}
+
+SqlCall JsonExistsFunctionCall() :
+{
+    List<SqlNode> args = new ArrayList<SqlNode>();
+    List<SqlNode> commonSyntax;
+    final Span span;
+    SqlJsonExistsErrorBehavior errorBehavior;
+}
+{
+    <JSON_EXISTS> { span = span(); }
+    <LPAREN> commonSyntax = JsonApiCommonSyntax() {
+        args.addAll(commonSyntax);
+    }
+    [
+        errorBehavior = JsonExistsErrorBehavior() { args.add(errorBehavior.symbol(getPos())); }
+        <ON> <ERROR>
+    ]
+    <RPAREN> {
+        return SqlStdOperatorTable.JSON_EXISTS.createCall(span.end(this), args);
+    }
+}
+
+List<SqlNode> JsonValueEmptyOrErrorBehavior() :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    (
+        <ERROR> {
+            list.add(SqlJsonValueEmptyOrErrorBehavior.ERROR.symbol(getPos()));
+        }
+    |
+        <NULL> {
+            list.add(SqlJsonValueEmptyOrErrorBehavior.NULL.symbol(getPos()));
+        }
+    |
+        <DEFAULT_> {
+            list.add(SqlJsonValueEmptyOrErrorBehavior.DEFAULT.symbol(getPos()));
+        }
+        AddExpression(list, ExprContext.ACCEPT_NON_QUERY)
+    )
+    <ON>
+    (
+        <EMPTY> {
+            list.add(SqlJsonEmptyOrError.EMPTY.symbol(getPos()));
+        }
+    |
+        <ERROR> {
+            list.add(SqlJsonEmptyOrError.ERROR.symbol(getPos()));
+        }
+    )
+    { return list; }
+}
+
+SqlCall JsonValueFunctionCall() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>(7);
+    SqlNode e;
+    List<SqlNode> commonSyntax;
+    final Span span;
+    List<SqlNode> behavior;
+}
+{
+    <JSON_VALUE> { span = span(); }
+    <LPAREN> commonSyntax = JsonApiCommonSyntax() {
+        args.addAll(commonSyntax);
+    }
+    [
+        e = JsonReturningClause() {
+            args.add(SqlJsonValueReturning.RETURNING.symbol(getPos()));
+            args.add(e);
+        }
+    ]
+    (
+        behavior = JsonValueEmptyOrErrorBehavior() {
+            args.addAll(behavior);
+        }
+    )*
+    <RPAREN> {
+        return SqlStdOperatorTable.JSON_VALUE.createCall(span.end(this), args);
+    }
+}
+
+List<SqlNode> JsonQueryEmptyOrErrorBehavior() :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+}
+{
+    (
+        <ERROR> {
+            list.add(SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.ERROR, getPos()));
+        }
+    |
+        <NULL> {
+            list.add(SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.NULL, getPos()));
+        }
+    |
+        LOOKAHEAD(2)
+        <EMPTY> <ARRAY> {
+            list.add(SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.EMPTY_ARRAY, getPos()));
+        }
+    |
+        <EMPTY> <OBJECT> {
+            list.add(SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.EMPTY_OBJECT, getPos()));
+        }
+    )
+    <ON>
+    (
+        <EMPTY> {
+            list.add(SqlLiteral.createSymbol(SqlJsonEmptyOrError.EMPTY, getPos()));
+        }
+    |
+        <ERROR> {
+            list.add(SqlLiteral.createSymbol(SqlJsonEmptyOrError.ERROR, getPos()));
+        }
+    )
+    { return list; }
+}
+
+SqlNode JsonQueryWrapperBehavior() :
+{
+}
+{
+    <WITHOUT> [<ARRAY>] {
+        return SqlLiteral.createSymbol(SqlJsonQueryWrapperBehavior.WITHOUT_ARRAY, getPos());
+    }
+|
+    LOOKAHEAD(2)
+    <WITH> <CONDITIONAL> [<ARRAY>] {
+        return SqlLiteral.createSymbol(SqlJsonQueryWrapperBehavior.WITH_CONDITIONAL_ARRAY, getPos());
+    }
+|
+    <WITH> [<UNCONDITIONAL>] [<ARRAY>] {
+        return SqlLiteral.createSymbol(SqlJsonQueryWrapperBehavior.WITH_UNCONDITIONAL_ARRAY, getPos());
+    }
+}
+
+SqlCall JsonQueryFunctionCall() :
+{
+    final SqlNode[] args = new SqlNode[5];
+    SqlNode e;
+    List<SqlNode> commonSyntax;
+    final Span span;
+    List<SqlNode> behavior;
+}
+{
+    <JSON_QUERY> { span = span(); }
+    <LPAREN> commonSyntax = JsonApiCommonSyntax() {
+        args[0] = commonSyntax.get(0);
+        args[1] = commonSyntax.get(1);
+    }
+    [
+        e = JsonQueryWrapperBehavior() <WRAPPER> {
+            args[2] = e;
+        }
+    ]
+    (
+        behavior = JsonQueryEmptyOrErrorBehavior() {
+            final SqlJsonEmptyOrError symbol =
+                ((SqlLiteral) behavior.get(1)).getValueAs(SqlJsonEmptyOrError.class);
+            switch (symbol) {
+            case EMPTY:
+                args[3] = behavior.get(0);
+                break;
+            case ERROR:
+                args[4] = behavior.get(0);
+                break;
+            }
+        }
+    )*
+    <RPAREN> {
+        return SqlStdOperatorTable.JSON_QUERY.createCall(span.end(this), args);
+    }
+}
+
+SqlNode JsonName() :
+{
+    final SqlNode e;
+}
+{
+     e = Expression(ExprContext.ACCEPT_NON_QUERY) {
+        return e;
+     }
+}
+
+List<SqlNode> JsonNameAndValue() :
+{
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    final SqlNode e;
+    boolean kvMode = false;
+}
+{
+    [
+        LOOKAHEAD(2, <KEY> JsonName())
+        <KEY> { kvMode = true; }
+    ]
+    e = JsonName() {
+        list.add(e);
+    }
+    (
+        <VALUE>
+    |
+        <COLON> {
+            if (kvMode) {
+                throw SqlUtil.newContextException(getPos(), RESOURCE.illegalColon());
+            }
+        }
+    )
+    AddExpression(list, ExprContext.ACCEPT_NON_QUERY)
+    {
+        return list;
+    }
+}
+
+SqlNode JsonConstructorNullClause() :
+{
+}
+{
+    <NULL> <ON> <NULL> {
+        return SqlLiteral.createSymbol(SqlJsonConstructorNullClause.NULL_ON_NULL, getPos());
+    }
+|
+    <ABSENT> <ON> <NULL> {
+        return SqlLiteral.createSymbol(SqlJsonConstructorNullClause.ABSENT_ON_NULL, getPos());
+    }
+}
+
+SqlCall JsonObjectFunctionCall() :
+{
+    final List<SqlNode> nvArgs = new ArrayList<SqlNode>();
+    final SqlNode[] otherArgs = new SqlNode[1];
+    SqlNode e;
+    List<SqlNode> list;
+    final Span span;
+}
+{
+    <JSON_OBJECT> { span = span(); }
+    <LPAREN> [
+        LOOKAHEAD(2)
+        list = JsonNameAndValue() {
+            nvArgs.addAll(list);
+        }
+        (
+            <COMMA>
+            list = JsonNameAndValue() {
+                nvArgs.addAll(list);
+            }
+        )*
+    ]
+    [
+        e = JsonConstructorNullClause() {
+            otherArgs[0] = e;
+        }
+    ]
+    <RPAREN> {
+        final List<SqlNode> args = new ArrayList();
+        args.addAll(Arrays.asList(otherArgs));
+        args.addAll(nvArgs);
+        return SqlStdOperatorTable.JSON_OBJECT.createCall(span.end(this), args);
+    }
+}
+
+SqlCall JsonObjectAggFunctionCall() :
+{
+    final SqlNode[] args = new SqlNode[2];
+    List<SqlNode> list;
+    final Span span;
+    SqlJsonConstructorNullClause nullClause =
+        SqlJsonConstructorNullClause.NULL_ON_NULL;
+    final SqlNode e;
+}
+{
+    <JSON_OBJECTAGG> { span = span(); }
+    <LPAREN> list = JsonNameAndValue() {
+        args[0] = list.get(0);
+        args[1] = list.get(1);
+    }
+    [
+        e = JsonConstructorNullClause() {
+            nullClause = (SqlJsonConstructorNullClause) ((SqlLiteral) e).getValue();
+        }
+    ]
+    <RPAREN> {
+        return SqlStdOperatorTable.JSON_OBJECTAGG.with(nullClause)
+            .createCall(span.end(this), args);
+    }
+}
+
+SqlCall JsonArrayFunctionCall() :
+{
+    final List<SqlNode> elements = new ArrayList<SqlNode>();
+    final SqlNode[] otherArgs = new SqlNode[1];
+    SqlNode e;
+    final Span span;
+}
+{
+    <JSON_ARRAY> { span = span(); }
+    <LPAREN> [
+        LOOKAHEAD(2)
+        AddExpression(elements, ExprContext.ACCEPT_NON_QUERY)
+        ( <COMMA> AddExpression(elements, ExprContext.ACCEPT_NON_QUERY) )*
+    ]
+    [
+        e = JsonConstructorNullClause() {
+            otherArgs[0] = e;
+        }
+    ]
+    <RPAREN> {
+        final List<SqlNode> args = new ArrayList();
+        args.addAll(Arrays.asList(otherArgs));
+        args.addAll(elements);
+        return SqlStdOperatorTable.JSON_ARRAY.createCall(span.end(this), args);
+    }
+}
+
+SqlNodeList JsonArrayAggOrderByClause() :
+{
+    final SqlNodeList orderList;
+}
+{
+    orderList = OrderBy(true)
+    { return orderList; }
+}
+
+SqlCall JsonArrayAggFunctionCall() :
+{
+    final SqlNode valueExpr;
+    final SqlNodeList orderList;
+    final Span span;
+    final SqlJsonConstructorNullClause nullClause;
+    SqlNode e;
+    final SqlNode aggCall;
+}
+{
+    <JSON_ARRAYAGG> { span = span(); }
+    <LPAREN> e = Expression(ExprContext.ACCEPT_NON_QUERY) {
+        valueExpr = e;
+    }
+    ( orderList = JsonArrayAggOrderByClause() | { orderList = null; } )
+    (
+        e = JsonConstructorNullClause() {
+            nullClause = (SqlJsonConstructorNullClause) ((SqlLiteral) e).getValue();
+        }
+    |   { nullClause = SqlJsonConstructorNullClause.ABSENT_ON_NULL; }
+    )
+    <RPAREN>
+    {
+        aggCall = SqlStdOperatorTable.JSON_ARRAYAGG.with(nullClause)
+            .createCall(span.end(this), valueExpr, orderList);
+    }
+    [
+        e = withinGroup(aggCall) {
+            if (orderList != null) {
+                throw SqlUtil.newContextException(span.pos().plus(e.getParserPosition()),
+                    RESOURCE.ambiguousSortOrderInJsonArrayAggFunc());
+            }
+            return (SqlCall) e;
+        }
+    ]
+    {
+        if (orderList == null) {
+            return SqlStdOperatorTable.JSON_ARRAYAGG.with(nullClause)
+                .createCall(span.end(this), valueExpr);
+        }
+        return SqlStdOperatorTable.JSON_ARRAYAGG.with(nullClause)
+            .createCall(span.end(this), valueExpr, orderList);
+    }
+}
+
+/**
+ * Parses a call to TIMESTAMPADD.
+ */
+SqlCall TimestampAddFunctionCall() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final Span s;
+    final SqlIntervalQualifier unit;
+}
+{
+    <TIMESTAMPADD> { s = span(); }
+    <LPAREN>
+    unit = TimeUnitOrName() { args.add(unit); }
+    <COMMA>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <RPAREN> {
+        return SqlStdOperatorTable.TIMESTAMP_ADD.createCall(
+            s.end(this), args);
+    }
+}
+
+/**
+ * Parses a call to TIMESTAMPDIFF.
+ */
+SqlCall TimestampDiffFunctionCall() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final Span s;
+    final SqlIntervalQualifier unit;
+}
+{
+    <TIMESTAMPDIFF> { s = span(); }
+    <LPAREN>
+    unit = TimeUnitOrName() { args.add(unit); }
+    <COMMA>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <RPAREN> {
+        return SqlStdOperatorTable.TIMESTAMP_DIFF.createCall(
+            s.end(this), args);
+    }
+}
+
+/**
+ * Parses a call to BigQuery's TIMESTAMP_DIFF.
+ *
+ * <p>The difference between TIMESTAMPDIFF and TIMESTAMP_DIFF is the ordering of
+ * the parameters and the arrangement of the subtraction.
+ * TIMESTAMPDIFF uses (unit, timestamp1, timestamp2) with (t2 - t1), while
+ * TIMESTAMP_DIFF uses (timestamp1, timestamp2, unit) with (t1 - t2).
+ */
+SqlCall TimestampDiff3FunctionCall() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final Span s;
+    final SqlIntervalQualifier unit;
+}
+{
+    <TIMESTAMP_DIFF> { s = span(); }
+    <LPAREN>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    unit = TimeUnitOrName() { args.add(unit); }
+    <RPAREN> {
+        return SqlLibraryOperators.TIMESTAMP_DIFF3.createCall(s.end(this), args);
+    }
+}
+
+/**
+ * Parses a call to DATE_TRUNC.
+ */
+SqlCall DateTruncFunctionCall() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final Span s;
+    final SqlIntervalQualifier unit;
+}
+{
+    <DATE_TRUNC> { s = span(); }
+    <LPAREN>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    // A choice of arguments allows us to support both
+    // the BigQuery variant, e.g. "DATE_TRUNC(d, YEAR)",
+    // and the Redshift variant, e.g. "DATE_TRUNC('year', DATE '2008-09-08')".
+    (
+        LOOKAHEAD(2)
+        unit = TimeUnit() { args.add(unit); }
+    |
+        AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    )
+    <RPAREN> {
+        return SqlLibraryOperators.DATE_TRUNC.createCall(s.end(this), args);
+    }
+}
+
+/**
+ * Parses a call to TIMESTAMP_TRUNC.
+ */
+SqlCall TimestampTruncFunctionCall() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final Span s;
+    final SqlIntervalQualifier unit;
+}
+{
+    <TIMESTAMP_TRUNC> { s = span(); }
+    <LPAREN>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    unit = TimeUnitOrName() { args.add(unit); }
+    <RPAREN> {
+        return SqlLibraryOperators.TIMESTAMP_TRUNC.createCall(s.end(this), args);
+    }
+}
+
+/**
+ * Parses a call to BigQuery's TIME_DIFF.
+ */
+SqlCall TimeDiffFunctionCall() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final Span s;
+    final SqlIntervalQualifier unit;
+}
+{
+    <TIME_DIFF> { s = span(); }
+    <LPAREN>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    unit = TimeUnitOrName() { args.add(unit); }
+    <RPAREN> {
+        return SqlLibraryOperators.TIME_DIFF.createCall(s.end(this), args);
+    }
+}
+
+/**
+ * Parses a call to TIME_TRUNC.
+ */
+SqlCall TimeTruncFunctionCall() :
+{
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final Span s;
+    final SqlIntervalQualifier unit;
+}
+{
+    <TIME_TRUNC> { s = span(); }
+    <LPAREN>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    <COMMA>
+    unit = TimeUnitOrName() { args.add(unit); }
+    <RPAREN> {
+        return SqlLibraryOperators.TIME_TRUNC.createCall(s.end(this), args);
+    }
+}
+
+/**
+ * Parses a call to a grouping function inside the GROUP BY clause,
+ * for example {@code TUMBLE(rowtime, INTERVAL '1' MINUTE)}.
+ */
+SqlCall GroupByWindowingCall() :
+{
+    final Span s;
+    final List<SqlNode> args;
+    final SqlOperator op;
+}
+{
+    (
+        <TUMBLE> { op = SqlStdOperatorTable.TUMBLE_OLD; }
+    |
+        <HOP> { op = SqlStdOperatorTable.HOP_OLD; }
+    |
+        <SESSION> { op = SqlStdOperatorTable.SESSION_OLD; }
+    )
+    { s = span(); }
+    args = UnquantifiedFunctionParameterList(ExprContext.ACCEPT_SUB_QUERY) {
+        return op.createCall(s.end(this), args);
+    }
+}
+
+SqlCall MatchRecognizeFunctionCall() :
+{
+    final SqlCall func;
+    final Span s;
+}
+{
+    (
+        <CLASSIFIER> { s = span(); } <LPAREN> <RPAREN> {
+            func = SqlStdOperatorTable.CLASSIFIER.createCall(s.end(this));
+        }
+    |
+        <MATCH_NUMBER> { s = span(); } <LPAREN> <RPAREN> {
+            func = SqlStdOperatorTable.MATCH_NUMBER.createCall(s.end(this));
+        }
+    |
+        LOOKAHEAD(3)
+        func = MatchRecognizeNavigationLogical()
+    |
+        LOOKAHEAD(2)
+        func = MatchRecognizeNavigationPhysical()
+    |
+        func = MatchRecognizeCallWithModifier()
+    )
+    { return func; }
+}
+
+SqlCall MatchRecognizeCallWithModifier() :
+{
+    final Span s;
+    final SqlOperator runningOp;
+    final SqlNode func;
+}
+{
+    (
+        <RUNNING> { runningOp = SqlStdOperatorTable.RUNNING; }
+    |
+        <FINAL> { runningOp = SqlStdOperatorTable.FINAL; }
+    )
+    { s = span(); }
+    func = NamedFunctionCall() {
+        return runningOp.createCall(s.end(func), func);
+    }
+}
+
+SqlCall MatchRecognizeNavigationLogical() :
+{
+    final Span s = Span.of();
+    SqlCall func;
+    final SqlOperator funcOp;
+    final SqlOperator runningOp;
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    SqlNode e;
+}
+{
+    (
+        <RUNNING> { runningOp = SqlStdOperatorTable.RUNNING; s.add(this); }
+    |
+        <FINAL> { runningOp = SqlStdOperatorTable.FINAL; s.add(this); }
+    |   { runningOp = null; }
+    )
+    (
+        <FIRST> { funcOp = SqlStdOperatorTable.FIRST; }
+    |
+        <LAST> { funcOp = SqlStdOperatorTable.LAST; }
+    )
+    { s.add(this); }
+    <LPAREN>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    (
+        <COMMA> e = NumericLiteral() { args.add(e); }
+    |
+        { args.add(LITERAL_ZERO); }
+    )
+    <RPAREN> {
+        func = funcOp.createCall(s.end(this), args);
+        if (runningOp != null) {
+            return runningOp.createCall(s.end(this), func);
+        } else {
+            return func;
+        }
+    }
+}
+
+SqlCall MatchRecognizeNavigationPhysical() :
+{
+    final Span s;
+    final SqlOperator funcOp;
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    SqlNode e;
+}
+{
+    (
+        <PREV> { funcOp = SqlStdOperatorTable.PREV; }
+    |
+        <NEXT> { funcOp = SqlStdOperatorTable.NEXT; }
+    )
+    { s = span(); }
+    <LPAREN>
+    AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    (
+        <COMMA> e = NumericLiteral() { args.add(e); }
+    |
+        { args.add(LITERAL_ONE); }
+    )
+    <RPAREN> {
+        return funcOp.createCall(s.end(this), args);
+    }
+}
+
+SqlCall withinDistinct(SqlNode arg) :
+{
+    final Span s;
+    final SqlNodeList distinctList;
+}
+{
+    <WITHIN> { s = span(); }
+    <DISTINCT>
+    <LPAREN>
+    distinctList = ExpressionCommaList(s, ExprContext.ACCEPT_SUB_QUERY)
+    <RPAREN> {
+        return SqlStdOperatorTable.WITHIN_DISTINCT.createCall(
+            s.end(this), arg, distinctList);
+    }
+}
+
+SqlCall withinGroup(SqlNode arg) :
+{
+    final Span s;
+    final SqlNodeList orderList;
+}
+{
+    <WITHIN> { s = span(); }
+    <GROUP>
+    <LPAREN>
+    orderList = OrderBy(true)
+    <RPAREN> {
+        return SqlStdOperatorTable.WITHIN_GROUP.createCall(
+            s.end(this), arg, orderList);
+    }
+}
+
+Pair<SqlParserPos, SqlOperator> NullTreatment() :
+{
+    final Span span;
+}
+{
+    <IGNORE> { span = span(); } <NULLS> {
+        return Pair.of(span.end(this), SqlStdOperatorTable.IGNORE_NULLS);
+    }
+|
+    <RESPECT> { span = span(); } <NULLS> {
+        return Pair.of(span.end(this), SqlStdOperatorTable.RESPECT_NULLS);
+    }
+}
+
+SqlCall nullTreatment(SqlCall arg) :
+{
+    final Pair<SqlParserPos, SqlOperator> pair;
+}
+{
+    pair = NullTreatment() { return pair.right.createCall(pair.left, arg); }
+}
+
+/**
+ * Parses a call to a named function (could be a builtin with regular
+ * syntax, or else a UDF).
+ *
+ * <p>NOTE: every UDF has two names: an <em>invocation name</em> and a
+ * <em>specific name</em>.  Normally, function calls are resolved via overload
+ * resolution and invocation names.  The SPECIFIC prefix allows overload
+ * resolution to be bypassed.  Note that usage of the SPECIFIC prefix in
+ * queries is non-standard; it is used internally by Farrago, e.g. in stored
+ * view definitions to permanently bind references to a particular function
+ * after the overload resolution performed by view creation.
+ *
+ * <p>TODO jvs 25-Mar-2005:  Once we have SQL-Flagger support, flag SPECIFIC
+ * as non-standard.
+ */
+SqlNode NamedFunctionCall() :
+{
+    SqlCall call;
+    final Span filterSpan;
+    final SqlNode filter;
+    final Span overSpan;
+    final SqlNode over;
+}
+{
+    (
+        LOOKAHEAD(2)
+        call = StringAggFunctionCall()
+    |
+        call = NamedCall()
+    )
+    [
+        LOOKAHEAD(2) call = nullTreatment(call)
+    ]
+    [
+        LOOKAHEAD(2) // decide between WITHIN DISTINCT and WITHIN GROUP
+        call = withinDistinct(call)
+    ]
+    [
+        call = withinGroup(call)
+    ]
+    [
+        <FILTER> { filterSpan = span(); }
+        <LPAREN>
+        <WHERE>
+        filter = Expression(ExprContext.ACCEPT_SUB_QUERY)
+        <RPAREN> {
+            call = SqlStdOperatorTable.FILTER.createCall(
+                filterSpan.end(this), call, filter);
+        }
+    ]
+    [
+        <OVER> { overSpan = span(); }
+        (
+            over = SimpleIdentifier()
+        |
+            over = WindowSpecification()
+        )
+        {
+            call = SqlStdOperatorTable.OVER.createCall(overSpan.end(over), call, over);
+        }
+    ]
+    {
+        return call;
+    }
+}
+
+SqlCall NamedCall() :
+{
+    final SqlFunctionCategory funcType;
+    final SqlIdentifier qualifiedName;
+    final Span s;
+    final List<SqlNode> args;
+    SqlLiteral quantifier = null;
+}
+{
+    (
+        <SPECIFIC> {
+            funcType = SqlFunctionCategory.USER_DEFINED_SPECIFIC_FUNCTION;
+        }
+    |
+        { funcType = SqlFunctionCategory.USER_DEFINED_FUNCTION; }
+    )
+    qualifiedName = FunctionName() {
+        s = span();
+    }
+    (
+        LOOKAHEAD(2) <LPAREN> <STAR> {
+            args = ImmutableList.of(SqlIdentifier.star(getPos()));
+        }
+        <RPAREN>
+    |
+        LOOKAHEAD(2) <LPAREN> <RPAREN> {
+            args = ImmutableList.of();
+        }
+    |
+        args = FunctionParameterList(ExprContext.ACCEPT_SUB_QUERY) {
+            quantifier = (SqlLiteral) args.get(0);
+            args.remove(0);
+        }
+    )
+    {
+        return createCall(qualifiedName, s.end(this), funcType, quantifier, args);
+    }
+}
+
+/*
+* Parse Floor/Ceil function parameters
+*/
+SqlNode StandardFloorCeilOptions(Span s, boolean floorFlag) :
+{
+    SqlNode e;
+    final List<SqlNode> args = new ArrayList<SqlNode>();
+    final SqlIntervalQualifier unit;
+    SqlCall function;
+    final Span s1;
+}
+{
+    <LPAREN> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+    (
+        <TO> unit = TimeUnitOrName() {
+            args.add(unit);
+        }
+    )?
+    <RPAREN> {
+        SqlOperator op = floorFlag
+            ? SqlStdOperatorTable.FLOOR
+            : SqlStdOperatorTable.CEIL;
+        function =  op.createCall(s.end(this), args);
+    }
+    (
+        <OVER> { s1 = span(); }
+        (
+            e = SimpleIdentifier()
+        |
+            e = WindowSpecification()
+        )
+        {
+            return SqlStdOperatorTable.OVER.createCall(s1.end(this), function, e);
+        }
+    |
+        { return function; }
+    )
+}
+
+/**
+ * Parses the name of a JDBC function that is a token but is not reserved.
+ */
+String NonReservedJdbcFunctionName() :
+{
+}
+{
+    (
+        <SUBSTRING>
+    )
+    {
+        return unquotedIdentifier();
+    }
+}
+
+/**
+ * Parses the name of a function (either a compound identifier or
+ * a reserved word which can be used as a function name).
+ */
+SqlIdentifier FunctionName() :
+{
+    SqlIdentifier qualifiedName;
+}
+{
+    (
+        qualifiedName = CompoundIdentifier()
+    |
+        qualifiedName = ReservedFunctionName()
+    )
+    {
+        return qualifiedName;
+    }
+}
+
+/**
+ * Parses a reserved word which is used as the name of a function.
+ */
+SqlIdentifier ReservedFunctionName() :
+{
+}
+{
+    (
+        <ABS>
+    |   <AVG>
+    |   <CARDINALITY>
+    |   <CEILING>
+    |   <CHAR>
+    |   <CHAR_LENGTH>
+    |   <CHARACTER_LENGTH>
+    |   <COALESCE>
+    |   <COLLECT>
+    |   <COVAR_POP>
+    |   <COVAR_SAMP>
+    |   <CUME_DIST>
+    |   <COUNT>
+    |   <CURRENT_DATE>
+    |   <CURRENT_TIME>
+    |   <CURRENT_TIMESTAMP>
+    |   <DENSE_RANK>
+    |   <ELEMENT>
+    |   <EVERY>
+    |   <EXP>
+    |   <FIRST_VALUE>
+    |   <FLOOR>
+    |   <FUSION>
+    |   <INTERSECTION>
+    |   <GROUPING>
+    |   <HOUR>
+    |   <LAG>
+    |   <LEAD>
+    |   <LEFT>
+    |   <LAST_VALUE>
+    |   <LN>
+    |   <LOCALTIME>
+    |   <LOCALTIMESTAMP>
+    |   <LOWER>
+    |   <MAX>
+    |   <MIN>
+    |   <MINUTE>
+    |   <MOD>
+    |   <MONTH>
+    |   <NTH_VALUE>
+    |   <NTILE>
+    |   <NULLIF>
+    |   <OCTET_LENGTH>
+    |   <PERCENT_RANK>
+    |   <PERCENTILE_CONT>
+    |   <PERCENTILE_DISC>
+    |   <POWER>
+    |   <RANK>
+    |   <REGR_COUNT>
+    |   <REGR_SXX>
+    |   <REGR_SYY>
+    |   <RIGHT>
+    |   <ROW_NUMBER>
+    |   <SECOND>
+    |   <SOME>
+    |   <SQRT>
+    |   <STDDEV_POP>
+    |   <STDDEV_SAMP>
+    |   <SUM>
+    |   <UPPER>
+    |   <TRUNCATE>
+    |   <USER>
+    |   <VAR_POP>
+    |   <VAR_SAMP>
+    |   <YEAR>
+    )
+    {
+        return new SqlIdentifier(unquotedIdentifier(), getPos());
+    }
+}
+
+SqlIdentifier ContextVariable() :
+{
+}
+{
+    (
+        <CURRENT_CATALOG>
+    |   <CURRENT_DATE>
+    |   <CURRENT_DEFAULT_TRANSFORM_GROUP>
+    |   <CURRENT_PATH>
+    |   <CURRENT_ROLE>
+    |   <CURRENT_SCHEMA>
+    |   <CURRENT_TIME>
+    |   <CURRENT_TIMESTAMP>
+    |   <CURRENT_USER>
+    |   <LOCALTIME>
+    |   <LOCALTIMESTAMP>
+    |   <SESSION_USER>
+    |   <SYSTEM_USER>
+    |   <USER>
+    )
+    {
+        return new SqlIdentifier(unquotedIdentifier(), getPos());
+    }
+}
+
+/**
+ * Parses a function call expression with JDBC syntax.
+ */
+SqlNode JdbcFunctionCall() :
+{
+    String name;
+    SqlIdentifier id;
+    SqlNode e;
+    SqlLiteral tl;
+    SqlNodeList args;
+    SqlCall call;
+    final Span s, s1;
+}
+{
+    <LBRACE_FN> {
+        s = span();
+    }
+    (
+        LOOKAHEAD(1)
+        call = TimestampAddFunctionCall() {
+            name = call.getOperator().getName();
+            args = new SqlNodeList(call.getOperandList(), getPos());
+        }
+    |
+        LOOKAHEAD(1)
+        call = DateTruncFunctionCall() {
+            name = call.getOperator().getName();
+            args = new SqlNodeList(call.getOperandList(), getPos());
+        }
+    |
+        LOOKAHEAD(1)
+        call = TimestampTruncFunctionCall() {
+            name = call.getOperator().getName();
+            args = new SqlNodeList(call.getOperandList(), getPos());
+        }
+    |
+        LOOKAHEAD(1)
+        call = TimeTruncFunctionCall() {
+            name = call.getOperator().getName();
+            args = new SqlNodeList(call.getOperandList(), getPos());
+        }
+    |
+        LOOKAHEAD(1)
+        call = TimestampDiff3FunctionCall() {
+            name = call.getOperator().getName();
+            args = new SqlNodeList(call.getOperandList(), getPos());
+        }
+    |
+        LOOKAHEAD(1)
+        call = TimeDiffFunctionCall() {
+            name = call.getOperator().getName();
+            args = new SqlNodeList(call.getOperandList(), getPos());
+        }
+    |
+        LOOKAHEAD(3)
+        call = TimestampDiffFunctionCall() {
+            name = call.getOperator().getName();
+            args = new SqlNodeList(call.getOperandList(), getPos());
+        }
+    |
+        <CONVERT> { name = unquotedIdentifier(); }
+        <LPAREN>
+        e = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+            args = new SqlNodeList(getPos());
+            args.add(e);
+        }
+        <COMMA>
+        tl = JdbcOdbcDataType() { args.add(tl); }
+        <RPAREN>
+    |
+        (
+            // INSERT is a reserved word, but we need to handle {fn insert}
+            // Similarly LEFT, RIGHT, TRUNCATE
+            LOOKAHEAD(1)
+            ( <INSERT> | <LEFT> | <RIGHT> | <TRUNCATE> ) { name = unquotedIdentifier(); }
+        |
+            // For cases like {fn power(1,2)} and {fn lower('a')}
+            id = ReservedFunctionName() { name = id.getSimple(); }
+        |
+            // For cases like {fn substring('foo', 1,2)}
+            name = NonReservedJdbcFunctionName()
+        |
+            name = Identifier()
+        )
+        (
+            LOOKAHEAD(2) <LPAREN> <STAR> { s1 = span(); } <RPAREN>
+            {
+                args = new SqlNodeList(s1.pos());
+                args.add(SqlIdentifier.star(s1.pos()));
+            }
+        |
+            LOOKAHEAD(2) <LPAREN> <RPAREN> { args = SqlNodeList.EMPTY; }
+        |
+            args = ParenthesizedQueryOrCommaList(ExprContext.ACCEPT_SUB_QUERY)
+        )
+    )
+    <RBRACE> {
+        return new SqlJdbcFunctionCall(name).createCall(s.end(this),
+            args.getList());
+    }
+}
+
+/**
+ * Parses a binary query operator like UNION.
+ */
+SqlBinaryOperator BinaryQueryOperator() :
+{
+}
+{
+    // If both the ALL or DISTINCT keywords are missing, DISTINCT is implicit.
+    (
+        <UNION>
+        (
+            <ALL> { return SqlStdOperatorTable.UNION_ALL; }
+        |   <DISTINCT> { return SqlStdOperatorTable.UNION; }
+        |   { return SqlStdOperatorTable.UNION; }
+        )
+    |
+        <INTERSECT>
+        (
+            <ALL> { return SqlStdOperatorTable.INTERSECT_ALL; }
+        |   <DISTINCT> { return SqlStdOperatorTable.INTERSECT; }
+        |   { return SqlStdOperatorTable.INTERSECT; }
+        )
+    |
+        (
+            <EXCEPT>
+        |
+            <SET_MINUS> {
+                if (!this.conformance.isMinusAllowed()) {
+                    throw SqlUtil.newContextException(getPos(), RESOURCE.minusNotAllowed());
+                }
+            }
+        )
+        (
+            <ALL> { return SqlStdOperatorTable.EXCEPT_ALL; }
+        |   <DISTINCT> { return SqlStdOperatorTable.EXCEPT; }
+        |   { return SqlStdOperatorTable.EXCEPT; }
+        )
+    )
+}
+
+/**
+ * Parses a binary multiset operator.
+ */
+SqlBinaryOperator BinaryMultisetOperator() :
+{
+}
+{
+    // If both the ALL or DISTINCT keywords are missing, DISTINCT is implicit
+    <MULTISET>
+    (
+        <UNION>
+        [
+            <ALL>
+        |   <DISTINCT> { return SqlStdOperatorTable.MULTISET_UNION_DISTINCT; }
+        ]
+        { return SqlStdOperatorTable.MULTISET_UNION; }
+    |
+        <INTERSECT>
+        [
+            <ALL>
+        |   <DISTINCT> { return SqlStdOperatorTable.MULTISET_INTERSECT_DISTINCT; }
+        ]
+        { return SqlStdOperatorTable.MULTISET_INTERSECT; }
+    |
+        <EXCEPT>
+        [
+            <ALL>
+        |   <DISTINCT> { return SqlStdOperatorTable.MULTISET_EXCEPT_DISTINCT; }
+        ]
+        { return SqlStdOperatorTable.MULTISET_EXCEPT; }
+    )
+}
+
+/**
+ * Parses a binary row operator like AND.
+ */
+SqlBinaryOperator BinaryRowOperator() :
+{
+    SqlBinaryOperator op;
+}
+{
+    // <IN> is handled as a special case
+    <EQ> { return SqlStdOperatorTable.EQUALS; }
+|   <GT> { return SqlStdOperatorTable.GREATER_THAN; }
+|   <LT> { return SqlStdOperatorTable.LESS_THAN; }
+|   <LE> { return SqlStdOperatorTable.LESS_THAN_OR_EQUAL; }
+|   <GE> { return SqlStdOperatorTable.GREATER_THAN_OR_EQUAL; }
+|   <NE> { return SqlStdOperatorTable.NOT_EQUALS; }
+|   <NE2> {
+        if (!this.conformance.isBangEqualAllowed()) {
+            throw SqlUtil.newContextException(getPos(), RESOURCE.bangEqualNotAllowed());
+        }
+        return SqlStdOperatorTable.NOT_EQUALS;
+    }
+|   <PLUS> { return SqlStdOperatorTable.PLUS; }
+|   <MINUS> { return SqlStdOperatorTable.MINUS; }
+|   <STAR> { return SqlStdOperatorTable.MULTIPLY; }
+|   <SLASH> { return SqlStdOperatorTable.DIVIDE; }
+|   <PERCENT_REMAINDER> {
+        if (!this.conformance.isPercentRemainderAllowed()) {
+            throw SqlUtil.newContextException(getPos(), RESOURCE.percentRemainderNotAllowed());
+        }
+        return SqlStdOperatorTable.PERCENT_REMAINDER;
+    }
+|   <CONCAT> { return SqlStdOperatorTable.CONCAT; }
+|   <AND> { return SqlStdOperatorTable.AND; }
+|   <OR> { return SqlStdOperatorTable.OR; }
+|   LOOKAHEAD(2) <IS> <DISTINCT> <FROM> { return SqlStdOperatorTable.IS_DISTINCT_FROM; }
+|   <IS> <NOT> <DISTINCT> <FROM> { return SqlStdOperatorTable.IS_NOT_DISTINCT_FROM; }
+|   <MEMBER> <OF> { return SqlStdOperatorTable.MEMBER_OF; }
+|   LOOKAHEAD(2) <SUBMULTISET> <OF> { return SqlStdOperatorTable.SUBMULTISET_OF; }
+|   <NOT> <SUBMULTISET> <OF> { return SqlStdOperatorTable.NOT_SUBMULTISET_OF; }
+|   <CONTAINS> { return SqlStdOperatorTable.CONTAINS; }
+|   <OVERLAPS> { return SqlStdOperatorTable.OVERLAPS; }
+|   <EQUALS> { return SqlStdOperatorTable.PERIOD_EQUALS; }
+|   <PRECEDES> { return SqlStdOperatorTable.PRECEDES; }
+|   <SUCCEEDS> { return SqlStdOperatorTable.SUCCEEDS; }
+|   LOOKAHEAD(2) <IMMEDIATELY> <PRECEDES> { return SqlStdOperatorTable.IMMEDIATELY_PRECEDES; }
+|   <IMMEDIATELY> <SUCCEEDS> { return SqlStdOperatorTable.IMMEDIATELY_SUCCEEDS; }
+|   op = BinaryMultisetOperator() { return op; }
+}
+
+/**
+ * Parses a prefix row operator like NOT.
+ */
+SqlPrefixOperator PrefixRowOperator() :
+{}
+{
+    <PLUS> { return SqlStdOperatorTable.UNARY_PLUS; }
+|   <MINUS> { return SqlStdOperatorTable.UNARY_MINUS; }
+|   <NOT> { return SqlStdOperatorTable.NOT; }
+|   <EXISTS> { return SqlStdOperatorTable.EXISTS; }
+|   <UNIQUE> { return SqlStdOperatorTable.UNIQUE; }
+}
+
+/**
+ * Parses a postfix row operator like IS NOT NULL.
+ */
+SqlPostfixOperator PostfixRowOperator() :
+{}
+{
+    <IS>
+    (
+        <A> <SET> { return SqlStdOperatorTable.IS_A_SET; }
+    |
+        <NOT>
+        (
+            <NULL> { return SqlStdOperatorTable.IS_NOT_NULL; }
+        |   <TRUE> { return SqlStdOperatorTable.IS_NOT_TRUE; }
+        |   <FALSE> { return SqlStdOperatorTable.IS_NOT_FALSE; }
+        |   <UNKNOWN> { return SqlStdOperatorTable.IS_NOT_UNKNOWN; }
+        |   <A> <SET> { return SqlStdOperatorTable.IS_NOT_A_SET; }
+        |   <EMPTY> { return SqlStdOperatorTable.IS_NOT_EMPTY; }
+        |   LOOKAHEAD(2) <JSON> <VALUE> { return SqlStdOperatorTable.IS_NOT_JSON_VALUE; }
+        |   LOOKAHEAD(2) <JSON> <OBJECT> { return SqlStdOperatorTable.IS_NOT_JSON_OBJECT; }
+        |   LOOKAHEAD(2) <JSON> <ARRAY> { return SqlStdOperatorTable.IS_NOT_JSON_ARRAY; }
+        |   LOOKAHEAD(2) <JSON> <SCALAR> { return SqlStdOperatorTable.IS_NOT_JSON_SCALAR; }
+        |   <JSON> { return SqlStdOperatorTable.IS_NOT_JSON_VALUE; }
+        )
+    |
+        (
+            <NULL> { return SqlStdOperatorTable.IS_NULL; }
+        |   <TRUE> { return SqlStdOperatorTable.IS_TRUE; }
+        |   <FALSE> { return SqlStdOperatorTable.IS_FALSE; }
+        |   <UNKNOWN> { return SqlStdOperatorTable.IS_UNKNOWN; }
+        |   <EMPTY> { return SqlStdOperatorTable.IS_EMPTY; }
+        |   LOOKAHEAD(2) <JSON> <VALUE> { return SqlStdOperatorTable.IS_JSON_VALUE; }
+        |   LOOKAHEAD(2) <JSON> <OBJECT> { return SqlStdOperatorTable.IS_JSON_OBJECT; }
+        |   LOOKAHEAD(2) <JSON> <ARRAY> { return SqlStdOperatorTable.IS_JSON_ARRAY; }
+        |   LOOKAHEAD(2) <JSON> <SCALAR> { return SqlStdOperatorTable.IS_JSON_SCALAR; }
+        |   <JSON> { return SqlStdOperatorTable.IS_JSON_VALUE; }
+        )
+    )
+|
+    <FORMAT>
+    (
+        JsonRepresentation() {
+            return SqlStdOperatorTable.JSON_VALUE_EXPRESSION;
+        }
+    )
+}
+
+
+/* KEYWORDS:  anything in this list is a reserved word unless it appears
+   in the NonReservedKeyWord() production. */
+
+<DEFAULT, DQID, BTID, BQID, BQHID> TOKEN :
+{
+    < A: "A" >
+|   < ABS: "ABS" >
+|   < ABSENT: "ABSENT" >
+|   < ABSOLUTE: "ABSOLUTE" >
+|   < ACTION: "ACTION" >
+|   < ADA: "ADA" >
+|   < ADD: "ADD" >
+|   < ADMIN: "ADMIN" >
+|   < AFTER: "AFTER" >
+|   < ALL: "ALL" >
+|   < ALLOCATE: "ALLOCATE" >
+|   < ALLOW: "ALLOW" >
+|   < ALTER: "ALTER" >
+|   < ALWAYS: "ALWAYS" >
+|   < AND: "AND" >
+|   < ANY: "ANY" >
+|   < APPLY: "APPLY" >
+|   < ARE: "ARE" >
+|   < ARRAY: "ARRAY" >
+|   < ARRAY_AGG: "ARRAY_AGG" >
+|   < ARRAY_CONCAT_AGG: "ARRAY_CONCAT_AGG" >
+|   < ARRAY_MAX_CARDINALITY: "ARRAY_MAX_CARDINALITY" >
+|   < AS: "AS" >
+|   < ASC: "ASC" >
+|   < ASENSITIVE: "ASENSITIVE" >
+|   < ASSERTION: "ASSERTION" >
+|   < ASSIGNMENT: "ASSIGNMENT" >
+|   < ASYMMETRIC: "ASYMMETRIC" >
+|   < AT: "AT" >
+|   < ATOMIC: "ATOMIC" >
+|   < ATTRIBUTE: "ATTRIBUTE" >
+|   < ATTRIBUTES: "ATTRIBUTES" >
+|   < AUTHORIZATION: "AUTHORIZATION" >
+|   < AVG: "AVG" >
+|   < BEFORE: "BEFORE" >
+|   < BEGIN: "BEGIN" >
+|   < BEGIN_FRAME: "BEGIN_FRAME" >
+|   < BEGIN_PARTITION: "BEGIN_PARTITION" >
+|   < BERNOULLI: "BERNOULLI" >
+|   < BETWEEN: "BETWEEN" >
+|   < BIGINT: "BIGINT" >
+|   < BINARY: "BINARY" >
+|   < BIT: "BIT" >
+|   < BLOB: "BLOB" >
+|   < BOOLEAN: "BOOLEAN" >
+|   < BOTH: "BOTH" >
+|   < BREADTH: "BREADTH" >
+|   < BY: "BY" >
+|   < C: "C" >
+|   < CALL: "CALL" >
+|   < CALLED: "CALLED" >
+|   < CARDINALITY: "CARDINALITY" >
+|   < CASCADE: "CASCADE" >
+|   < CASCADED: "CASCADED" >
+|   < CASE: "CASE" >
+|   < CAST: "CAST" >
+|   < CATALOG: "CATALOG" >
+|   < CATALOG_NAME: "CATALOG_NAME" >
+|   < CEIL: "CEIL" >
+|   < CEILING: "CEILING" >
+|   < CENTURY: "CENTURY" >
+|   < CHAIN: "CHAIN" >
+|   < CHAR: "CHAR" >
+|   < CHAR_LENGTH: "CHAR_LENGTH" >
+|   < CHARACTER: "CHARACTER" >
+|   < CHARACTER_LENGTH: "CHARACTER_LENGTH" >
+|   < CHARACTER_SET_CATALOG: "CHARACTER_SET_CATALOG" >
+|   < CHARACTER_SET_NAME: "CHARACTER_SET_NAME" >
+|   < CHARACTER_SET_SCHEMA: "CHARACTER_SET_SCHEMA" >
+|   < CHARACTERISTICS: "CHARACTERISTICS" >
+|   < CHARACTERS: "CHARACTERS" >
+|   < CHECK: "CHECK" >
+|   < CLASSIFIER: "CLASSIFIER" >
+|   < CLASS_ORIGIN: "CLASS_ORIGIN" >
+|   < CLOB: "CLOB" >
+|   < CLOSE: "CLOSE" >
+|   < COALESCE: "COALESCE" >
+|   < COBOL: "COBOL" >
+|   < COLLATE: "COLLATE" >
+|   < COLLATION: "COLLATION" >
+|   < COLLATION_CATALOG: "COLLATION_CATALOG" >
+|   < COLLATION_NAME: "COLLATION_NAME" >
+|   < COLLATION_SCHEMA: "COLLATION_SCHEMA" >
+|   < COLLECT: "COLLECT" >
+|   < COLUMN: "COLUMN" >
+|   < COLUMN_NAME: "COLUMN_NAME" >
+|   < COMMAND_FUNCTION: "COMMAND_FUNCTION" >
+|   < COMMAND_FUNCTION_CODE: "COMMAND_FUNCTION_CODE" >
+|   < COMMIT: "COMMIT" >
+|   < COMMITTED: "COMMITTED" >
+|   < CONDITION: "CONDITION" >
+|   < CONDITIONAL: "CONDITIONAL" >
+|   < CONDITION_NUMBER: "CONDITION_NUMBER" >
+|   < CONNECT: "CONNECT" >
+|   < CONNECTION: "CONNECTION" >
+|   < CONNECTION_NAME: "CONNECTION_NAME" >
+|   < CONSTRAINT: "CONSTRAINT" >
+|   < CONSTRAINT_CATALOG: "CONSTRAINT_CATALOG" >
+|   < CONSTRAINT_NAME: "CONSTRAINT_NAME" >
+|   < CONSTRAINT_SCHEMA: "CONSTRAINT_SCHEMA" >
+|   < CONSTRAINTS: "CONSTRAINTS" >
+|   < CONSTRUCTOR: "CONSTRUCTOR" >
+|   < CONTAINS: "CONTAINS" >
+|   < CONTINUE: "CONTINUE" >
+|   < CONVERT: "CONVERT" >
+|   < CORR: "CORR" >
+|   < CORRESPONDING: "CORRESPONDING" >
+|   < COUNT: "COUNT" >
+|   < COVAR_POP: "COVAR_POP" >
+|   < COVAR_SAMP: "COVAR_SAMP" >
+|   < CREATE: "CREATE" >
+|   < CROSS: "CROSS" >
+|   < CUBE: "CUBE" >
+|   < CUME_DIST: "CUME_DIST" >
+|   < CURRENT: "CURRENT" >
+|   < CURRENT_CATALOG: "CURRENT_CATALOG" >
+|   < CURRENT_DATE: "CURRENT_DATE" >
+|   < CURRENT_DEFAULT_TRANSFORM_GROUP: "CURRENT_DEFAULT_TRANSFORM_GROUP" >
+|   < CURRENT_PATH: "CURRENT_PATH" >
+|   < CURRENT_ROLE: "CURRENT_ROLE" >
+|   < CURRENT_ROW: "CURRENT_ROW" >
+|   < CURRENT_SCHEMA: "CURRENT_SCHEMA" >
+|   < CURRENT_TIME: "CURRENT_TIME" >
+|   < CURRENT_TIMESTAMP: "CURRENT_TIMESTAMP" >
+|   < CURRENT_TRANSFORM_GROUP_FOR_TYPE: "CURRENT_TRANSFORM_GROUP_FOR_TYPE" >
+|   < CURRENT_USER: "CURRENT_USER" >
+|   < CURSOR: "CURSOR" >
+|   < CURSOR_NAME: "CURSOR_NAME" >
+|   < CYCLE: "CYCLE" >
+|   < DATA: "DATA" >
+|   < DATABASE: "DATABASE" >
+|   < DATE: "DATE" >
+|   < DATE_TRUNC: "DATE_TRUNC" >
+|   < DATETIME: "DATETIME" >
+|   < DATETIME_INTERVAL_CODE: "DATETIME_INTERVAL_CODE" >
+|   < DATETIME_INTERVAL_PRECISION: "DATETIME_INTERVAL_PRECISION" >
+|   < DAY: "DAY" >
+|   < DAYS: "DAYS" >
+|   < DEALLOCATE: "DEALLOCATE" >
+|   < DEC: "DEC" >
+|   < DECADE: "DECADE" >
+|   < DECIMAL: "DECIMAL" >
+|   < DECLARE: "DECLARE" >
+|   < DEFAULT_: "DEFAULT" >
+|   < DEFAULTS: "DEFAULTS" >
+|   < DEFERRABLE: "DEFERRABLE" >
+|   < DEFERRED: "DEFERRED" >
+|   < DEFINE: "DEFINE" >
+|   < DEFINED: "DEFINED" >
+|   < DEFINER: "DEFINER" >
+|   < DEGREE: "DEGREE" >
+|   < DELETE: "DELETE" > { beforeTableName(); }
+|   < DENSE_RANK: "DENSE_RANK" >
+|   < DEPTH: "DEPTH" >
+|   < DEREF: "DEREF" >
+|   < DERIVED: "DERIVED" >
+|   < DESC: "DESC" >
+|   < DESCRIBE: "DESCRIBE" > { beforeTableName(); }
+|   < DESCRIPTION: "DESCRIPTION" >
+|   < DESCRIPTOR: "DESCRIPTOR" >
+|   < DETERMINISTIC: "DETERMINISTIC" >
+|   < DIAGNOSTICS: "DIAGNOSTICS" >
+|   < DISALLOW: "DISALLOW" >
+|   < DISCONNECT: "DISCONNECT" >
+|   < DISPATCH: "DISPATCH" >
+|   < DISTINCT: "DISTINCT" >
+|   < DOMAIN: "DOMAIN" >
+|   < DOT_FORMAT: "DOT" >
+|   < DOUBLE: "DOUBLE" >
+|   < DOW: "DOW" >
+|   < DOY: "DOY" >
+|   < DROP: "DROP" >
+|   < DYNAMIC: "DYNAMIC" >
+|   < DYNAMIC_FUNCTION: "DYNAMIC_FUNCTION" >
+|   < DYNAMIC_FUNCTION_CODE: "DYNAMIC_FUNCTION_CODE" >
+|   < EACH: "EACH" >
+|   < ELEMENT: "ELEMENT" >
+|   < ELSE: "ELSE" >
+|   < EMPTY: "EMPTY" >
+|   < ENCODING: "ENCODING">
+|   < END: "END" >
+|   < END_EXEC: "END-EXEC" >
+|   < END_FRAME: "END_FRAME" >
+|   < END_PARTITION: "END_PARTITION" >
+|   < EPOCH: "EPOCH" >
+|   < EQUALS: "EQUALS" >
+|   < ERROR: "ERROR" >
+|   < ESCAPE: "ESCAPE" >
+|   < EVERY: "EVERY" >
+|   < EXCEPT: "EXCEPT" >
+|   < EXCEPTION: "EXCEPTION" >
+|   < EXCLUDE: "EXCLUDE" >
+|   < EXCLUDING: "EXCLUDING" >
+|   < EXEC: "EXEC" >
+|   < EXECUTE: "EXECUTE" >
+|   < EXISTS: "EXISTS" >
+|   < EXP: "EXP" >
+|   < EXPLAIN: "EXPLAIN" >
+|   < EXTEND: "EXTEND" >
+|   < EXTERNAL: "EXTERNAL" >
+|   < EXTRACT: "EXTRACT" >
+|   < FALSE: "FALSE" >
+|   < FETCH: "FETCH" >
+|   < FILTER: "FILTER" >
+|   < FINAL: "FINAL" >
+|   < FIRST: "FIRST" >
+|   < FIRST_VALUE: "FIRST_VALUE">
+|   < FLOAT: "FLOAT" >
+|   < FLOOR: "FLOOR" >
+|   < FOLLOWING: "FOLLOWING" >
+|   < FOR: "FOR" >
+|   < FORMAT: "FORMAT" >
+|   < FOREIGN: "FOREIGN" >
+|   < FORTRAN: "FORTRAN" >
+|   < FOUND: "FOUND" >
+|   < FRAC_SECOND: "FRAC_SECOND" >
+|   < FRAME_ROW: "FRAME_ROW" >
+|   < FREE: "FREE" >
+|   < FRIDAY: "FRIDAY" >
+|   < FROM: "FROM" > { beforeTableName(); }
+|   < FULL: "FULL" >
+|   < FUNCTION: "FUNCTION" >
+|   < FUSION: "FUSION" >
+|   < G: "G" >
+|   < GENERAL: "GENERAL" >
+|   < GENERATED: "GENERATED" >
+|   < GEOMETRY: "GEOMETRY" >
+|   < GET: "GET" >
+|   < GLOBAL: "GLOBAL" >
+|   < GO: "GO" >
+|   < GOTO: "GOTO" >
+|   < GRANT: "GRANT" >
+|   < GRANTED: "GRANTED" >
+|   < GROUP: "GROUP" >
+|   < GROUP_CONCAT: "GROUP_CONCAT" >
+|   < GROUPING: "GROUPING" >
+|   < GROUPS: "GROUPS" >
+|   < HAVING: "HAVING" >
+|   < HIERARCHY: "HIERARCHY" >
+|   < HOLD: "HOLD" >
+|   < HOP: "HOP" >
+|   < HOUR: "HOUR" >
+|   < HOURS: "HOURS" >
+|   < IDENTITY: "IDENTITY" >
+|   < IGNORE: "IGNORE" >
+|   < ILIKE: "ILIKE" >
+|   < IMMEDIATE: "IMMEDIATE" >
+|   < IMMEDIATELY: "IMMEDIATELY" >
+|   < IMPLEMENTATION: "IMPLEMENTATION" >
+|   < IMPORT: "IMPORT" >
+|   < IN: "IN" >
+|   < INCLUDE: "INCLUDE" >
+|   < INCLUDING: "INCLUDING" >
+|   < INCREMENT: "INCREMENT" >
+|   < INDICATOR: "INDICATOR" >
+|   < INITIAL: "INITIAL" >
+|   < INITIALLY: "INITIALLY" >
+|   < INNER: "INNER" >
+|   < INOUT: "INOUT" >
+|   < INPUT: "INPUT" >
+|   < INSENSITIVE: "INSENSITIVE" >
+|   < INSERT: "INSERT" > { beforeTableName(); }
+|   < INSTANCE: "INSTANCE" >
+|   < INSTANTIABLE: "INSTANTIABLE" >
+|   < INT: "INT" >
+|   < INTEGER: "INTEGER" >
+|   < INTERSECT: "INTERSECT" >
+|   < INTERSECTION: "INTERSECTION" >
+|   < INTERVAL: "INTERVAL" >
+|   < INTO: "INTO" >
+|   < INVOKER: "INVOKER" >
+|   < IS: "IS" >
+|   < ISODOW: "ISODOW" >
+|   < ISOYEAR: "ISOYEAR" >
+|   < ISOLATION: "ISOLATION" >
+|   < JAVA: "JAVA" >
+|   < JOIN: "JOIN" > { beforeTableName(); }
+|   < JSON: "JSON" >
+|   < JSON_ARRAY: "JSON_ARRAY">
+|   < JSON_ARRAYAGG: "JSON_ARRAYAGG">
+|   < JSON_EXISTS: "JSON_EXISTS" >
+|   < JSON_OBJECT: "JSON_OBJECT">
+|   < JSON_OBJECTAGG: "JSON_OBJECTAGG">
+|   < JSON_QUERY: "JSON_QUERY" >
+|   < JSON_VALUE: "JSON_VALUE" >
+|   < K: "K" >
+|   < KEY: "KEY" >
+|   < KEY_MEMBER: "KEY_MEMBER" >
+|   < KEY_TYPE: "KEY_TYPE" >
+|   < LABEL: "LABEL" >
+|   < LAG: "LAG" >
+|   < LANGUAGE: "LANGUAGE" >
+|   < LARGE: "LARGE" >
+|   < LAST: "LAST" >
+|   < LAST_VALUE: "LAST_VALUE" >
+|   < LATERAL: "LATERAL" >
+|   < LEAD: "LEAD" >
+|   < LEADING: "LEADING" >
+|   < LEFT: "LEFT" >
+|   < LENGTH: "LENGTH" >
+|   < LEVEL: "LEVEL" >
+|   < LIBRARY: "LIBRARY" >
+|   < LIKE: "LIKE" >
+|   < LIKE_REGEX: "LIKE_REGEX" >
+|   < LIMIT: "LIMIT" >
+|   < LN: "LN" >
+|   < LOCAL: "LOCAL" >
+|   < LOCALTIME: "LOCALTIME" >
+|   < LOCALTIMESTAMP: "LOCALTIMESTAMP" >
+|   < LOCATOR: "LOCATOR" >
+|   < LOWER: "LOWER" >
+|   < M: "M" >
+|   < MAP: "MAP" >
+|   < MATCH: "MATCH" >
+|   < MATCHED: "MATCHED" >
+|   < MATCHES: "MATCHES" >
+|   < MATCH_NUMBER: "MATCH_NUMBER">
+|   < MATCH_RECOGNIZE: "MATCH_RECOGNIZE">
+|   < MAX: "MAX" >
+|   < MAXVALUE: "MAXVALUE" >
+|   < MEASURES: "MEASURES" >
+|   < MEMBER: "MEMBER" >
+|   < MERGE: "MERGE" > { beforeTableName(); }
+|   < MESSAGE_LENGTH: "MESSAGE_LENGTH" >
+|   < MESSAGE_OCTET_LENGTH: "MESSAGE_OCTET_LENGTH" >
+|   < MESSAGE_TEXT: "MESSAGE_TEXT" >
+|   < METHOD: "METHOD" >
+|   < MICROSECOND: "MICROSECOND" >
+|   < MILLISECOND: "MILLISECOND" >
+|   < MILLENNIUM: "MILLENNIUM" >
+|   < MIN: "MIN" >
+|   < MINUTE: "MINUTE" >
+|   < MINUTES: "MINUTES" >
+|   < MINVALUE: "MINVALUE" >
+|   < MOD: "MOD" >
+|   < MODIFIES: "MODIFIES" >
+|   < MODULE: "MODULE" >
+|   < MONDAY: "MONDAY" >
+|   < MONTH: "MONTH" >
+|   < MONTHS: "MONTHS" >
+|   < MORE_: "MORE" >
+|   < MULTISET: "MULTISET" >
+|   < MUMPS: "MUMPS" >
+|   < NAME: "NAME" >
+|   < NAMES: "NAMES" >
+|   < NANOSECOND: "NANOSECOND" >
+|   < NATIONAL: "NATIONAL" >
+|   < NATURAL: "NATURAL" >
+|   < NCHAR: "NCHAR" >
+|   < NCLOB: "NCLOB" >
+|   < NESTING: "NESTING" >
+|   < NEW: "NEW" >
+|   < NEXT: "NEXT" >
+|   < NO: "NO" >
+|   < NONE: "NONE" >
+|   < NORMALIZE: "NORMALIZE" >
+|   < NORMALIZED: "NORMALIZED" >
+|   < NOT: "NOT" >
+|   < NTH_VALUE: "NTH_VALUE" >
+|   < NTILE: "NTILE" >
+|   < NULL: "NULL" >
+|   < NULLABLE: "NULLABLE" >
+|   < NULLIF: "NULLIF" >
+|   < NULLS: "NULLS" >
+|   < NUMBER: "NUMBER" >
+|   < NUMERIC: "NUMERIC" >
+|   < OBJECT: "OBJECT" >
+|   < OCCURRENCES_REGEX: "OCCURRENCES_REGEX" >
+|   < OCTET_LENGTH: "OCTET_LENGTH" >
+|   < OCTETS: "OCTETS" >
+|   < OF: "OF" >
+|   < OFFSET: "OFFSET" >
+|   < OLD: "OLD" >
+|   < OMIT: "OMIT" >
+|   < ON: "ON" >
+|   < ONE: "ONE" >
+|   < ONLY: "ONLY" >
+|   < OPEN: "OPEN" >
+|   < OPTION: "OPTION" >
+|   < OPTIONS: "OPTIONS" >
+|   < OR: "OR" >
+|   < ORDER: "ORDER" >
+|   < ORDERING: "ORDERING" >
+|   < ORDINALITY: "ORDINALITY" >
+|   < OTHERS: "OTHERS" >
+|   < OUT: "OUT" >
+|   < OUTER: "OUTER" >
+|   < OUTPUT: "OUTPUT" >
+|   < OVER: "OVER" >
+|   < OVERLAPS: "OVERLAPS" >
+|   < OVERLAY: "OVERLAY" >
+|   < OVERRIDING: "OVERRIDING" >
+|   < PAD: "PAD" >
+|   < PARAMETER: "PARAMETER" >
+|   < PARAMETER_MODE: "PARAMETER_MODE" >
+|   < PARAMETER_NAME: "PARAMETER_NAME" >
+|   < PARAMETER_ORDINAL_POSITION: "PARAMETER_ORDINAL_POSITION" >
+|   < PARAMETER_SPECIFIC_CATALOG: "PARAMETER_SPECIFIC_CATALOG" >
+|   < PARAMETER_SPECIFIC_NAME: "PARAMETER_SPECIFIC_NAME" >
+|   < PARAMETER_SPECIFIC_SCHEMA: "PARAMETER_SPECIFIC_SCHEMA" >
+|   < PARTIAL: "PARTIAL" >
+|   < PARTITION: "PARTITION" >
+|   < PASCAL: "PASCAL" >
+|   < PASSING: "PASSING" >
+|   < PASSTHROUGH: "PASSTHROUGH" >
+|   < PAST: "PAST" >
+|   < PATH: "PATH" >
+|   < PATTERN: "PATTERN" >
+|   < PER: "PER" >
+|   < PERCENT: "PERCENT" >
+|   < PERCENTILE_CONT: "PERCENTILE_CONT" >
+|   < PERCENTILE_DISC: "PERCENTILE_DISC" >
+|   < PERCENT_RANK: "PERCENT_RANK" >
+|   < PERIOD: "PERIOD" >
+|   < PERMUTE: "PERMUTE" >
+|   < PIVOT: "PIVOT" >
+|   < PLACING: "PLACING" >
+|   < PLAN: "PLAN" >
+|   < PLI: "PLI" >
+|   < PORTION: "PORTION" >
+|   < POSITION: "POSITION" >
+|   < POSITION_REGEX: "POSITION_REGEX" >
+|   < POWER: "POWER" >
+|   < PRECEDES: "PRECEDES" >
+|   < PRECEDING: "PRECEDING" >
+|   < PRECISION: "PRECISION" >
+|   < PREPARE: "PREPARE" >
+|   < PRESERVE: "PRESERVE" >
+|   < PREV: "PREV" >
+|   < PRIMARY: "PRIMARY" >
+|   < PRIOR: "PRIOR" >
+|   < PRIVILEGES: "PRIVILEGES" >
+|   < PROCEDURE: "PROCEDURE" >
+|   < PUBLIC: "PUBLIC" >
+|   < QUALIFY: "QUALIFY" >
+|   < QUARTER: "QUARTER" >
+|   < QUARTERS: "QUARTERS" >
+|   < RANGE: "RANGE" >
+|   < RANK: "RANK" >
+|   < READ: "READ" >
+|   < READS: "READS" >
+|   < REAL: "REAL" >
+|   < RECURSIVE: "RECURSIVE" >
+|   < REF: "REF" >
+|   < REFERENCES: "REFERENCES" >
+|   < REFERENCING: "REFERENCING" >
+|   < REGR_AVGX: "REGR_AVGX" >
+|   < REGR_AVGY: "REGR_AVGY" >
+|   < REGR_COUNT: "REGR_COUNT" >
+|   < REGR_INTERCEPT: "REGR_INTERCEPT" >
+|   < REGR_R2: "REGR_R2" >
+|   < REGR_SLOPE: "REGR_SLOPE" >
+|   < REGR_SXX: "REGR_SXX" >
+|   < REGR_SXY: "REGR_SXY" >
+|   < REGR_SYY: "REGR_SYY" >
+|   < RELATIVE: "RELATIVE" >
+|   < RELEASE: "RELEASE" >
+|   < REPEATABLE: "REPEATABLE" >
+|   < REPLACE: "REPLACE" >
+|   < RESET: "RESET" >
+|   < RESPECT: "RESPECT" >
+|   < RESTART: "RESTART" >
+|   < RESTRICT: "RESTRICT" >
+|   < RESULT: "RESULT" >
+|   < RETURN: "RETURN" >
+|   < RETURNED_CARDINALITY: "RETURNED_CARDINALITY" >
+|   < RETURNED_LENGTH: "RETURNED_LENGTH" >
+|   < RETURNED_OCTET_LENGTH: "RETURNED_OCTET_LENGTH" >
+|   < RETURNED_SQLSTATE: "RETURNED_SQLSTATE" >
+|   < RETURNING: "RETURNING" >
+|   < RETURNS: "RETURNS" >
+|   < REVOKE: "REVOKE" >
+|   < RIGHT: "RIGHT" >
+|   < RLIKE: "RLIKE" >
+|   < ROLE: "ROLE" >
+|   < ROLLBACK: "ROLLBACK" >
+|   < ROLLUP: "ROLLUP" >
+|   < ROUTINE: "ROUTINE" >
+|   < ROUTINE_CATALOG: "ROUTINE_CATALOG" >
+|   < ROUTINE_NAME: "ROUTINE_NAME" >
+|   < ROUTINE_SCHEMA: "ROUTINE_SCHEMA" >
+|   < ROW: "ROW" >
+|   < ROW_COUNT: "ROW_COUNT" >
+|   < ROW_NUMBER: "ROW_NUMBER" >
+|   < ROWS: "ROWS" >
+|   < RUNNING: "RUNNING" >
+|   < SATURDAY: "SATURDAY" >
+|   < SAVEPOINT: "SAVEPOINT" >
+|   < SCALAR: "SCALAR" >
+|   < SCALE: "SCALE" >
+|   < SCHEMA: "SCHEMA" >
+|   < SCHEMA_NAME: "SCHEMA_NAME" >
+|   < SCOPE: "SCOPE" >
+|   < SCOPE_CATALOGS: "SCOPE_CATALOGS" >
+|   < SCOPE_NAME: "SCOPE_NAME" >
+|   < SCOPE_SCHEMA: "SCOPE_SCHEMA" >
+|   < SCROLL: "SCROLL" >
+|   < SEARCH: "SEARCH" >
+|   < SECOND: "SECOND" >
+|   < SECONDS: "SECONDS" >
+|   < SECTION: "SECTION" >
+|   < SECURITY: "SECURITY" >
+|   < SEEK: "SEEK" >
+|   < SELECT: "SELECT" > { afterTableName(); }
+|   < SELF: "SELF" >
+|   < SENSITIVE: "SENSITIVE" >
+|   < SEPARATOR: "SEPARATOR" >
+|   < SEQUENCE: "SEQUENCE" >
+|   < SERIALIZABLE: "SERIALIZABLE" >
+|   < SERVER: "SERVER" >
+|   < SERVER_NAME: "SERVER_NAME" >
+|   < SESSION: "SESSION" >
+|   < SESSION_USER: "SESSION_USER" >
+|   < SET: "SET" > { afterTableName(); }
+|   < SETS: "SETS" >
+|   < SET_MINUS: "MINUS">
+|   < SHOW: "SHOW" >
+|   < SIMILAR: "SIMILAR" >
+|   < SIMPLE: "SIMPLE" >
+|   < SIZE: "SIZE" >
+|   < SKIP_: "SKIP" >
+|   < SMALLINT: "SMALLINT" >
+|   < SOME: "SOME" >
+|   < SOURCE: "SOURCE" >
+|   < SPACE: "SPACE" >
+|   < SPECIFIC: "SPECIFIC" >
+|   < SPECIFIC_NAME: "SPECIFIC_NAME" >
+|   < SPECIFICTYPE: "SPECIFICTYPE" >
+|   < SQL: "SQL" >
+|   < SQLEXCEPTION: "SQLEXCEPTION" >
+|   < SQLSTATE: "SQLSTATE" >
+|   < SQLWARNING: "SQLWARNING" >
+|   < SQL_BIGINT: "SQL_BIGINT" >
+|   < SQL_BINARY: "SQL_BINARY" >
+|   < SQL_BIT: "SQL_BIT" >
+|   < SQL_BLOB: "SQL_BLOB" >
+|   < SQL_BOOLEAN: "SQL_BOOLEAN" >
+|   < SQL_CHAR: "SQL_CHAR" >
+|   < SQL_CLOB: "SQL_CLOB" >
+|   < SQL_DATE: "SQL_DATE" >
+|   < SQL_DECIMAL: "SQL_DECIMAL" >
+|   < SQL_DOUBLE: "SQL_DOUBLE" >
+|   < SQL_FLOAT: "SQL_FLOAT" >
+|   < SQL_INTEGER: "SQL_INTEGER" >
+|   < SQL_INTERVAL_DAY: "SQL_INTERVAL_DAY" >
+|   < SQL_INTERVAL_DAY_TO_HOUR: "SQL_INTERVAL_DAY_TO_HOUR" >
+|   < SQL_INTERVAL_DAY_TO_MINUTE: "SQL_INTERVAL_DAY_TO_MINUTE" >
+|   < SQL_INTERVAL_DAY_TO_SECOND: "SQL_INTERVAL_DAY_TO_SECOND" >
+|   < SQL_INTERVAL_HOUR: "SQL_INTERVAL_HOUR" >
+|   < SQL_INTERVAL_HOUR_TO_MINUTE: "SQL_INTERVAL_HOUR_TO_MINUTE" >
+|   < SQL_INTERVAL_HOUR_TO_SECOND: "SQL_INTERVAL_HOUR_TO_SECOND" >
+|   < SQL_INTERVAL_MINUTE: "SQL_INTERVAL_MINUTE" >
+|   < SQL_INTERVAL_MINUTE_TO_SECOND: "SQL_INTERVAL_MINUTE_TO_SECOND" >
+|   < SQL_INTERVAL_MONTH: "SQL_INTERVAL_MONTH" >
+|   < SQL_INTERVAL_SECOND: "SQL_INTERVAL_SECOND" >
+|   < SQL_INTERVAL_YEAR: "SQL_INTERVAL_YEAR" >
+|   < SQL_INTERVAL_YEAR_TO_MONTH: "SQL_INTERVAL_YEAR_TO_MONTH" >
+|   < SQL_LONGVARBINARY: "SQL_LONGVARBINARY" >
+|   < SQL_LONGVARCHAR: "SQL_LONGVARCHAR" >
+|   < SQL_LONGVARNCHAR: "SQL_LONGVARNCHAR" >
+|   < SQL_NCHAR: "SQL_NCHAR" >
+|   < SQL_NCLOB: "SQL_NCLOB" >
+|   < SQL_NUMERIC: "SQL_NUMERIC" >
+|   < SQL_NVARCHAR: "SQL_NVARCHAR" >
+|   < SQL_REAL: "SQL_REAL" >
+|   < SQL_SMALLINT: "SQL_SMALLINT" >
+|   < SQL_TIME: "SQL_TIME" >
+|   < SQL_TIMESTAMP: "SQL_TIMESTAMP" >
+|   < SQL_TINYINT: "SQL_TINYINT" >
+|   < SQL_TSI_DAY: "SQL_TSI_DAY" >
+|   < SQL_TSI_FRAC_SECOND: "SQL_TSI_FRAC_SECOND" >
+|   < SQL_TSI_HOUR: "SQL_TSI_HOUR" >
+|   < SQL_TSI_MICROSECOND: "SQL_TSI_MICROSECOND" >
+|   < SQL_TSI_MINUTE: "SQL_TSI_MINUTE" >
+|   < SQL_TSI_MONTH: "SQL_TSI_MONTH" >
+|   < SQL_TSI_QUARTER: "SQL_TSI_QUARTER" >
+|   < SQL_TSI_SECOND: "SQL_TSI_SECOND" >
+|   < SQL_TSI_WEEK: "SQL_TSI_WEEK" >
+|   < SQL_TSI_YEAR: "SQL_TSI_YEAR" >
+|   < SQL_VARBINARY: "SQL_VARBINARY" >
+|   < SQL_VARCHAR: "SQL_VARCHAR" >
+|   < SQRT: "SQRT" >
+|   < START: "START" >
+|   < STATE: "STATE" >
+|   < STATEMENT: "STATEMENT" >
+|   < STATIC: "STATIC" >
+|   < STDDEV_POP: "STDDEV_POP" >
+|   < STDDEV_SAMP: "STDDEV_SAMP" >
+|   < STREAM: "STREAM" >
+|   < STRING_AGG: "STRING_AGG" >
+|   < STRUCTURE: "STRUCTURE" >
+|   < STYLE: "STYLE" >
+|   < SUBCLASS_ORIGIN: "SUBCLASS_ORIGIN" >
+|   < SUBMULTISET: "SUBMULTISET" >
+|   < SUBSET: "SUBSET" >
+|   < SUBSTITUTE: "SUBSTITUTE" >
+|   < SUBSTRING: "SUBSTRING" >
+|   < SUBSTRING_REGEX: "SUBSTRING_REGEX" >
+|   < SUCCEEDS: "SUCCEEDS" >
+|   < SUM: "SUM" >
+|   < SUNDAY: "SUNDAY" >
+|   < SYMMETRIC: "SYMMETRIC" >
+|   < SYSTEM: "SYSTEM" >
+|   < SYSTEM_TIME: "SYSTEM_TIME" >
+|   < SYSTEM_USER: "SYSTEM_USER" >
+|   < TABLE: "TABLE" > { beforeTableName(); }
+|   < TABLE_NAME: "TABLE_NAME" >
+|   < TABLESAMPLE: "TABLESAMPLE" >
+|   < TEMPORARY: "TEMPORARY" >
+|   < THEN: "THEN" >
+|   < THURSDAY: "THURSDAY" >
+|   < TIES: "TIES" >
+|   < TIME: "TIME" >
+|   < TIME_DIFF: "TIME_DIFF" >
+|   < TIME_TRUNC: "TIME_TRUNC" >
+|   < TIMESTAMP: "TIMESTAMP" >
+|   < TIMESTAMPADD: "TIMESTAMPADD" >
+|   < TIMESTAMPDIFF: "TIMESTAMPDIFF" >
+|   < TIMESTAMP_DIFF: "TIMESTAMP_DIFF" >
+|   < TIMESTAMP_TRUNC: "TIMESTAMP_TRUNC" >
+|   < TIMEZONE_HOUR: "TIMEZONE_HOUR" >
+|   < TIMEZONE_MINUTE: "TIMEZONE_MINUTE" >
+|   < TINYINT: "TINYINT" >
+|   < TO: "TO" >
+|   < TOP_LEVEL_COUNT: "TOP_LEVEL_COUNT" >
+|   < TRAILING: "TRAILING" >
+|   < TRANSACTION: "TRANSACTION" >
+|   < TRANSACTIONS_ACTIVE: "TRANSACTIONS_ACTIVE" >
+|   < TRANSACTIONS_COMMITTED: "TRANSACTIONS_COMMITTED" >
+|   < TRANSACTIONS_ROLLED_BACK: "TRANSACTIONS_ROLLED_BACK" >
+|   < TRANSFORM: "TRANSFORM" >
+|   < TRANSFORMS: "TRANSFORMS" >
+|   < TRANSLATE: "TRANSLATE" >
+|   < TRANSLATE_REGEX: "TRANSLATE_REGEX" >
+|   < TRANSLATION: "TRANSLATION" >
+|   < TREAT: "TREAT" >
+|   < TRIGGER: "TRIGGER" >
+|   < TRIGGER_CATALOG: "TRIGGER_CATALOG" >
+|   < TRIGGER_NAME: "TRIGGER_NAME" >
+|   < TRIGGER_SCHEMA: "TRIGGER_SCHEMA" >
+|   < TRIM: "TRIM" >
+|   < TRIM_ARRAY: "TRIM_ARRAY" >
+|   < TRUE: "TRUE" >
+|   < TRUNCATE: "TRUNCATE" >
+|   < TUESDAY: "TUESDAY" >
+|   < TUMBLE: "TUMBLE" >
+|   < TYPE: "TYPE" >
+|   < UESCAPE: "UESCAPE" >
+|   < UNBOUNDED: "UNBOUNDED" >
+|   < UNCOMMITTED: "UNCOMMITTED" >
+|   < UNCONDITIONAL: "UNCONDITIONAL" >
+|   < UNDER: "UNDER" >
+|   < UNION: "UNION" >
+|   < UNIQUE: "UNIQUE" >
+|   < UNKNOWN: "UNKNOWN" >
+|   < UNPIVOT: "UNPIVOT" >
+|   < UNNAMED: "UNNAMED" >
+|   < UNNEST: "UNNEST" >
+|   < UPDATE: "UPDATE" > { beforeTableName(); }
+|   < UPPER: "UPPER" >
+|   < UPSERT: "UPSERT" >
+|   < USAGE: "USAGE" >
+|   < USER: "USER" >
+|   < USER_DEFINED_TYPE_CATALOG: "USER_DEFINED_TYPE_CATALOG" >
+|   < USER_DEFINED_TYPE_CODE: "USER_DEFINED_TYPE_CODE" >
+|   < USER_DEFINED_TYPE_NAME: "USER_DEFINED_TYPE_NAME" >
+|   < USER_DEFINED_TYPE_SCHEMA: "USER_DEFINED_TYPE_SCHEMA" >
+|   < USING: "USING" >
+|   < UTF8: "UTF8" >
+|   < UTF16: "UTF16" >
+|   < UTF32: "UTF32" >
+|   < VALUE: "VALUE" >
+|   < VALUES: "VALUES" > { afterTableName(); }
+|   < VALUE_OF: "VALUE_OF" >
+|   < VAR_POP: "VAR_POP" >
+|   < VAR_SAMP: "VAR_SAMP" >
+|   < VARBINARY: "VARBINARY" >
+|   < VARCHAR: "VARCHAR" >
+|   < VARYING: "VARYING" >
+|   < VERSION: "VERSION" >
+|   < VERSIONING: "VERSIONING" >
+|   < VIEW: "VIEW" >
+|   < WEDNESDAY: "WEDNESDAY" >
+|   < WEEK: "WEEK" >
+|   < WEEKS: "WEEKS" >
+|   < WHEN: "WHEN" >
+|   < WHENEVER: "WHENEVER" >
+|   < WHERE: "WHERE" >
+|   < WIDTH_BUCKET: "WIDTH_BUCKET" >
+|   < WINDOW: "WINDOW" >
+|   < WITH: "WITH" >
+|   < WITHIN: "WITHIN" >
+|   < WITHOUT: "WITHOUT" >
+|   < WORK: "WORK" >
+|   < WRAPPER: "WRAPPER" >
+|   < WRITE: "WRITE" >
+|   < XML: "XML" >
+|   < YEAR: "YEAR" >
+|   < YEARS: "YEARS" >
+|   < ZONE: "ZONE" >
+<#-- additional parser keywords are included here -->
+<#list (parser.keywords!default.parser.keywords) as keyword>
+|   < ${keyword}: "${keyword}" >
+</#list>
+}
+
+/**
+ * Parses a non-reserved keyword for use as an identifier.
+ *
+ * <p>The method is broken up into several sub-methods; without this
+ * decomposition, parsers such as Babel with more than ~1,000 non-reserved
+ * keywords would generate such deeply nested 'if' statements that javac would
+ * fail with a {@link StackOverflowError}.
+ *
+ * <p>The list is generated from the FMPP config data. To add or remove
+ * keywords, modify config.fmpp. For parsers except Babel, make sure that
+ * keywords are not reserved by the SQL standard.
+ *
+ * @see Glossary#SQL2003 SQL:2003 Part 2 Section 5.2
+ */
+String NonReservedKeyWord() :
+{
+}
+{
+    (
+        NonReservedKeyWord0of3()
+    |   NonReservedKeyWord1of3()
+    |   NonReservedKeyWord2of3()
+    )
+    {
+        return unquotedIdentifier();
+    }
+}
+
+/** @see #NonReservedKeyWord */
+void NonReservedKeyWord0of3() :
+{
+}
+{
+    (
+<#list (parser.nonReservedKeywords!default.parser.nonReservedKeywords) + (parser.nonReservedKeywordsToAdd!default.parser.nonReservedKeywordsToAdd) as keyword>
+<#if keyword?index == 0>
+        <${keyword}>
+<#elseif keyword?index % 3 == 0>
+    |   <${keyword}>
+</#if>
+</#list>
+    )
+}
+
+/** @see #NonReservedKeyWord */
+void NonReservedKeyWord1of3() :
+{
+}
+{
+    (
+<#list (parser.nonReservedKeywords!default.parser.nonReservedKeywords) + (parser.nonReservedKeywordsToAdd!default.parser.nonReservedKeywordsToAdd) as keyword>
+<#if keyword?index == 1>
+        <${keyword}>
+<#elseif keyword?index % 3 == 1>
+    |   <${keyword}>
+</#if>
+</#list>
+    )
+}
+
+/** @see #NonReservedKeyWord */
+void NonReservedKeyWord2of3() :
+{
+}
+{
+    (
+<#list (parser.nonReservedKeywords!default.parser.nonReservedKeywords) + (parser.nonReservedKeywordsToAdd!default.parser.nonReservedKeywordsToAdd) as keyword>
+<#if keyword?index == 2>
+        <${keyword}>
+<#elseif keyword?index % 3 == 2>
+    |   <${keyword}>
+</#if>
+</#list>
+    )
+}
+
+/* LITERALS */
+
+<DEFAULT, DQID, BTID, BQID, BQHID> TOKEN :
+{
+    < UNSIGNED_INTEGER_LITERAL: (["0"-"9"])+ >
+|
+    < APPROX_NUMERIC_LITERAL:
+    (<UNSIGNED_INTEGER_LITERAL> | <DECIMAL_NUMERIC_LITERAL>) <EXPONENT> >
+|
+    < DECIMAL_NUMERIC_LITERAL:
+    (["0"-"9"])+(".")?(["0"-"9"])*
+    | "."(["0"-"9"])+
+    >
+|
+    < #EXPONENT: ["e","E"] (["+","-"])? (["0"-"9"])+ >
+|
+    < #HEXDIGIT: ["0"-"9","a"-"f","A"-"F"] >
+|
+    < #WHITESPACE:
+    [ " ","\t","\n","\r","\f" ]
+    >
+|
+    /* To improve error reporting, we allow all kinds of characters,
+     * not just hexits, in a binary string literal. */
+    < BINARY_STRING_LITERAL: ["x","X"] <QUOTE> ( (~["'"]) | ("''"))* <QUOTE> >
+}
+
+// All databases except BigQuery support standard single-quoted literals,
+// which use single-quote as the escape character.
+<DEFAULT, DQID, BTID> TOKEN :
+{
+    < QUOTED_STRING: <QUOTE> ( (~["'"]) | ("''"))* <QUOTE> >
+|
+    < PREFIXED_STRING_LITERAL: ("_" <CHARSETNAME> | "N") <QUOTED_STRING> >
+|
+    < UNICODE_STRING_LITERAL: "U" "&" <QUOTED_STRING> >
+|
+    < C_STYLE_ESCAPED_STRING_LITERAL: "E" <QUOTE> ( (~["'", "\\"]) | ("\\" ~[]) | "''")* <QUOTE> >
+|
+    < #CHARSETNAME: (["a"-"z","A"-"Z","0"-"9"])
+    (["a"-"z","A"-"Z","0"-"9",":",".","-","_"])*
+    >
+}
+
+// BigQuery supports single- and double-quoted literals with back-slash
+// as the escape character.
+<BQID, BQHID> TOKEN :
+{
+    // BigQuery-style double-quoted string, escaped using backslash
+    < BIG_QUERY_DOUBLE_QUOTED_STRING:
+      <DOUBLE_QUOTE> ( (~["\\", "\""]) | ("\\" ~[]) )* <DOUBLE_QUOTE>
+    >
+|
+    // BigQuery-style single-quoted string, escaped using backslash
+    < BIG_QUERY_QUOTED_STRING:
+      <QUOTE> ( (~["\\", "'"]) | ("\\" ~[]) )* <QUOTE>
+    >
+}
+
+<DEFAULT, DQID, BTID, BQID, BQHID> TOKEN :
+{
+    < UNICODE_QUOTED_ESCAPE_CHAR:
+    <QUOTE>
+    (~["0"-"9","a"-"f","A"-"F","+","\""," ","\t","\n","\r","\f"])
+    <QUOTE>
+    >
+}
+
+/* SEPARATORS */
+
+<DEFAULT, DQID, BTID, BQID, BQHID> TOKEN :
+{
+    < LPAREN: "(">
+|   < RPAREN: ")">
+<#if (parser.includeBraces!default.parser.includeBraces) >
+|   < LBRACE_D: "{" (" ")* ["d","D"] >
+|   < LBRACE_T: "{" (" ")* ["t","T"] >
+|   < LBRACE_TS: "{" (" ")* ["t","T"] ["s","S"] >
+|   < LBRACE_FN: "{" (" ")* ["f","F"] ["n","N"] >
+|   < LBRACE: "{" >
+|   < RBRACE: "}" >
+<#else>
+<#include "/@includes/braces.ftl" />
+</#if>
+|   < LBRACKET: "[" >
+|   < RBRACKET: "]" >
+|   < SEMICOLON: ";" >
+|   < DOT: "." >
+|   < COMMA: "," >
+}
+
+/* OPERATORS */
+
+<DEFAULT, DQID, BTID, BQID, BQHID> TOKEN :
+{
+    < EQ: "=" >
+|   < GT: ">" >
+|   < LT: "<" >
+|   < HOOK: "?" >
+|   < COLON: ":" >
+|   < LE: "<=" >
+|   < GE: ">=" >
+|   < NE: "<>" >
+|   < NE2: "!=" >
+|   < PLUS: "+" >
+|   < MINUS: "-" >
+|   < STAR: "*" >
+|   < SLASH: "/" >
+|   < PERCENT_REMAINDER: "%" >
+|   < CONCAT: "||" >
+|   < NAMED_ARGUMENT_ASSIGNMENT: "=>" >
+|   < DOUBLE_PERIOD: ".." >
+|   < QUOTE: "'" >
+|   < DOUBLE_QUOTE: "\"" >
+|   < VERTICAL_BAR: "|" >
+|   < CARET: "^" >
+|   < DOLLAR: "$" >
+<#list (parser.binaryOperatorsTokens!default.parser.binaryOperatorsTokens) as operator>
+|   ${operator}
+</#list>
+}
+
+
+/*****************************************
+ * Lexical Descriptions                  *
+ *****************************************/
+
+TOKEN_MGR_DECLS : {
+    final List<Integer> lexicalStateStack = new ArrayList<Integer>();
+
+    void pushState() {
+      lexicalStateStack.add(curLexState);
+    }
+
+    void popState() {
+      SwitchTo(lexicalStateStack.remove(lexicalStateStack.size() - 1));
+    }
+
+    void beforeTableName() {
+      if (curLexState == BQID) {
+        pushState();
+        SwitchTo(BQHID);
+      }
+    }
+
+    void afterTableName() {
+      if (curLexState == BQHID) {
+        popState();
+      }
+    }
+
+<#if (parser.includeAdditionalDeclarations!default.parser.includeAdditionalDeclarations)>
+  <#include "/@includes/tokenManagerDeclarations.ftl" />
+</#if>
+}
+
+/*
+Lexical states:
+
+DEFAULT: Identifiers are quoted in brackets, e.g. [My Identifier]
+DQID:    Identifiers are double-quoted, e.g. "My Identifier"
+BTID:    Identifiers are enclosed in back-ticks, escaped using back-ticks,
+         e.g. `My ``Quoted`` Identifier`
+BQID:    Identifiers are enclosed in back-ticks, escaped using backslash,
+         e.g. `My \`Quoted\` Identifier`,
+         and with the potential to shift into BQHID in contexts where table
+         names are expected, and thus allow hyphen-separated identifiers as
+         part of table names
+BQHID:   Identifiers are enclosed in back-ticks, escaped using backslash,
+         e.g. `My \`Quoted\` Identifier`
+         and unquoted identifiers may contain hyphens, e.g. foo-bar
+IN_SINGLE_LINE_COMMENT:
+IN_FORMAL_COMMENT:
+IN_MULTI_LINE_COMMENT:
+
+DEFAULT, DQID, BTID, BQID are the 4 'normal states'. Behavior is identical
+except for how quoted identifiers are recognized.
+
+The BQHID state exists only at the start of a table name (e.g. immediately after
+FROM or INSERT INTO). As soon as an identifier is seen, the state shifts back
+to BTID.
+
+After a comment has completed, the lexer returns to the previous state, one
+of the 'normal states'.
+*/
+
+/* WHITE SPACE */
+
+<DEFAULT, DQID, BTID, BQID, BQHID> SKIP :
+{
+    " "
+|   "\t"
+|   "\n"
+|   "\r"
+|   "\f"
+}
+
+/* COMMENTS */
+
+<DEFAULT, DQID, BTID, BQID, BQHID> TOKEN :
+{
+    < HINT_BEG: "/*+">
+|   < COMMENT_END: "*/" >
+}
+
+<DEFAULT, DQID, BTID, BQID, BQHID> MORE :
+{
+    <"/**" ~["/"]> { pushState(); } : IN_FORMAL_COMMENT
+}
+
+<DEFAULT, DQID, BTID, BQID, BQHID> MORE :
+{
+    "/*" { pushState(); } : IN_MULTI_LINE_COMMENT
+}
+
+<DEFAULT, DQID, BTID, BQID, BQHID> SKIP :
+{
+    <SINGLE_LINE_COMMENT: ("//"|"--")(~["\n","\r"])* ("\n"|"\r"|"\r\n")? >
+}
+
+<IN_FORMAL_COMMENT>
+SPECIAL_TOKEN :
+{
+    <FORMAL_COMMENT: <COMMENT_END> > { popState(); }
+}
+
+<IN_MULTI_LINE_COMMENT>
+SPECIAL_TOKEN :
+{
+    <MULTI_LINE_COMMENT: <COMMENT_END> > { popState(); }
+}
+
+<IN_FORMAL_COMMENT,IN_MULTI_LINE_COMMENT>
+MORE :
+{
+    < ~[] >
+}
+
+
+/* IDENTIFIERS */
+
+<DEFAULT> TOKEN :
+{
+    < BRACKET_QUOTED_IDENTIFIER:
+    "["
+    (
+        (~["]","\n","\r"])
+    |
+        ("]]")
+    )+
+    "]"
+    >
+}
+
+<DQID> TOKEN :
+{
+    < QUOTED_IDENTIFIER:
+    "\""
+    (
+        (~["\"","\n","\r"])
+    |
+        ("\"\"")
+    )+
+    "\""
+    >
+}
+
+<BTID> TOKEN :
+{
+    < BACK_QUOTED_IDENTIFIER:
+    "`"
+    (
+        (~["`","\n","\r"])
+    |
+        ("``")
+    )+
+    "`"
+    >
+}
+
+<BQID, BQHID> TOKEN :
+{
+    // BigQuery-style backtick-quoted identifier, escaped using backslash
+    < BIG_QUERY_BACK_QUOTED_IDENTIFIER:
+    "`"
+    (
+        (~["\\", "`"])
+    |
+        ("\\" ~[])
+    )*
+    "`"
+    >
+}
+
+<BQHID> TOKEN :
+{
+    // Per BigQuery: "Project IDs must contain 6-63 lowercase letters, digits,
+    // or dashes. IDs must start with a letter and may not end with a dash."
+    // We do not restrict length, or prevent identifiers from ending in a dash.
+    < HYPHENATED_IDENTIFIER: <LETTER> (<LETTER>|<DIGIT>|"-")* > { popState(); }
+}
+
+<DEFAULT, DQID, BTID, BQID> TOKEN :
+{
+    < IDENTIFIER: <LETTER> (<LETTER>|<DIGIT>)* >
+}
+
+<DEFAULT, DQID, BTID, BQID, BQHID> TOKEN :
+{
+    < COLLATION_ID:
+    (<LETTER>|<DIGIT>)+ (<LETTER>|<DIGIT>|":"|"."|"-"|"_")*
+    "$"
+    (<LETTER>|"_")+
+    ("$" (<LETTER>|<DIGIT>|"_")+)?
+    >
+|
+    < UNICODE_QUOTED_IDENTIFIER: "U" "&" <QUOTED_IDENTIFIER> >
+|
+    < #LETTER:
+    [
+        "\u0024",
+        "\u0041"-"\u005a",
+        "\u005f",
+        "\u0061"-"\u007a",
+        "\u00c0"-"\u00d6",
+        "\u00d8"-"\u00f6",
+        "\u00f8"-"\u00ff",
+        "\u0100"-"\u1fff",
+        "\u3040"-"\u318f",
+        "\u3300"-"\u337f",
+        "\u3400"-"\u3d2d",
+        "\u4e00"-"\u9fff",
+        "\uf900"-"\ufaff"
+    ]
+    >
+|
+    < #DIGIT:
+    [
+        "\u0030"-"\u0039",
+        "\u0660"-"\u0669",
+        "\u06f0"-"\u06f9",
+        "\u0966"-"\u096f",
+        "\u09e6"-"\u09ef",
+        "\u0a66"-"\u0a6f",
+        "\u0ae6"-"\u0aef",
+        "\u0b66"-"\u0b6f",
+        "\u0be7"-"\u0bef",
+        "\u0c66"-"\u0c6f",
+        "\u0ce6"-"\u0cef",
+        "\u0d66"-"\u0d6f",
+        "\u0e50"-"\u0e59",
+        "\u0ed0"-"\u0ed9",
+        "\u1040"-"\u1049"
+    ]
+    >
+}
+
+/* Special token to throw a wrench in the works. It is never valid in SQL,
+   and so when it occurs, it causes the parser to print which tokens would
+   have been valid at that point. Used by SqlAdvisor. */
+<DEFAULT, DQID, BTID, BQID, BQHID> TOKEN :
+{
+    < BEL:
+    [
+        "\u0007"
+    ]
+    >
+}
+
+/**
+ * Defines a production which can never be accepted by the parser.
+ * In effect, it tells the parser, "If you got here, you've gone too far."
+ * It is used as the default production for parser extension points;
+ * derived parsers replace it with a real production when they want to
+ * implement a particular extension point.
+ */
+void UnusedExtension() :
+{
+}
+{
+    (
+        LOOKAHEAD({false}) <ZONE>
+    )
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlSelectBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlSelectBuilder.java
index a6653900f9..07dcf01280 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlSelectBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlSelectBuilder.java
@@ -31,6 +31,7 @@ public class SqlSelectBuilder {
   private SqlNodeList groupBy;
   private SqlNode having;
   private SqlNodeList windowDecls;
+  private SqlNode qualify;
   private SqlNodeList orderBy;
   private SqlNode offset;
   private SqlNode fetch;
@@ -76,6 +77,11 @@ public class SqlSelectBuilder {
     return this;
   }
 
+  public SqlSelectBuilder qualify(SqlNode qualify) {
+    this.qualify = qualify;
+    return this;
+  }
+
   public SqlSelectBuilder orderBy(SqlNodeList orderBy) {
     this.orderBy = orderBy;
     return this;
@@ -98,7 +104,7 @@ public class SqlSelectBuilder {
 
   public SqlSelect build() {
     return new SqlSelect(parserPosition, keywordList, selectList, from,
-      where, groupBy, having, windowDecls, orderBy, offset, fetch, hints);
+      where, groupBy, having, windowDecls, qualify, orderBy, offset, fetch, hints);
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
index 6aca6ba847..2d006d38c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
@@ -66,7 +66,7 @@ public class CompoundIdentifierConverter extends SqlShuttle {
         .put(SqlAnalyzeTable.class, arrayOf(D, D, E, D))
         .put(SqlMetastoreAnalyzeTable.class, arrayOf(D, E, D, D, D))
         .put(SqlDropTableMetadata.class, arrayOf(D, D, D))
-        .put(SqlSelect.class, arrayOf(D, E, D, E, E, E, E, E, D, D, D))
+        .put(SqlSelect.class, arrayOf(D, E, D, E, E, E, E, E, E, D, D, D))
         .put(SqlCreateTable.class, arrayOf(D, D, D, E, D, D))
         .put(SqlCreateView.class, arrayOf(D, E, E, D))
         .put(DrillSqlDescribeTable.class, arrayOf(D, D, E))
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
index 19303cbf12..bcc504e2ea 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
@@ -19,9 +19,14 @@ package org.apache.drill.exec;
 
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSetComparison;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
 import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
 import org.apache.drill.exec.work.foreman.UnsupportedFunctionException;
 
@@ -1128,4 +1133,40 @@ public class TestWindowFunctions extends ClusterTest {
         .build()
         .run();
   }
+
+  @Test
+  public void testWindowFunctionWithQualifyClause() throws Exception {
+    String query = "select employee_id, full_name, birth_date " +
+        "from cp.`employee.json` " +
+        "qualify row_number() over (order by employee_id) between 5 and 7";
+
+    String[] expectedPlan = {
+      "Filter\\(condition=\\[SEARCH\\(\\$\\d, Sarg\\[\\[5..7\\]\\]\\)\\]\\)",
+      "Window\\(.*?\\[window\\(order by \\[\\d\\] rows between UNBOUNDED PRECEDING and CURRENT ROW aggs \\[ROW_NUMBER\\(\\)\\]\\)\\]\\)"
+    };
+
+    client.queryBuilder()
+      .sql(query)
+      .planMatcher()
+      .include(expectedPlan)
+      .exclude(new String[]{})
+      .match();
+
+    RowSet results = queryBuilder().sql(query).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+      .addNullable("employee_id", MinorType.BIGINT)
+      .addNullable("full_name", MinorType.VARCHAR)
+      .addNullable("birth_date", MinorType.VARCHAR)
+      .buildSchema();
+
+    RowSet expected = client.rowSetBuilder(expectedSchema)
+       .addRow(6, "Roberta Damstra", "1942-10-08")
+       .addRow(7, "Rebecca Kanagaki", "1949-03-27")
+       .addRow(8, "Kim Brunner", "1922-08-10")
+       .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+
+  }
 }
diff --git a/pom.xml b/pom.xml
index 7f54c44cfc..4b33d6d4b5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache</groupId>
     <artifactId>apache</artifactId>
-    <version>24</version>
+    <version>29</version>
     <relativePath />
   </parent>
 
@@ -54,7 +54,7 @@
     <parquet.version>1.12.2</parquet.version>
     <parquet.format.version>2.8.0</parquet.format.version>
     <calcite.groupId>org.apache.calcite</calcite.groupId>
-    <calcite.version>1.33.0</calcite.version>
+    <calcite.version>1.34.0</calcite.version>
     <avatica.version>1.23.0</avatica.version>
     <janino.version>3.1.8</janino.version>
     <sqlline.version>1.12.0</sqlline.version>
@@ -1063,10 +1063,6 @@
             <doclint>none</doclint>
           </configuration>
         </plugin>
-
-        <!--Note: apache-21.pom has the latest versions of apache-rat-plugin, maven-dependency-plugin and
-        maven-clean-plugin. Once the newer versions are needed and present in maven repo, the plugins versions can be
-        overridden here-->
       </plugins>
     </pluginManagement>
   </build>