You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by ho...@apache.org on 2019/04/06 03:14:28 UTC

[calcite] branch master updated: [CALCITE-2847] Optimize global LOOKAHEAD for SQL parsers

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

hongze pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/master by this push:
     new 7b1feec  [CALCITE-2847] Optimize global LOOKAHEAD for SQL parsers
7b1feec is described below

commit 7b1feec6640c625a6e6f2fdefd0a905c150e9192
Author: hongzezhang <ho...@tencent.com>
AuthorDate: Thu Feb 14 16:22:18 2019 +0800

    [CALCITE-2847] Optimize global LOOKAHEAD for SQL parsers
---
 .../org/apache/calcite/test/BabelParserTest.java   |  59 +++++++
 core/pom.xml                                       |   4 +-
 core/src/main/codegen/templates/Parser.jj          | 196 ++++++++++++---------
 .../apache/calcite/sql/parser/SqlParserTest.java   | 151 ++++++++--------
 .../apache/calcite/sql/test/SqlAdvisorTest.java    |  15 +-
 .../apache/calcite/test/SqlValidatorMatchTest.java |   2 +-
 .../org/apache/calcite/test/SqlValidatorTest.java  |   2 +-
 core/src/test/resources/sql/lateral.iq             |  30 ++--
 piglet/pom.xml                                     |   2 +-
 piglet/src/main/javacc/PigletParser.jj             |   6 +-
 server/pom.xml                                     |   2 +-
 server/src/main/codegen/includes/parserImpls.ftl   |   2 +-
 server/src/test/resources/sql/schema.iq            |   4 +-
 13 files changed, 278 insertions(+), 197 deletions(-)

diff --git a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
index 896a96e..8e3d312 100644
--- a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
+++ b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
@@ -19,13 +19,17 @@ package org.apache.calcite.test;
 import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
 import org.apache.calcite.sql.parser.SqlParserImplFactory;
 import org.apache.calcite.sql.parser.SqlParserTest;
+import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.parser.babel.SqlBabelParserImpl;
 
+import com.google.common.base.Throwables;
+
 import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.Arrays;
 import java.util.Locale;
+import java.util.Objects;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -140,6 +144,61 @@ public class BabelParserTest extends SqlParserTest {
         + "ORDER BY `DESC`, `DESC` DESC";
     sql(sql).ok(expected);
   }
+
+  /**
+   * This is a failure test making sure the LOOKAHEAD for WHEN clause is 2 in Babel, where
+   * in core parser this number is 1.
+   *
+   * @see SqlParserTest#testCaseExpression()
+   * @see <a href="https://issues.apache.org/jira/browse/CALCITE-2847">[CALCITE-2847]
+   * Optimize global LOOKAHEAD for SQL parsers</a>
+   */
+  @Test public void testCaseExpressionBabel() {
+    checkFails(
+        "case x when 2, 4 then 3 ^when^ then 5 else 4 end",
+        "(?s)Encountered \"when then\" at .*");
+  }
+
+  /**
+   * Babel parser's global LOOKAHEAD is larger than the core parser's, this causes different
+   * parse error message between these two parser types. Here we define a looser error checker
+   * for Babel to reuse failure testing codes from {@link SqlParserTest}.
+   *
+   * Test cases just written in this file is still checked by {@link SqlParserTest}'s checker.
+   */
+  @Override protected Tester getTester() {
+    return new TesterImpl() {
+
+      @Override protected void checkEx(String expectedMsgPattern,
+          SqlParserUtil.StringAndPos sap, Throwable thrown) {
+        if (thrownByBabelTest(thrown)) {
+          super.checkEx(expectedMsgPattern, sap, thrown);
+        } else {
+          checkExNotNull(sap, thrown);
+        }
+      }
+
+      private boolean thrownByBabelTest(Throwable ex) {
+        Throwable rootCause = Throwables.getRootCause(ex);
+        StackTraceElement[] stackTrace = rootCause.getStackTrace();
+        for (StackTraceElement stackTraceElement : stackTrace) {
+          String className = stackTraceElement.getClassName();
+          if (Objects.equals(className, BabelParserTest.class.getName())) {
+            return true;
+          }
+        }
+        return false;
+      }
+
+      private void checkExNotNull(SqlParserUtil.StringAndPos sap, Throwable thrown) {
+        if (thrown == null) {
+          throw new AssertionError("Expected query to throw exception, "
+              + "but it did not; query [" + sap.sql
+              + "]");
+        }
+      }
+    };
+  }
 }
 
 // End BabelParserTest.java
diff --git a/core/pom.xml b/core/pom.xml
index a38234c..a8fbb69 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -306,7 +306,7 @@ limitations under the License.
               <includes>
                 <include>**/Parser.jj</include>
               </includes>
-              <lookAhead>2</lookAhead>
+              <lookAhead>1</lookAhead>
               <isStatic>false</isStatic>
             </configuration>
           </execution>
@@ -322,7 +322,7 @@ limitations under the License.
               <includes>
                 <include>**/Parser.jj</include>
               </includes>
