You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2014/11/22 04:17:09 UTC

incubator-calcite git commit: [CALCITE-464] Make parser accept configurable max length for SQL identifier

Repository: incubator-calcite
Updated Branches:
  refs/heads/master ca980a469 -> a5584ea79


[CALCITE-464] Make parser accept configurable max length for SQL identifier

Add new SqlParser.Config interface and SqlParser.ConfigBuilder class; change parser .constructor and factory methods to use it.

Contribution from Jinfeng Ni, some fix-up by Julian Hyde.


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

Branch: refs/heads/master
Commit: a5584ea79244ecbf86150ec8c82bac50d319e0e2
Parents: ca980a4
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Thu Nov 13 18:41:22 2014 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Nov 21 19:14:04 2014 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  11 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |   9 +-
 .../org/apache/calcite/prepare/PlannerImpl.java |  19 +-
 .../sql/parser/SqlAbstractParserImpl.java       |   5 +
 .../apache/calcite/sql/parser/SqlParser.java    | 192 ++++++++++++++++---
 .../apache/calcite/tools/FrameworkConfig.java   |  14 +-
 .../org/apache/calcite/tools/Frameworks.java    |  36 ++--
 .../calcite/sql/parser/SqlParserTest.java       |   9 +-
 .../calcite/sql/test/DefaultSqlTestFactory.java |  14 +-
 .../apache/calcite/test/InterpreterTest.java    |   4 +-
 .../org/apache/calcite/tools/PlannerTest.java   |  30 ++-
 11 files changed, 250 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index e636e87..9c99d5a 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -125,6 +125,7 @@ public class ${parser.class} extends SqlAbstractParserImpl
 
     private Casing unquotedCasing;
     private Casing quotedCasing;
+    private int identifierMaxLength;
 
     /**
      * {@link SqlParserImplFactory} implementation for creating parser.
@@ -187,6 +188,12 @@ public class ${parser.class} extends SqlAbstractParserImpl
     }
 
     // implement SqlAbstractParserImpl
+    public void setIdentifierMaxLength(int identifierMaxLength)
+    {
+        this.identifierMaxLength = identifierMaxLength;
+    }
+
+    // implement SqlAbstractParserImpl
     public SqlNode parseSqlExpressionEof() throws Exception
     {
         return SqlExpressionEof();
@@ -3363,9 +3370,9 @@ String Identifier() :
         | id = NonReservedKeyWord()
     )
     {
-        if (id.length() > 128) {
+        if (id.length() > this.identifierMaxLength) {
             throw SqlUtil.newContextException(getPos(),
-                RESOURCE.identifierTooLong(id, 128));
+                RESOURCE.identifierTooLong(id, this.identifierMaxLength));
         }
         return id;
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 2f850f5..9061d69 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -99,7 +99,6 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlConformance;
@@ -444,8 +443,12 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     if (sql != null) {
       assert queryable == null;
       final CalciteConnectionConfig config = context.config();
-      SqlParser parser = SqlParser.create(SqlParserImpl.FACTORY, sql,
-          config.quoting(), config.unquotedCasing(), config.quotedCasing());
+      SqlParser parser = SqlParser.create(sql,
+          SqlParser.configBuilder()
+              .setQuotedCasing(config.quotedCasing())
+              .setUnquotedCasing(config.unquotedCasing())
+              .setQuoting(config.quoting())
+              .build());
       SqlNode sqlNode;
       try {
         sqlNode = parser.parseStmt();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index 278ba11..accdbf3 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.prepare;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.config.Lex;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPlanner;
@@ -33,7 +32,6 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
 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.validate.SqlValidator;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
@@ -59,13 +57,9 @@ public class PlannerImpl implements Planner {
   /** Holds the trait definitions to be registered with planner. May be null. */
   private final ImmutableList<RelTraitDef> traitDefs;
 
-  private final Lex lex;
-  private final SqlParserImplFactory parserFactory;
+  private final SqlParser.Config parserConfig;
   private final SqlRexConvertletTable convertletTable;
 
-  // Options. TODO: allow client to set these. Maybe use a ConnectionConfig.
-  private boolean caseSensitive = true;
-
   private State state;
 
   // set in STATE_1_RESET
@@ -90,8 +84,7 @@ public class PlannerImpl implements Planner {
     this.defaultSchema = config.getDefaultSchema();
     this.operatorTable = config.getOperatorTable();
     this.programs = config.getPrograms();
-    this.lex = config.getLex();
-    this.parserFactory = config.getParserFactory();
+    this.parserConfig = config.getParserConfig();
     this.state = State.STATE_0_CLOSED;
     this.traitDefs = config.getTraitDefs();
     this.convertletTable = config.getConvertletTable();
@@ -164,8 +157,7 @@ public class PlannerImpl implements Planner {
       ready();
     }
     ensure(State.STATE_2_READY);
-    SqlParser parser = SqlParser.create(parserFactory, sql,
-        lex.quoting, lex.unquotedCasing, lex.quotedCasing);
+    SqlParser parser = SqlParser.create(sql, parserConfig);
     SqlNode sqlNode = parser.parseStmt();
     state = State.STATE_3_PARSED;
     return sqlNode;
@@ -206,8 +198,7 @@ public class PlannerImpl implements Planner {
   public class ViewExpanderImpl implements ViewExpander {
     public RelNode expandView(RelDataType rowType, String queryString,
         List<String> schemaPath) {
-      final SqlParser parser = SqlParser.create(parserFactory, queryString,
-          lex.quoting, lex.unquotedCasing, lex.quotedCasing);
+      SqlParser parser = SqlParser.create(queryString, parserConfig);
       SqlNode sqlNode;
       try {
         sqlNode = parser.parseQuery();
@@ -235,7 +226,7 @@ public class PlannerImpl implements Planner {
     SchemaPlus rootSchema = rootSchema(defaultSchema);
     return new CalciteCatalogReader(
         CalciteSchema.from(rootSchema),
-        caseSensitive,
+        parserConfig.caseSensitive(),
         CalciteSchema.from(defaultSchema).path(null),
         typeFactory);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
index 13f761b..b08278c 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
@@ -437,6 +437,11 @@ public abstract class SqlAbstractParserImpl {
   public abstract void setUnquotedCasing(Casing unquotedCasing);
 
   /**
+   * Sets the maximum length for sql identifier.
+   */
+  public abstract void setIdentifierMaxLength(int identifierMaxLength);
+
+  /**
    * Change parser state.
    *
    * @param stateName new state.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
index 2555b5d..7c0660e 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
@@ -18,30 +18,35 @@ package org.apache.calcite.sql.parser;
 
 import org.apache.calcite.avatica.Casing;
 import org.apache.calcite.avatica.Quoting;
+import org.apache.calcite.config.Lex;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 
+import com.google.common.base.Preconditions;
+
 import java.io.StringReader;
 
 /**
  * A <code>SqlParser</code> parses a SQL statement.
  */
 public class SqlParser {
-  //~ Instance fields --------------------------------------------------------
+  public static final int DEFAULT_IDENTIFIER_MAX_LENGTH = 128;
 
+  //~ Instance fields --------------------------------------------------------
   private final SqlAbstractParserImpl parser;
-  private String originalInput;
+  private final String originalInput;
 
   //~ Constructors -----------------------------------------------------------
   private SqlParser(String s, SqlAbstractParserImpl parser,
-      Quoting quoting, Casing unquotedCasing, Casing quotedCasing) {
+      Config config) {
     this.originalInput = s;
     this.parser = parser;
     parser.setTabSize(1);
-    parser.setQuotedCasing(quotedCasing);
-    parser.setUnquotedCasing(unquotedCasing);
-    switch (quoting) {
+    parser.setQuotedCasing(config.quotedCasing());
+    parser.setUnquotedCasing(config.unquotedCasing());
+    parser.setIdentifierMaxLength(config.identifierMaxLength());
+    switch (config.quoting()) {
     case DOUBLE_QUOTE:
       parser.switchTo("DQID");
       break;
@@ -60,12 +65,15 @@ public class SqlParser {
    * Creates a <code>SqlParser</code> to parse the given string using
    * Calcite's parser implementation.
    *
+   * <p>The default lexical policy is similar to Oracle.
+   *
+   * @see Lex#ORACLE
+   *
    * @param s An SQL statement or expression to parse.
-   * @return A <code>SqlParser</code> object.
+   * @return A parser
    */
   public static SqlParser create(String s) {
-    return create(SqlParserImpl.FACTORY, s, Quoting.DOUBLE_QUOTE,
-        Casing.TO_UPPER, Casing.UNCHANGED);
+    return create(s, configBuilder().build());
   }
 
   /**
@@ -73,22 +81,15 @@ public class SqlParser {
    * parser implementation created from given {@link SqlParserImplFactory}
    * with given quoting syntax and casing policies for identifiers.
    *
-   * @param parserFactory {@link SqlParserImplFactory} to get the parser
-   *     implementation.
-   * @param s An SQL statement or expression to parse.
-   * @param quoting Syntax for quoting identifiers in SQL statements.
-   * @param unquotedCasing Policy for converting case of <i>unquoted</i>
-   *     identifiers.
-   * @param quotedCasing Policy for converting case of <i>quoted</i>
-   *     identifiers.
-   * @return A <code>SqlParser</code> object.
+   * @param sql A SQL statement or expression to parse.
+   * @param config The parser configuration (identifier max length, etc.)
+   * @return A parser
    */
-  public static SqlParser create(SqlParserImplFactory parserFactory, String s,
-      Quoting quoting, Casing unquotedCasing, Casing quotedCasing) {
-    SqlAbstractParserImpl parser = parserFactory.getParser(
-        new StringReader(s));
+  public static SqlParser create(String sql, Config config) {
+    SqlAbstractParserImpl parser =
+        config.parserFactory().getParser(new StringReader(sql));
 
-    return new SqlParser(s, parser, quoting, unquotedCasing, quotedCasing);
+    return new SqlParser(sql, parser, config);
   }
 
   /**
@@ -158,6 +159,151 @@ public class SqlParser {
   public SqlAbstractParserImpl.Metadata getMetadata() {
     return parser.getMetadata();
   }
+
+  /**
+   * Builder for a {@link Config}.
+   */
+  public static ConfigBuilder configBuilder() {
+    return new ConfigBuilder();
+  }
+
+  /**
+   * Builder for a {@link Config} that starts with an existing {@code Config}.
+   */
+  public static ConfigBuilder configBuilder(Config config) {
+    return new ConfigBuilder().setConfig(config);
+  }
+
+  /**
+   * Interface to define the configuration for a SQL parser.
+   *
+   * @see ConfigBuilder
+   */
+  public interface Config {
+    /** Default configuration. */
+    Config DEFAULT = configBuilder().build();
+
+    int identifierMaxLength();
+    Casing quotedCasing();
+    Casing unquotedCasing();
+    Quoting quoting();
+    boolean caseSensitive();
+    SqlParserImplFactory parserFactory();
+  }
+
+  /** Builder for a {@link Config}. */
+  public static class ConfigBuilder {
+    private Casing quotedCasing = Lex.ORACLE.quotedCasing;
+    private Casing unquotedCasing = Lex.ORACLE.unquotedCasing;
+    private Quoting quoting = Lex.ORACLE.quoting;
+    private int identifierMaxLength = DEFAULT_IDENTIFIER_MAX_LENGTH;
+    private boolean caseSensitive = Lex.ORACLE.caseSensitive;
+    private SqlParserImplFactory parserFactory = SqlParserImpl.FACTORY;
+
+    private ConfigBuilder() {}
+
+    /** Sets configuration identical to a given {@link Config}. */
+    public ConfigBuilder setConfig(Config config) {
+      this.quotedCasing = config.quotedCasing();
+      this.unquotedCasing = config.unquotedCasing();
+      this.quoting = config.quoting();
+      this.identifierMaxLength = config.identifierMaxLength();
+      this.parserFactory = config.parserFactory();
+      return this;
+    }
+
+    public ConfigBuilder setQuotedCasing(Casing quotedCasing) {
+      this.quotedCasing = Preconditions.checkNotNull(quotedCasing);
+      return this;
+    }
+
+    public ConfigBuilder setUnquotedCasing(Casing unquotedCasing) {
+      this.unquotedCasing = Preconditions.checkNotNull(unquotedCasing);
+      return this;
+    }
+
+    public ConfigBuilder setQuoting(Quoting quoting) {
+      this.quoting = Preconditions.checkNotNull(quoting);
+      return this;
+    }
+
+    public ConfigBuilder setCaseSensitive(boolean caseSensitive) {
+      this.caseSensitive = caseSensitive;
+      return this;
+    }
+
+    public ConfigBuilder setIdentifierMaxLength(int identifierMaxLength) {
+      this.identifierMaxLength = identifierMaxLength;
+      return this;
+    }
+
+    public ConfigBuilder setParserFactory(SqlParserImplFactory factory) {
+      this.parserFactory = Preconditions.checkNotNull(factory);
+      return this;
+    }
+
+    public ConfigBuilder setLex(Lex lex) {
+      setCaseSensitive(lex.caseSensitive);
+      setUnquotedCasing(lex.unquotedCasing);
+      setQuotedCasing(lex.quotedCasing);
+      setQuoting(lex.quoting);
+      return this;
+    }
+
+    /** Builds a
+     * {@link Config}. */
+    public Config build() {
+      return new ConfigImpl(identifierMaxLength, quotedCasing,
+          unquotedCasing, quoting, caseSensitive, parserFactory);
+    }
+  }
+
+  /** Implementation of
+   * {@link Config}.
+   * Called by builder; all values are in private final fields. */
+  private static class ConfigImpl implements Config {
+    private final int identifierMaxLength;
+    private final boolean caseSensitive;
+    private final Casing quotedCasing;
+    private final Casing unquotedCasing;
+    private final Quoting quoting;
+    private final SqlParserImplFactory parserFactory;
+
+    private ConfigImpl(int identifierMaxLength, Casing quotedCasing,
+        Casing unquotedCasing, Quoting quoting, boolean caseSensitive,
+        SqlParserImplFactory parserFactory) {
+      this.identifierMaxLength = identifierMaxLength;
+      this.caseSensitive = caseSensitive;
+      this.quotedCasing = Preconditions.checkNotNull(quotedCasing);
+      this.unquotedCasing = Preconditions.checkNotNull(unquotedCasing);
+      this.quoting = Preconditions.checkNotNull(quoting);
+      this.parserFactory = Preconditions.checkNotNull(parserFactory);
+    }
+
+    public int identifierMaxLength() {
+      return identifierMaxLength;
+    }
+
+    public Casing quotedCasing() {
+      return quotedCasing;
+    }
+
+    public Casing unquotedCasing() {
+      return unquotedCasing;
+    }
+
+    public Quoting quoting() {
+      return quoting;
+    }
+
+    public boolean caseSensitive() {
+      return caseSensitive;
+    }
+
+    public SqlParserImplFactory parserFactory() {
+      return parserFactory;
+    }
+  }
 }
 
 // End SqlParser.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java b/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
index d3118dd..6301321 100644
--- a/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
+++ b/core/src/main/java/org/apache/calcite/tools/FrameworkConfig.java
@@ -16,14 +16,13 @@
  */
 package org.apache.calcite.tools;
 
-import org.apache.calcite.config.Lex;
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.RelOptCostFactory;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
 
 import com.google.common.collect.ImmutableList;
@@ -36,16 +35,9 @@ import com.google.common.collect.ImmutableList;
  */
 public interface FrameworkConfig {
   /**
-   * The type of lexical analysis the SqlParser should do.  Controls case rules
-   * and quoted identifier syntax.
+   * The configuration of SQL parser.
    */
-  Lex getLex();
-
-  /**
-   * Provides the parser factory that creates the SqlParser used in parsing
-   * queries.
-   */
-  SqlParserImplFactory getParserFactory();
+  SqlParser.Config getParserConfig();
 
   /**
    * Returns the default schema that should be checked before looking at the

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/main/java/org/apache/calcite/tools/Frameworks.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Frameworks.java b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
index 4f835a5..a8ba4b4 100644
--- a/core/src/main/java/org/apache/calcite/tools/Frameworks.java
+++ b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.tools;
 
 import org.apache.calcite.config.CalciteConnectionProperty;
-import org.apache.calcite.config.Lex;
 import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.plan.Context;
@@ -32,8 +31,7 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParserImplFactory;
-import org.apache.calcite.sql.parser.impl.SqlParserImpl;
+import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
 import org.apache.calcite.sql2rel.StandardConvertletTable;
 
@@ -178,17 +176,17 @@ public class Frameworks {
     private ImmutableList<Program> programs = ImmutableList.of();
     private Context context;
     private ImmutableList<RelTraitDef> traitDefs;
-    private Lex lex = Lex.ORACLE;
+    private SqlParser.Config parserConfig =
+        SqlParser.Config.DEFAULT;
     private SchemaPlus defaultSchema;
     private RelOptCostFactory costFactory;
-    private SqlParserImplFactory parserFactory = SqlParserImpl.FACTORY;
     private RelDataTypeSystem typeSystem = RelDataTypeSystem.DEFAULT;
 
     private ConfigBuilder() {}
 
     public FrameworkConfig build() {
       return new StdFrameworkConfig(context, convertletTable, operatorTable,
-          programs, traitDefs, lex, defaultSchema, costFactory, parserFactory,
+          programs, traitDefs, parserConfig, defaultSchema, costFactory,
           typeSystem);
     }
 
@@ -222,8 +220,8 @@ public class Frameworks {
       return this;
     }
 
-    public ConfigBuilder lex(Lex lex) {
-      this.lex = Preconditions.checkNotNull(lex);
+    public ConfigBuilder parserConfig(SqlParser.Config parserConfig) {
+      this.parserConfig = Preconditions.checkNotNull(parserConfig);
       return this;
     }
 
@@ -255,11 +253,6 @@ public class Frameworks {
       return this;
     }
 
-    public ConfigBuilder parserFactory(SqlParserImplFactory parserFactory) {
-      this.parserFactory = Preconditions.checkNotNull(parserFactory);
-      return this;
-    }
-
     public ConfigBuilder typeSystem(RelDataTypeSystem typeSystem) {
       this.typeSystem = Preconditions.checkNotNull(typeSystem);
       return this;
@@ -276,10 +269,9 @@ public class Frameworks {
     private final SqlOperatorTable operatorTable;
     private final ImmutableList<Program> programs;
     private final ImmutableList<RelTraitDef> traitDefs;
-    private final Lex lex;
+    private final SqlParser.Config parserConfig;
     private final SchemaPlus defaultSchema;
     private final RelOptCostFactory costFactory;
-    private final SqlParserImplFactory parserFactory;
     private final RelDataTypeSystem typeSystem;
 
     public StdFrameworkConfig(Context context,
@@ -287,29 +279,23 @@ public class Frameworks {
         SqlOperatorTable operatorTable,
         ImmutableList<Program> programs,
         ImmutableList<RelTraitDef> traitDefs,
-        Lex lex,
+        SqlParser.Config parserConfig,
         SchemaPlus defaultSchema,
         RelOptCostFactory costFactory,
-        SqlParserImplFactory parserFactory,
         RelDataTypeSystem typeSystem) {
       this.context = context;
       this.convertletTable = convertletTable;
       this.operatorTable = operatorTable;
       this.programs = programs;
       this.traitDefs = traitDefs;
-      this.lex = lex;
+      this.parserConfig = parserConfig;
       this.defaultSchema = defaultSchema;
       this.costFactory = costFactory;
-      this.parserFactory = parserFactory;
       this.typeSystem = typeSystem;
     }
 
-    public Lex getLex() {
-      return lex;
-    }
-
-    public SqlParserImplFactory getParserFactory() {
-      return parserFactory;
+    public SqlParser.Config getParserConfig() {
+      return parserConfig;
     }
 
     public SchemaPlus getDefaultSchema() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 9cd6eef..38edbef 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
@@ -21,7 +21,6 @@ import org.apache.calcite.avatica.Quoting;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlSetOption;
-import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.test.SqlValidatorTestCase;
 import org.apache.calcite.util.Bug;
@@ -84,8 +83,12 @@ public class SqlParserTest {
   }
 
   private SqlParser getSqlParser(String sql) {
-    return SqlParser.create(SqlParserImpl.FACTORY, sql, quoting,
-        unquotedCasing, quotedCasing);
+    return SqlParser.create(sql,
+        SqlParser.configBuilder()
+            .setQuoting(quoting)
+            .setUnquotedCasing(unquotedCasing)
+            .setQuotedCasing(quotedCasing)
+            .build());
   }
 
   protected SqlNode parseStmt(String sql) throws SqlParseException {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
index 16a5391..9d9adee 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
@@ -24,7 +24,6 @@ import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.advise.SqlAdvisor;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.parser.impl.SqlParserImpl;
 import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlValidator;
@@ -36,7 +35,7 @@ import org.apache.calcite.test.MockSqlOperatorTable;
 import com.google.common.collect.ImmutableMap;
 
 /**
-* Default implementation of {@link SqlTestFactory}.
+ * Default implementation of {@link SqlTestFactory}.
  *
  * <p>Suitable for most tests. If you want different behavior, you can extend;
  * if you want a factory with different properties (e.g. SQL conformance level
@@ -67,11 +66,12 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
   }
 
   public SqlParser createParser(SqlTestFactory factory, String sql) {
-    Quoting quoting = (Quoting) factory.get("quoting");
-    Casing quotedCasing = (Casing) factory.get("quotedCasing");
-    Casing unquotedCasing = (Casing) factory.get("unquotedCasing");
-    return SqlParser.create(SqlParserImpl.FACTORY, sql, quoting,
-        unquotedCasing, quotedCasing);
+    return SqlParser.create(sql,
+        SqlParser.configBuilder()
+            .setQuoting((Quoting) factory.get("quoting"))
+            .setUnquotedCasing((Casing) factory.get("unquotedCasing"))
+            .setQuotedCasing((Casing) factory.get("quotedCasing"))
+            .build());
   }
 
   public SqlValidator getValidator(SqlTestFactory factory) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
index 0cc0fcf..6a89061 100644
--- a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
@@ -18,12 +18,12 @@ package org.apache.calcite.test;
 
 import org.apache.calcite.DataContext;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.config.Lex;
 import org.apache.calcite.interpreter.Interpreter;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
@@ -76,7 +76,7 @@ public class InterpreterTest {
   @Before public void setUp() {
     rootSchema = Frameworks.createRootSchema(true);
     final FrameworkConfig config = Frameworks.newConfigBuilder()
-        .lex(Lex.ORACLE)
+        .parserConfig(SqlParser.Config.DEFAULT)
         .defaultSchema(
             CalciteAssert.addSchema(rootSchema, CalciteAssert.SchemaSpec.HR))
         .build();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/a5584ea7/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index db202f8..6cfb117 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -57,6 +57,7 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -109,6 +110,23 @@ public class PlannerTest {
             + "    EnumerableTableScan(table=[[hr, emps]])\n");
   }
 
+  @Test(expected = SqlParseException.class)
+  public void testParseIdentiferMaxLengthWithDefault() throws Exception {
+    Planner planner = getPlanner(null, SqlParser.configBuilder().build());
+    planner.parse("select name as "
+        + "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa from \"emps\""
+    );
+  }
+
+  @Test
+  public void testParseIdentiferMaxLengthWithIncreased() throws Exception {
+    Planner planner = getPlanner(null,
+        SqlParser.configBuilder().setIdentifierMaxLength(512).build());
+    planner.parse("select name as "
+        + "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa from \"emps\""
+    );
+  }
+
   /** Unit test that parses, validates and converts the query using
    * order by and offset. */
   @Test public void testParseAndConvertWithOrderByAndOffset() throws Exception {
@@ -204,9 +222,15 @@ public class PlannerTest {
   }
 
   private Planner getPlanner(List<RelTraitDef> traitDefs, Program... programs) {
+    return getPlanner(traitDefs, SqlParser.Config.DEFAULT, programs);
+  }
+
+  private Planner getPlanner(List<RelTraitDef> traitDefs,
+                             SqlParser.Config parserConfig,
+                             Program... programs) {
     final SchemaPlus rootSchema = Frameworks.createRootSchema(true);
     final FrameworkConfig config = Frameworks.newConfigBuilder()
-        .lex(Lex.ORACLE)
+        .parserConfig(parserConfig)
         .defaultSchema(
             CalciteAssert.addSchema(rootSchema, CalciteAssert.SchemaSpec.HR))
         .traitDefs(traitDefs)
@@ -720,7 +744,7 @@ public class PlannerTest {
   private void checkBushy(String sql, String expected) throws Exception {
     final SchemaPlus rootSchema = Frameworks.createRootSchema(true);
     final FrameworkConfig config = Frameworks.newConfigBuilder()
-        .lex(Lex.ORACLE)
+        .parserConfig(SqlParser.Config.DEFAULT)
         .defaultSchema(
             CalciteAssert.addSchema(rootSchema,
                 CalciteAssert.SchemaSpec.CLONE_FOODMART))
@@ -840,7 +864,7 @@ public class PlannerTest {
             new ReflectiveSchema(new TpchSchema()));
 
     final FrameworkConfig config = Frameworks.newConfigBuilder()
-        .lex(Lex.MYSQL)
+        .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build())
         .defaultSchema(schema)
         .programs(Programs.ofRules(Programs.RULE_SET))
         .build();