-              <lookAhead>2</lookAhead>
+              <lookAhead>1</lookAhead>
               <isStatic>false</isStatic>
             </configuration>
           </execution>
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 9960fd1..9e0a85d 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -610,6 +610,7 @@ SqlNode OrderedQueryOrExpr(ExprContext exprContext) :
         <LIMIT>
         (
             // MySQL-style syntax. "LIMIT start, count"
+            LOOKAHEAD(2)
             start = UnsignedNumericLiteralOrParam()
             <COMMA> count = UnsignedNumericLiteralOrParam() {
                 if (!this.conformance.isLimitStartCountAllowed()) {
@@ -877,7 +878,7 @@ void Arg0(List list, ExprContext exprContext) :
 }
 {
     [
-        name = SimpleIdentifier() <NAMED_ARGUMENT_ASSIGNMENT>
+        LOOKAHEAD(2) name = SimpleIdentifier() <NAMED_ARGUMENT_ASSIGNMENT>
     ]
     (
         e = Default()
@@ -902,7 +903,7 @@ void Arg(List list, ExprContext exprContext) :
 }
 {
     [
-        name = SimpleIdentifier() <NAMED_ARGUMENT_ASSIGNMENT>
+        LOOKAHEAD(2) name = SimpleIdentifier() <NAMED_ARGUMENT_ASSIGNMENT>
     ]
     (
         e = Default()
@@ -952,7 +953,7 @@ SqlNode SqlStmt() :
     (
 <#-- Add methods to parse additional statements here -->
 <#list parser.statementParserMethods as method>
-        stmt = ${method}
+        LOOKAHEAD(2) stmt = ${method}
     |
 </#list>
         stmt = SqlSetOption(Span.of(), null)
@@ -1090,6 +1091,7 @@ SqlNode SqlExplain() :
     [ detailLevel = ExplainDetailLevel() ]
     depth = ExplainDepth()
     (
+        LOOKAHEAD(2)
         <AS> <XML> { format = SqlExplainFormat.XML; }
     |
         <AS> <JSON> { format = SqlExplainFormat.JSON; }
@@ -1197,7 +1199,7 @@ SqlNode SqlDescribe() :
 {
     <DESCRIBE> { s = span(); }
     (
-        (<DATABASE> | <CATALOG> | <SCHEMA>)
+        LOOKAHEAD(2) (<DATABASE> | <CATALOG> | <SCHEMA>)
         id = CompoundIdentifier() {
             // DESCRIBE DATABASE and DESCRIBE CATALOG currently do the same as
             // DESCRIBE SCHEMA but should be different. See
@@ -1220,7 +1222,7 @@ SqlNode SqlDescribe() :
                 table, column);
         }
     |
-        (<STATEMENT>)?
+        (LOOKAHEAD(1) <STATEMENT>)?
         stmt = SqlQueryOrDml() {
             // DESCRIBE STATEMENT currently does the same as EXPLAIN. See
             //   [CALCITE-1221] Implement DESCRIBE DATABASE, CATALOG, STATEMENT
@@ -1699,94 +1701,105 @@ SqlNode FromClause() :
 {
     e = TableRef()
     (
-        // Decide whether to read a JOIN clause or a comma, or to quit having
-        // seen a single entry FROM clause like 'FROM emps'. See comments
-        // elsewhere regarding <COMMA> lookahead.
         LOOKAHEAD(2)
-        natural = Natural()
-        joinType = JoinType()
-        e2 = TableRef()
         (
-            <ON> {
-                joinConditionType = JoinConditionType.ON.symbol(getPos());
-            }
-            condition = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+            // Decide whether to read a JOIN clause or a comma, or to quit having
+            // seen a single entry FROM clause like 'FROM emps'. See comments
+            // elsewhere regarding <COMMA> lookahead.
+            //
+            // And LOOKAHEAD(3) is needed here rather than a LOOKAHEAD(2). Because currently 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
+            LOOKAHEAD(3)
+            natural = Natural()
+            joinType = JoinType()
+            e2 = TableRef()
+            (
+                <ON> {
+                    joinConditionType = JoinConditionType.ON.symbol(getPos());
+                }
+                condition = Expression(ExprContext.ACCEPT_SUB_QUERY) {
+                    e = new SqlJoin(joinType.getParserPosition(),
+                        e,
+                        natural,
+                        joinType,
+                        e2,
+                        joinConditionType,
+                        condition);
+                }
+            |
+                <USING> {
+                    joinConditionType = JoinConditionType.USING.symbol(getPos());
+                }
+                list = ParenthesizedSimpleIdentifierList() {
+                    e = new SqlJoin(joinType.getParserPosition(),
+                        e,
+                        natural,
+                        joinType,
+                        e2,
+                        joinConditionType,
+                        new SqlNodeList(list.getList(), Span.of(joinConditionType).end(this)));
+                }
+            |
+                {
+                    e = new SqlJoin(joinType.getParserPosition(),
+                        e,
+                        natural,
+                        joinType,
+                        e2,
+                        JoinConditionType.NONE.symbol(joinType.getParserPosition()),
+                        null);
+                }
+            )
+        |
+            // NOTE jvs 6-Feb-2004:  See comments at top of file for why
+            // hint is necessary here.  I had to use this special semantic
+            // lookahead form to get JavaCC to shut up, which makes
+            // me even more uneasy.
+            //LOOKAHEAD({true})
+            <COMMA> { joinType = JoinType.COMMA.symbol(getPos()); }
+            e2 = TableRef() {
                 e = new SqlJoin(joinType.getParserPosition(),
                     e,
-                    natural,
+                    SqlLiteral.createBoolean(false, joinType.getParserPosition()),
                     joinType,
                     e2,
-                    joinConditionType,
-                    condition);
+                    JoinConditionType.NONE.symbol(SqlParserPos.ZERO),
+                    null);
             }
         |
-            <USING> {
-                joinConditionType = JoinConditionType.USING.symbol(getPos());
-            }
-            list = ParenthesizedSimpleIdentifierList() {
+            <CROSS> { joinType = JoinType.CROSS.symbol(getPos()); } <APPLY>
+            e2 = TableRef2(true) {
+                if (!this.conformance.isApplyAllowed()) {
+                    throw new ParseException(RESOURCE.applyNotAllowed().str());
+                }
                 e = new SqlJoin(joinType.getParserPosition(),
                     e,
-                    natural,
+                    SqlLiteral.createBoolean(false, joinType.getParserPosition()),
                     joinType,
                     e2,
-                    joinConditionType,
-                    new SqlNodeList(list.getList(), Span.of(joinConditionType).end(this)));
+                    JoinConditionType.NONE.symbol(SqlParserPos.ZERO),
+                    null);
             }
         |
-            {
+            <OUTER> { joinType = JoinType.LEFT.symbol(getPos()); } <APPLY>
+            e2 = TableRef2(true) {
+                if (!this.conformance.isApplyAllowed()) {
+                    throw new ParseException(RESOURCE.applyNotAllowed().str());
+                }
                 e = new SqlJoin(joinType.getParserPosition(),
                     e,
-                    natural,
+                    SqlLiteral.createBoolean(false, joinType.getParserPosition()),
                     joinType,
                     e2,
-                    JoinConditionType.NONE.symbol(joinType.getParserPosition()),
-                    null);
+                    JoinConditionType.ON.symbol(SqlParserPos.ZERO),
+                    SqlLiteral.createBoolean(true, joinType.getParserPosition()));
             }
         )
-    |
-        // NOTE jvs 6-Feb-2004:  See comments at top of file for why
-        // hint is necessary here.  I had to use this special semantic
-        // lookahead form to get JavaCC to shut up, which makes
-        // me even more uneasy.
-        //LOOKAHEAD({true})
-        <COMMA> { joinType = JoinType.COMMA.symbol(getPos()); }
-        e2 = TableRef() {
-            e = new SqlJoin(joinType.getParserPosition(),
-                e,
-                SqlLiteral.createBoolean(false, joinType.getParserPosition()),
-                joinType,
-                e2,
-                JoinConditionType.NONE.symbol(SqlParserPos.ZERO),
-                null);
-        }
-    |
-        <CROSS> { joinType = JoinType.CROSS.symbol(getPos()); } <APPLY>
-        e2 = TableRef2(true) {
-            if (!this.conformance.isApplyAllowed()) {
-                throw new ParseException(RESOURCE.applyNotAllowed().str());
-            }
-            e = 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 new ParseException(RESOURCE.applyNotAllowed().str());
-            }
-            e = new SqlJoin(joinType.getParserPosition(),
-                e,
-                SqlLiteral.createBoolean(false, joinType.getParserPosition()),
-                joinType,
-                e2,
-                JoinConditionType.ON.symbol(SqlParserPos.ZERO),
-                SqlLiteral.createBoolean(true, joinType.getParserPosition()));
-        }
     )*
     {
         return e;
@@ -1852,6 +1865,7 @@ SqlNode TableRef2(boolean lateral) :
             tableRef = MatchRecognize(tableRef)
         ]
     |
+        LOOKAHEAD(2)
         [ <LATERAL> { lateral = true; } ]
         tableRef = ParenthesizedExpression(ExprContext.ACCEPT_QUERY)
         over = TableOverOpt()
@@ -1880,7 +1894,7 @@ SqlNode TableRef2(boolean lateral) :
             tableRef = unnestOp.createCall(s.end(this), args.toArray());
         }
     |
-        [ <LATERAL> { lateral = true; } ]
+        [ LOOKAHEAD(1) <LATERAL> { lateral = true; } ]
         <TABLE> { s = span(); } <LPAREN>
         tableRef = TableFunctionCall(s.pos())
         <RPAREN>
@@ -2227,6 +2241,7 @@ List<SqlNode> GroupingElementList() :
 {
     e = GroupingElement() { list.add(e); }
     (
+        LOOKAHEAD(2)
         <COMMA>
         e = GroupingElement() { list.add(e); }
     )*
@@ -2241,6 +2256,7 @@ SqlNode GroupingElement() :
     final Span s;
 }
 {
+    LOOKAHEAD(2)
     <GROUPING> { s = span(); }
     <SETS> <LPAREN> list = GroupingElementList() <RPAREN> {
         return SqlStdOperatorTable.GROUPING_SETS.createCall(s.end(this), list);
@@ -2412,10 +2428,12 @@ SqlNode WindowRange() :
     final Span s;
 }
 {
+    LOOKAHEAD(2)
     <CURRENT> { s = span(); } <ROW> {
         return SqlWindow.createCurrentRow(s.end(this));
     }
 |
+    LOOKAHEAD(2)
     <UNBOUNDED> { s = span(); }
     (
         <PRECEDING> {
@@ -2487,6 +2505,7 @@ SqlNode OrderItem() :
         }
     )?
     (
+        LOOKAHEAD(2)
         <NULLS> <FIRST> {
             e = SqlStdOperatorTable.NULLS_FIRST.createCall(getPos(), e);
         }
@@ -2572,12 +2591,13 @@ SqlMatchRecognize MatchRecognize(SqlNode tableRef) :
                         .symbol(s1.end(this));
                 }
             |
+                LOOKAHEAD(2)
                 <FIRST> var = SimpleIdentifier() {
                     after = SqlMatchRecognize.SKIP_TO_FIRST.createCall(
                         s1.end(var), var);
                 }
             |
-                [ <LAST> ] var = SimpleIdentifier() {
+                [ LOOKAHEAD(1) <LAST> ] var = SimpleIdentifier() {
                     after = SqlMatchRecognize.SKIP_TO_LAST.createCall(
                         s1.end(var), var);
                 }
@@ -2704,6 +2724,7 @@ SqlNode PatternFactor() :
 {
     e = PatternPrimary()
     [
+        LOOKAHEAD(1)
         (
             <STAR> {
                 startNum = SqlLiteral.createExactNumeric("0", SqlParserPos.ZERO);
@@ -3021,6 +3042,7 @@ void Expression2b(ExprContext exprContext, List<Object> list) :
 }
 {
     (
+        LOOKAHEAD(1)
         op = PrefixRowOperator() {
             checkNonQueryExpression(exprContext);
             list.add(new SqlParserUtil.ToTreeListItem(op, getPos()));
@@ -3030,7 +3052,7 @@ void Expression2b(ExprContext exprContext, List<Object> list) :
         list.add(e);
     }
     (
-        <DOT>
+        LOOKAHEAD(2) <DOT>
         p = SimpleIdentifier() {
             list.add(
                 new SqlParserUtil.ToTreeListItem(
@@ -3140,7 +3162,7 @@ List<Object> Expression2(ExprContext exprContext) :
                     list3.clear();
                 }
             |
-                {
+                LOOKAHEAD(2) {
                     checkNonQueryExpression(exprContext);
                     s.clear().add(this);
                 }
@@ -3186,7 +3208,7 @@ List<Object> Expression2(ExprContext exprContext) :
                     list.add(e);
                 }
                 (
-                    <DOT>
+                    LOOKAHEAD(2) <DOT>
                     p = SimpleIdentifier() {
                         list.add(
                             new SqlParserUtil.ToTreeListItem(
@@ -3349,6 +3371,7 @@ SqlOperator periodOperator() :
 {
      <OVERLAPS> { return SqlStdOperatorTable.OVERLAPS; }
 |
+     LOOKAHEAD(2)
      <IMMEDIATELY> <PRECEDES> { return SqlStdOperatorTable.IMMEDIATELY_PRECEDES; }
 |
      <PRECEDES> { return SqlStdOperatorTable.PRECEDES; }
@@ -3404,6 +3427,7 @@ SqlNode AtomicRowExpression() :
     |
         e = DynamicParam()
     |
+        LOOKAHEAD(2)
         e = BuiltinFunctionCall()
     |
         e = JdbcFunctionCall()
@@ -3412,6 +3436,7 @@ SqlNode AtomicRowExpression() :
     |
         e = ArrayConstructor()
     |
+        LOOKAHEAD(3)
         e = MapConstructor()
     |
         e = PeriodConstructor()
@@ -3897,7 +3922,7 @@ SqlNode MultisetConstructor() :
 {
     <MULTISET> { s = span(); }
     (
-        LOOKAHEAD(1)
+        LOOKAHEAD(2)
         <LPAREN>
         // by sub query "MULTISET(SELECT * FROM T)"
         e = LeafQueryOrExpr(ExprContext.ACCEPT_QUERY)
@@ -4314,10 +4339,12 @@ SqlIdentifier CompoundIdentifier() :
 {
     IdentifierSegment(nameList, posList)
     (
+        LOOKAHEAD(2)
         <DOT>
         IdentifierSegment(nameList, posList)
     )*
     (
+        LOOKAHEAD(2)
         <DOT>
         <STAR> {
             star = true;
@@ -4486,6 +4513,7 @@ SqlIdentifier TypeName() :
 }
 {
     (
+        LOOKAHEAD(2)
         sqlTypeName = SqlTypeName(s) {
             typeName = new SqlIdentifier(sqlTypeName.name(), s.end(this));
         }
@@ -5167,6 +5195,7 @@ List<SqlNode> JsonQueryEmptyOrErrorBehavior() :
             list.add(SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.NULL, getPos()));
         }
     |
+        LOOKAHEAD(2)
         <EMPTY> <ARRAY> {
             list.add(SqlLiteral.createSymbol(SqlJsonQueryEmptyOrErrorBehavior.EMPTY_ARRAY, getPos()));
         }
@@ -5261,6 +5290,7 @@ List<SqlNode> JsonNameAndValue() :
 }
 {
     [
+        LOOKAHEAD(1)
         <KEY> { kvMode = true; }
     ]
     e = JsonName() {
@@ -5307,6 +5337,7 @@ SqlCall JsonObjectFunctionCall() :
 {
     <JSON_OBJECT> { span = span(); }
     <LPAREN> [
+        LOOKAHEAD(2)
         list = JsonNameAndValue() {
             nvArgs.addAll(list);
         }
@@ -5415,6 +5446,7 @@ SqlCall JsonArrayFunctionCall() :
 {
     <JSON_ARRAY> { span = span(); }
     <LPAREN> [
+        LOOKAHEAD(2)
         e = JsonValueExpression(false) {
             elements.add(e);
         }
@@ -5568,8 +5600,10 @@ SqlCall MatchRecognizeFunctionCall() :
             func = SqlStdOperatorTable.MATCH_NUMBER.createCall(s.end(this));
         }
     |
+        LOOKAHEAD(3)
         func = MatchRecognizeNavigationLogical()
     |
+        LOOKAHEAD(2)
         func = MatchRecognizeNavigationPhysical()
     |
         func = MatchRecognizeCallWithModifier()
@@ -5976,6 +6010,7 @@ SqlNode JdbcFunctionCall() :
             args = new SqlNodeList(call.getOperandList(), getPos());
         }
     |
+        LOOKAHEAD(3)
         call = TimestampDiffFunctionCall() {
             name = call.getOperator().getName();
             args = new SqlNodeList(call.getOperandList(), getPos());
@@ -5996,6 +6031,7 @@ SqlNode JdbcFunctionCall() :
             <INSERT> { name = unquotedIdentifier(); }
         |
             // TRUNCATE is a reserved word, but we need to handle {fn truncate}
+            LOOKAHEAD(1)
             <TRUNCATE> { name = unquotedIdentifier(); }
         |
             // For cases like {fn power(1,2)} and {fn lower('a')}
@@ -6141,7 +6177,7 @@ SqlBinaryOperator BinaryRowOperator() :
 |   <EQUALS> { return SqlStdOperatorTable.PERIOD_EQUALS; }
 |   <PRECEDES> { return SqlStdOperatorTable.PRECEDES; }
 |   <SUCCEEDS> { return SqlStdOperatorTable.SUCCEEDS; }
-|   <IMMEDIATELY> <PRECEDES> { return SqlStdOperatorTable.IMMEDIATELY_PRECEDES; }
+|   LOOKAHEAD(2) <IMMEDIATELY> <PRECEDES> { return SqlStdOperatorTable.IMMEDIATELY_PRECEDES; }
 |   <IMMEDIATELY> <SUCCEEDS> { return SqlStdOperatorTable.IMMEDIATELY_SUCCEEDS; }
 |   op = BinaryMultisetOperator() { return op; }
 }
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 62a3798..bfb0291 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -721,8 +721,8 @@ public class SqlParserTest {
         "(?s).*Encountered \".1\" at line 1, column 13.\n"
             + "Was expecting one of:\n"
             + "    <EOF> \n"
-            + "    \"ORDER\" ...\n"
-            + "    \"LIMIT\" ...\n"
+            + "    \"AND\" \\.\\.\\.\n"
+            + "    \"AS\" \\.\\.\\.\n"
             + ".*");
   }
 
@@ -1402,10 +1402,9 @@ public class SqlParserTest {
         "VALUES (ROW((`A` SIMILAR TO (`B` LIKE (`C` SIMILAR TO `D` ESCAPE `E`) ESCAPE `F`))))");
 
     if (isReserved("ESCAPE")) {
-      // FIXME should fail at "escape"
       checkFails(
-          "select * from t ^where^ escape 'e'",
-          "(?s).*Encountered \"where escape\" at .*");
+          "select * from t where ^escape^ 'e'",
+          "(?s).*Encountered \"escape\" at .*");
     }
 
     // LIKE with +
@@ -1420,7 +1419,6 @@ public class SqlParserTest {
 
     // ESCAPE with no expression
     if (isReserved("ESCAPE")) {
-      // FIXME should fail at "escape"
       checkFails(
           "values a ^like^ escape d",
           "(?s).*Encountered \"like escape\" at .*");
@@ -1596,8 +1594,8 @@ public class SqlParserTest {
         .fails("(?s)Encountered \"\\+ DEFAULT\" at .*");
     sql("select power(0, DEFAULT ^+^ empno) from emp")
         .fails("(?s)Encountered \"\\+\" at .*");
-    sql("select * from emp join dept ^on^ DEFAULT")
-        .fails("(?s)Encountered \"on DEFAULT\" at .*");
+    sql("select * from emp join dept on ^DEFAULT^")
+        .fails("(?s)Encountered \"DEFAULT\" at .*");
     sql("select * from emp where empno ^>^ DEFAULT or deptno < 10")
         .fails("(?s)Encountered \"> DEFAULT\" at .*");
     sql("select * from emp order by ^DEFAULT^ desc")
@@ -1608,8 +1606,8 @@ public class SqlParserTest {
         .ok(expected);
     sql("insert into dept (name, deptno) values ('a', 1 ^+^ DEFAULT)")
         .fails("(?s)Encountered \"\\+ DEFAULT\" at .*");
-    sql("insert into dept (name, deptno) select 'a'^,^ DEFAULT from (values 0)")
-        .fails("(?s)Encountered \", DEFAULT\" at .*");
+    sql("insert into dept (name, deptno) select 'a', ^DEFAULT^ from (values 0)")
+        .fails("(?s)Encountered \"DEFAULT\" at .*");
   }
 
   @Test public void testAggregateFilter() {
@@ -2136,11 +2134,11 @@ public class SqlParserTest {
    * standard.) */
   @Test public void testMinusIsReserved() {
     sql("select ^minus^ from t")
-        .fails("(?s).*Encountered \"minus from\" at .*");
+        .fails("(?s).*Encountered \"minus\" at .*");
     sql("select ^minus^ select")
-        .fails("(?s).*Encountered \"minus select\" at .*");
-    sql("select * from t ^as^ minus where x < y")
-        .fails("(?s).*Encountered \"as minus\" at .*");
+        .fails("(?s).*Encountered \"minus\" at .*");
+    sql("select * from t as ^minus^ where x < y")
+        .fails("(?s).*Encountered \"minus\" at .*");
   }
 
   @Test public void testIntersect() {
@@ -3162,22 +3160,24 @@ public class SqlParserTest {
 
   @Test public void testFromValuesWithoutParens() {
     checkFails(
-        "select 1 ^from^ values('x')",
-        "(?s)Encountered \"from values\" at line 1, column 10\\.\n"
+        "select 1 from ^values^('x')",
+        "(?s)Encountered \"values\" at line 1, column 15\\.\n"
             + "Was expecting one of:\n"
-            + "    <EOF> \n"
-            + "    \"ORDER\" \\.\\.\\.\n"
-            + "    \"LIMIT\" \\.\\.\\.\n"
-            + ".*"
-            + "    \"FROM\" <IDENTIFIER> \\.\\.\\.\n"
-            + "    \"FROM\" <QUOTED_IDENTIFIER> \\.\\.\\.\n"
-            + ".*");
+            + "    \"LATERAL\" \\.\\.\\.\n"
+            + "    \"TABLE\" \\.\\.\\.\n"
+            + "    \"UNNEST\" \\.\\.\\.\n"
+            + "    <IDENTIFIER> \\.\\.\\.\n"
+            + "    <QUOTED_IDENTIFIER> \\.\\.\\.\n"
+            + "    <BACK_QUOTED_IDENTIFIER> \\.\\.\\.\n"
+            + "    <BRACKET_QUOTED_IDENTIFIER> \\.\\.\\.\n"
+            + "    <UNICODE_QUOTED_IDENTIFIER> \\.\\.\\.\n"
+            + "    \"\\(\" \\.\\.\\.\n.*");
   }
 
   @Test public void testEmptyValues() {
     checkFails(
-        "select * from (values^(^))",
-        "(?s).*Encountered \"\\( \\)\" at .*");
+        "select * from (values(^)^)",
+        "(?s).*Encountered \"\\)\" at .*");
   }
 
   /** Test case for
@@ -3221,10 +3221,9 @@ public class SqlParserTest {
   @Test public void testExplicitTable() {
     check("table emp", "(TABLE `EMP`)");
 
-    // FIXME should fail at "123"
     checkFails(
-        "^table^ 123",
-        "(?s)Encountered \"table 123\" at line 1, column 1\\.\n.*");
+        "table ^123^",
+        "(?s)Encountered \"123\" at line 1, column 7\\.\n.*");
   }
 
   @Test public void testExplicitTableOrdered() {
@@ -3242,14 +3241,13 @@ public class SqlParserTest {
   }
 
   @Test public void testSelectFromBareExplicitTableFails() {
-    // FIXME should fail at "emp"
     checkFails(
-        "select * from ^table^ emp",
-        "(?s).*Encountered \"table emp\" at .*");
+        "select * from table ^emp^",
+        "(?s).*Encountered \"emp\" at .*");
 
     checkFails(
-        "select * from (^table^ (select empno from emp))",
-        "(?s)Encountered \"table \\(\".*");
+        "select * from (table ^(^select empno from emp))",
+        "(?s)Encountered \"\\(\".*");
   }
 
   @Test public void testCollectionTable() {
@@ -3278,8 +3276,8 @@ public class SqlParserTest {
 
   @Test public void testLateral() {
     // Bad: LATERAL table
-    sql("select * from ^lateral^ emp")
-        .fails("(?s)Encountered \"lateral emp\" at .*");
+    sql("select * from lateral ^emp^")
+        .fails("(?s)Encountered \"emp\" at .*");
     sql("select * from lateral table ^emp^ as e")
         .fails("(?s)Encountered \"emp\" at .*");
 
@@ -3296,8 +3294,8 @@ public class SqlParserTest {
     sql("select * from lateral table(ramp(1)) as t(x)")
         .ok(expected + " AS `T` (`X`)");
     // Bad: Parentheses make it look like a sub-query
-    sql("select * from lateral (^table^(ramp(1)))")
-        .fails("(?s)Encountered \"table \\(\" at .*");
+    sql("select * from lateral (table^(^ramp(1)))")
+        .fails("(?s)Encountered \"\\(\" at .*");
 
     // Good: LATERAL (subQuery)
     final String expected2 = "SELECT *\n"
@@ -3465,10 +3463,10 @@ public class SqlParserTest {
         + "VALUES (ROW(1, 'a'))";
     check("describe insert into emps values (1, 'a')", expected3);
     // only allow query or DML, not explain, inside describe
-    checkFails("^describe^ explain plan for select * from emps",
-        "(?s).*Encountered \"describe explain\" at .*");
-    checkFails("describe ^statement^ explain plan for select * from emps",
-        "(?s).*Encountered \"statement explain\" at .*");
+    checkFails("describe ^explain^ plan for select * from emps",
+        "(?s).*Encountered \"explain\" at .*");
+    checkFails("describe statement ^explain^ plan for select * from emps",
+        "(?s).*Encountered \"explain\" at .*");
   }
 
   @Test public void testSelectIsNotDdl() {
@@ -3515,8 +3513,8 @@ public class SqlParserTest {
   @Test public void testInsertValuesRawDefault() {
     final String expected = "INSERT INTO `EMPS`\n"
         + "VALUES (ROW(DEFAULT))";
-    sql("insert into emps ^values^ default")
-        .fails("(?s).*Encountered \"values default\" at .*");
+    sql("insert into emps values ^default^")
+        .fails("(?s).*Encountered \"default\" at .*");
     sql("insert into emps values (default)")
         .ok(expected)
         .node(not(isDdl()));
@@ -3858,8 +3856,8 @@ public class SqlParserTest {
         "(CASE WHEN (`X` IN (2, 4)) THEN 3 ELSE 4 END)");
     // comma-list must not be empty
     checkFails(
-        "case x when 2, 4 then 3 ^when^ then 5 else 4 end",
-        "(?s)Encountered \"when then\" at .*");
+        "case x when 2, 4 then 3 when ^then^ 5 else 4 end",
+        "(?s)Encountered \"then\" at .*");
     // commas not allowed in boolean case
     checkFails(
         "case when b1, b2 ^when^ 2, 4 then 3 else 4 end",
@@ -4217,10 +4215,9 @@ public class SqlParserTest {
         "(?s).*Encountered \"unbounded\".*");
 
     // WINDOW keyword is not permissible.
-    // FIXME should fail at "window"
     checkFails(
-        "select sum(x) ^over^ window (order by x) from bids",
-        "(?s).*Encountered \"over window\".*");
+        "select sum(x) over ^window^ (order by x) from bids",
+        "(?s).*Encountered \"window\".*");
 
     // ORDER BY must be before Frame spec
     checkFails(
@@ -4339,8 +4336,8 @@ public class SqlParserTest {
 
     // must have at least one column
     checkFails(
-        "select x from (values (1, 2), (3, 4)) as t1 ^(^)",
-        "(?s).*Encountered \"\\( \\)\" at .*");
+        "select x from (values (1, 2), (3, 4)) as t1 (^)^",
+        "(?s).*Encountered \"\\)\" at .*");
 
     // cannot have expressions
     checkFails(
@@ -6680,12 +6677,12 @@ public class SqlParserTest {
         "interval '1^'^",
         "Encountered \"<EOF>\" at line 1, column 12\\.\n"
             + "Was expecting one of:\n"
-            + "    \"YEAR\" \\.\\.\\.\n"
-            + "    \"MONTH\" \\.\\.\\.\n"
             + "    \"DAY\" \\.\\.\\.\n"
             + "    \"HOUR\" \\.\\.\\.\n"
             + "    \"MINUTE\" \\.\\.\\.\n"
+            + "    \"MONTH\" \\.\\.\\.\n"
             + "    \"SECOND\" \\.\\.\\.\n"
+            + "    \"YEAR\" \\.\\.\\.\n"
             + "    ");
 
     // illegal qualifiers, no precision in either field
@@ -6694,8 +6691,8 @@ public class SqlParserTest {
         "(?s)Encountered \"to year\" at line 1, column 19.\n"
             + "Was expecting one of:\n"
             + "    <EOF> \n"
-            + "    \"\\.\" \\.\\.\\.\n"
-            + "    \"NOT\" \\.\\.\\..*");
+            + "    \"AND\" \\.\\.\\.\n"
+            + "    \"BETWEEN\" \\.\\.\\..*");
     checkExpFails("interval '1-2' year ^to^ day", ANY);
     checkExpFails("interval '1-2' year ^to^ hour", ANY);
     checkExpFails("interval '1-2' year ^to^ minute", ANY);
@@ -6855,24 +6852,23 @@ public class SqlParserTest {
     checkExpFails("interval '1-2' second(3) ^to^ second(2,6)", ANY);
 
     // precision of -1 (< minimum allowed)
-    // FIXME should fail at "-" or "-1"
-    checkExpFails("INTERVAL '0' YEAR^(^-1)", ANY);
-    checkExpFails("INTERVAL '0-0' YEAR^(^-1) TO MONTH", ANY);
-    checkExpFails("INTERVAL '0' MONTH^(^-1)", ANY);
-    checkExpFails("INTERVAL '0' DAY^(^-1)", ANY);
-    checkExpFails("INTERVAL '0 0' DAY^(^-1) TO HOUR", ANY);
-    checkExpFails("INTERVAL '0 0' DAY^(^-1) TO MINUTE", ANY);
-    checkExpFails("INTERVAL '0 0:0:0' DAY^(^-1) TO SECOND", ANY);
-    checkExpFails("INTERVAL '0 0:0:0' DAY TO SECOND^(^-1)", ANY);
-    checkExpFails("INTERVAL '0' HOUR^(^-1)", ANY);
-    checkExpFails("INTERVAL '0:0' HOUR^(^-1) TO MINUTE", ANY);
-    checkExpFails("INTERVAL '0:0:0' HOUR^(^-1) TO SECOND", ANY);
-    checkExpFails("INTERVAL '0:0:0' HOUR TO SECOND^(^-1)", ANY);
-    checkExpFails("INTERVAL '0' MINUTE^(^-1)", ANY);
-    checkExpFails("INTERVAL '0:0' MINUTE^(^-1) TO SECOND", ANY);
-    checkExpFails("INTERVAL '0:0' MINUTE TO SECOND^(^-1)", ANY);
-    checkExpFails("INTERVAL '0' SECOND^(^-1)", ANY);
-    checkExpFails("INTERVAL '0' SECOND(1^,^ -1)", ANY);
+    checkExpFails("INTERVAL '0' YEAR(^-^1)", ANY);
+    checkExpFails("INTERVAL '0-0' YEAR(^-^1) TO MONTH", ANY);
+    checkExpFails("INTERVAL '0' MONTH(^-^1)", ANY);
+    checkExpFails("INTERVAL '0' DAY(^-^1)", ANY);
+    checkExpFails("INTERVAL '0 0' DAY(^-^1) TO HOUR", ANY);
+    checkExpFails("INTERVAL '0 0' DAY(^-^1) TO MINUTE", ANY);
+    checkExpFails("INTERVAL '0 0:0:0' DAY(^-^1) TO SECOND", ANY);
+    checkExpFails("INTERVAL '0 0:0:0' DAY TO SECOND(^-^1)", ANY);
+    checkExpFails("INTERVAL '0' HOUR(^-^1)", ANY);
+    checkExpFails("INTERVAL '0:0' HOUR(^-^1) TO MINUTE", ANY);
+    checkExpFails("INTERVAL '0:0:0' HOUR(^-^1) TO SECOND", ANY);
+    checkExpFails("INTERVAL '0:0:0' HOUR TO SECOND(^-^1)", ANY);
+    checkExpFails("INTERVAL '0' MINUTE(^-^1)", ANY);
+    checkExpFails("INTERVAL '0:0' MINUTE(^-^1) TO SECOND", ANY);
+    checkExpFails("INTERVAL '0:0' MINUTE TO SECOND(^-^1)", ANY);
+    checkExpFails("INTERVAL '0' SECOND(^-^1)", ANY);
+    checkExpFails("INTERVAL '0' SECOND(1, ^-^1)", ANY);
 
     // These may actually be legal per SQL2003, as the first field is
     // "more significant" than the last, but we do not support them
@@ -7141,8 +7137,8 @@ public class SqlParserTest {
     sql(sql).ok(expected);
 
     // LATERAL UNNEST is not valid
-    sql("select * from dept, ^lateral^ unnest(dept.employees)")
-        .fails("(?s)Encountered \"lateral unnest\" at .*");
+    sql("select * from dept, lateral ^unnest^(dept.employees)")
+        .fails("(?s)Encountered \"unnest\" at .*");
   }
 
   @Test public void testUnnestWithOrdinality() {
@@ -8646,7 +8642,7 @@ public class SqlParserTest {
         thrown = ex;
       }
 
-      SqlValidatorTestCase.checkEx(thrown, expectedMsgPattern, sap);
+      checkEx(expectedMsgPattern, sap, thrown);
     }
 
     public void checkNode(String sql, Matcher<SqlNode> matcher) {
@@ -8675,6 +8671,11 @@ public class SqlParserTest {
         thrown = ex;
       }
 
+      checkEx(expectedMsgPattern, sap, thrown);
+    }
+
+    protected void checkEx(String expectedMsgPattern, SqlParserUtil.StringAndPos sap,
+        Throwable thrown) {
       SqlValidatorTestCase.checkEx(thrown, expectedMsgPattern, sap);
     }
   }
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index 9204b7b..aa3d706 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -830,16 +830,13 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
         "select ^dummy, b.dummy from sales.emp a join sales.dept b "
             + "on a.deptno=b.deptno where empno=1";
     assertHint(
-        sql, getSelectKeywords(), EXPR_KEYWORDS, AB_TABLES, SETOPS,
-        FETCH_OFFSET);
+        sql, getSelectKeywords(), EXPR_KEYWORDS, AB_TABLES);
 
     sql = "select ^ from (values (1))";
     assertComplete(
         sql,
         getSelectKeywords(),
         EXPR_KEYWORDS,
-        SETOPS,
-        FETCH_OFFSET,
         Arrays.asList("TABLE(EXPR$0)", "COLUMN(EXPR$0)"));
 
     sql = "select ^ from (values (1)) as t(c)";
@@ -847,14 +844,11 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
         sql,
         getSelectKeywords(),
         EXPR_KEYWORDS,
-        SETOPS,
-        FETCH_OFFSET,
         Arrays.asList("TABLE(T)", "COLUMN(C)"));
 
     sql = "select ^, b.dummy from sales.emp a join sales.dept b ";
     assertComplete(
-        sql, getSelectKeywords(), EXPR_KEYWORDS, SETOPS, AB_TABLES,
-        FETCH_OFFSET);
+        sql, getSelectKeywords(), EXPR_KEYWORDS, AB_TABLES);
 
     sql =
         "select dummy, ^b.dummy from sales.emp a join sales.dept b "
@@ -876,8 +870,6 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
         getSelectKeywords(),
         EXPR_KEYWORDS,
         EMP_COLUMNS,
-        SETOPS,
-        FETCH_OFFSET,
         Arrays.asList("TABLE(EMP)"));
 
     sql = "select emp.^ from sales.emp";
@@ -932,8 +924,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
     sql =
         "select ^t.dummy from (select 1 as x, 2 as y from sales.emp) as t where t.dummy=1";
     assertHint(
-        sql, EXPR_KEYWORDS, getSelectKeywords(), xyColumns, tTable, SETOPS,
-        FETCH_OFFSET);
+        sql, EXPR_KEYWORDS, getSelectKeywords(), xyColumns, tTable);
 
     sql = "select t.^ from (select 1 as x, 2 as y from sales.emp) as t";
     assertComplete(sql, xyColumns, STAR_KEYWORD);
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorMatchTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorMatchTest.java
index 5ab86a2..0f4f47a 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorMatchTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorMatchTest.java
@@ -140,7 +140,7 @@ public class SqlValidatorMatchTest extends SqlValidatorTestCase {
         + "      up as up.sal > prev(up.sal)\n"
         + "  ) mr";
     sql(sql)
-        .fails("(?s).*Encountered \"to null\" at .*");
+        .fails("(?s).*Encountered \"null\" at .*");
   }
 
   @Test public void testMatchRecognizeSkipTo2() throws Exception {
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 89e0f5c..b12bbca 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -7818,7 +7818,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("select count(ename, 1, deptno) from emp");
     check("select count(distinct ename, 1, deptno) from emp");
     checkFails("select count(deptno, *) from emp",
-        "(?s).*Encountered \", \\*\" at .*");
+        "(?s).*Encountered \"\\*\" at .*");
     checkFails(
         "select count(*, deptno) from emp",
         "(?s).*Encountered \",\" at .*");
diff --git a/core/src/test/resources/sql/lateral.iq b/core/src/test/resources/sql/lateral.iq
index 046b509..b38b0d7 100644
--- a/core/src/test/resources/sql/lateral.iq
+++ b/core/src/test/resources/sql/lateral.iq
@@ -20,16 +20,15 @@
 
 # Bad: LATERAL tableName
 select * from "scott".emp join lateral "scott".dept using (deptno);
-parse failed: Encountered "lateral \"scott\"" at line 1, column 32.
+parse failed: Encountered "join lateral \"scott\"" at line 1, column 27.
 Was expecting one of:
-    <IDENTIFIER> ...
-    <QUOTED_IDENTIFIER> ...
-    <BACK_QUOTED_IDENTIFIER> ...
-    <BRACKET_QUOTED_IDENTIFIER> ...
-    <UNICODE_QUOTED_IDENTIFIER> ...
-    "LATERAL" "(" ...
-    "UNNEST" ...
-    "LATERAL" "TABLE" ...
+    "AS" ...
+    "CROSS" ...
+    "EXTEND" ...
+    "FOR" ...
+    "MATCH_RECOGNIZE" ...
+    "OUTER" ...
+    "TABLESAMPLE" ...
 !error
 
 # Bad: LATERAL TABLE
@@ -80,18 +79,9 @@ select * from (table "scott".emp) where deptno = 10;
 
 # Bad: Explicit TABLE
 select * from table "scott".emp;
-parse failed: Encountered "table \"scott\"" at line 1, column 15.
-Was expecting one of:
-    <IDENTIFIER> ...
-    <QUOTED_IDENTIFIER> ...
-    <BACK_QUOTED_IDENTIFIER> ...
-    <BRACKET_QUOTED_IDENTIFIER> ...
-    <UNICODE_QUOTED_IDENTIFIER> ...
-    "LATERAL" ...
+parse failed: Encountered "\"scott\"" at line 1, column 21.
+Was expecting:
     "(" ...
-    "UNNEST" ...
-    "TABLE" ...
-    "TABLE" "(" ...
 !error
 
 select * from lateral (select * from "scott".emp) as e
diff --git a/piglet/pom.xml b/piglet/pom.xml
index f860db5..ac5b1f8 100644
--- a/piglet/pom.xml
+++ b/piglet/pom.xml
@@ -132,7 +132,7 @@ limitations under the License.
               <includes>
                 <include>**/PigletParser.jj</include>
               </includes>
-              <lookAhead>2</lookAhead>
+              <lookAhead>1</lookAhead>
               <isStatic>false</isStatic>
             </configuration>
           </execution>
diff --git a/piglet/src/main/javacc/PigletParser.jj b/piglet/src/main/javacc/PigletParser.jj
index ae244d9..30f31e6 100644
--- a/piglet/src/main/javacc/PigletParser.jj
+++ b/piglet/src/main/javacc/PigletParser.jj
@@ -186,7 +186,7 @@ Stmt stmt() :
 }
 {
   (
-    target = simpleIdentifier() <EQ>
+    LOOKAHEAD(2) target = simpleIdentifier() <EQ>
     (
       s = loadStmt(target)
     |
@@ -205,6 +205,7 @@ Stmt stmt() :
       s = groupStmt(target)
     )
   |
+    LOOKAHEAD(2)
     s = describeStmt()
   |
     s = dumpStmt()
@@ -333,6 +334,7 @@ List<Stmt> nestedStmtList() :
     list.add(s);
   }
   (
+    LOOKAHEAD(2)
     s = nestedStmt() {
       list.add(s);
     }
@@ -450,6 +452,7 @@ GroupStmt groupStmt(final Identifier target) :
   |
     <BY>
     (
+      LOOKAHEAD(3)
       keys = tuple()
     |
       exp = exp() {
@@ -594,6 +597,7 @@ Node exp3() :
 }
 {
   (
+    LOOKAHEAD(3)
     <NOT> {
       p = pos();
     }
diff --git a/server/pom.xml b/server/pom.xml
index 33e13ed..ca37f56 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -215,7 +215,7 @@ limitations under the License.
               <includes>
                 <include>**/Parser.jj</include>
               </includes>
-              <lookAhead>2</lookAhead>
+              <lookAhead>1</lookAhead>
               <isStatic>false</isStatic>
             </configuration>
           </execution>
diff --git a/server/src/main/codegen/includes/parserImpls.ftl b/server/src/main/codegen/includes/parserImpls.ftl
index 51ea3b4..e135152 100644
--- a/server/src/main/codegen/includes/parserImpls.ftl
+++ b/server/src/main/codegen/includes/parserImpls.ftl
@@ -128,7 +128,7 @@ void TableElement(List<SqlNode> list) :
     final ColumnStrategy strategy;
 }
 {
-    id = SimpleIdentifier()
+    LOOKAHEAD(2) id = SimpleIdentifier()
     (
         type = DataType()
         (
diff --git a/server/src/test/resources/sql/schema.iq b/server/src/test/resources/sql/schema.iq
index 82ad470..7a334c8 100755
--- a/server/src/test/resources/sql/schema.iq
+++ b/server/src/test/resources/sql/schema.iq
@@ -91,10 +91,10 @@ Encountered "library" at line 1, column 18.
 !error
 
 create foreign schema fs;
-Encountered "<EOF>" at line 1, column 24.
+parse failed: Encountered "<EOF>" at line 1, column 24.
 Was expecting one of:
-    "TYPE" ...
     "LIBRARY" ...
+    "TYPE" ...
     "." ...
 !error