You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ki...@apache.org on 2020/10/29 17:03:55 UTC

[shardingsphere] branch master updated: Move ShardingSphereSQLParserEngine to infra-parser module (#7965)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new eb86a69  Move ShardingSphereSQLParserEngine to infra-parser module (#7965)
eb86a69 is described below

commit eb86a693685f6d65bf3c9cf7e16ce3b6a08e960f
Author: Liang Zhang <te...@163.com>
AuthorDate: Fri Oct 30 01:03:36 2020 +0800

    Move ShardingSphereSQLParserEngine to infra-parser module (#7965)
    
    * Decouple SQLStatementParserEngine
    
    * Remove SQLStatementParserEngine
    
    * Rename SQLStatementParserEngine
    
    * For code format
    
    * Move ShardingSphereSQLParserEngine to infra
    
    * Add DistSQLParserFactory
    
    * Fix checkstyle
    
    * Update java doc
---
 .../shardingsphere-distsql-parser-engine/pom.xml   |  7 +--
 .../distsql/parser/DistSQLParserFactory.java       | 59 ++++++++++++++++++++++
 ...utor.java => DistSQLStatementParserEngine.java} | 50 +++++++-----------
 .../engine/DistSQLStatementParserEngine.java       | 57 ---------------------
 .../shardingsphere-distsql-parser-sql/pom.xml      |  4 +-
 .../EncryptSQLRewriterParameterizedTest.java       |  7 ++-
 .../MixSQLRewriterParameterizedTest.java           |  7 ++-
 .../ShardingSQLRewriterParameterizedTest.java      |  7 ++-
 .../engine/type/standard/AbstractSQLRouteTest.java |  6 +--
 .../shardingsphere-infra-parser/pom.xml            |  5 ++
 .../infra/parser/SQLStatementParserEngine.java     | 35 -------------
 .../parser/ShardingSphereSQLParserEngine.java      | 31 +++++++-----
 .../SQLStatementParserEngine.java}                 | 15 ++++--
 .../SQLStatementParserEngineFactory.java}          | 18 +++----
 .../statement/ShardingSpherePreparedStatement.java |  4 +-
 .../core/statement/ShardingSphereStatement.java    |  4 +-
 .../text/TextProtocolBackendHandlerFactory.java    |  4 +-
 .../explain/ShardingCTLExplainBackendHandler.java  |  4 +-
 .../execute/MySQLComStmtExecuteExecutor.java       |  4 +-
 .../prepare/MySQLComStmtPrepareExecutor.java       |  4 +-
 .../fieldlist/MySQLComFieldListPacketExecutor.java |  4 +-
 .../binary/bind/PostgreSQLComBindExecutor.java     |  4 +-
 .../binary/parse/PostgreSQLComParseExecutor.java   |  4 +-
 .../sql/parser/core/parser/SQLParserExecutor.java  |  2 +-
 24 files changed, 153 insertions(+), 193 deletions(-)

diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/pom.xml b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/pom.xml
index 16e02e3..7f8a12c 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/pom.xml
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/pom.xml
@@ -26,7 +26,7 @@
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>shardingsphere-distsql-parser-engine</artifactId>
-
+    
     <dependencies>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -39,11 +39,6 @@
         
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-infra-parser</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-distsql-parser-sql</artifactId>
             <version>${project.version}</version>
         </dependency>
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/DistSQLParserFactory.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/DistSQLParserFactory.java
new file mode 100644
index 0000000..5c6c006
--- /dev/null
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/DistSQLParserFactory.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.distsql.parser;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.antlr.v4.runtime.CharStream;
+import org.antlr.v4.runtime.CodePointBuffer;
+import org.antlr.v4.runtime.CodePointCharStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.Lexer;
+import org.antlr.v4.runtime.TokenStream;
+import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementLexer;
+import org.apache.shardingsphere.distsql.parser.sql.parser.DistSQLParser;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
+
+import java.nio.CharBuffer;
+
+/**
+ * Dist SQL parser factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class DistSQLParserFactory {
+    
+    /**
+     * New instance of Dist SQL parser.
+     *
+     * @param sql SQL
+     * @return Dist SQL parser
+     */
+    public static SQLParser newInstance(final String sql) {
+        return new DistSQLParser(createTokenStream(sql));
+    }
+    
+    private static TokenStream createTokenStream(final String sql) {
+        Lexer lexer = new DistSQLStatementLexer(getSQLCharStream(sql));
+        return new CommonTokenStream(lexer);
+    }
+    
+    private static CharStream getSQLCharStream(final String sql) {
+        CodePointBuffer buffer = CodePointBuffer.withChars(CharBuffer.wrap(sql.toCharArray()));
+        return CodePointCharStream.fromBuffer(buffer);
+    }
+}
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/executor/DistSQLParserExecutor.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/DistSQLStatementParserEngine.java
similarity index 50%
rename from shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/executor/DistSQLParserExecutor.java
rename to shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/DistSQLStatementParserEngine.java
index 883f115..1e60d39 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/executor/DistSQLParserExecutor.java
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/DistSQLStatementParserEngine.java
@@ -15,66 +15,52 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.engine.executor;
+package org.apache.shardingsphere.distsql.parser;
 
-import lombok.RequiredArgsConstructor;
 import org.antlr.v4.runtime.BailErrorStrategy;
-import org.antlr.v4.runtime.CodePointBuffer;
-import org.antlr.v4.runtime.CodePointCharStream;
-import org.antlr.v4.runtime.CommonTokenStream;
-import org.antlr.v4.runtime.DefaultErrorStrategy;
-import org.antlr.v4.runtime.Lexer;
 import org.antlr.v4.runtime.Parser;
 import org.antlr.v4.runtime.atn.PredictionMode;
 import org.antlr.v4.runtime.misc.ParseCancellationException;
 import org.antlr.v4.runtime.tree.ErrorNode;
-import org.apache.shardingsphere.distsql.parser.autogen.DistSQLStatementLexer;
-import org.apache.shardingsphere.distsql.parser.sql.parser.DistSQLParser;
+import org.apache.shardingsphere.distsql.parser.sql.visitor.DistSQLStatementVisitor;
 import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
 import org.apache.shardingsphere.sql.parser.core.parser.ParseASTNode;
 import org.apache.shardingsphere.sql.parser.exception.SQLParsingException;
-
-import java.nio.CharBuffer;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 /**
- * Dist SQL parser executor.
+ * Dist SQL statement parser engine.
  */
-@RequiredArgsConstructor
-public final class DistSQLParserExecutor {
-    
-    private final String sql;
+public final class DistSQLStatementParserEngine {
     
     /**
-     * Execute to parse SQL.
+     * Parse SQL.
      *
+     * @param sql SQL to be parsed
      * @return AST node
      */
-    public ParseASTNode execute() {
-        ParseASTNode result = twoPhaseParse();
-        if (result.getRootNode() instanceof ErrorNode) {
+    public SQLStatement parse(final String sql) {
+        ParseASTNode parseASTNode = twoPhaseParse(sql);
+        if (parseASTNode.getRootNode() instanceof ErrorNode) {
             throw new SQLParsingException("Unsupported SQL of `%s`", sql);
         }
-        return result;
+        return (SQLStatement) new DistSQLStatementVisitor().visit(parseASTNode.getRootNode());
     }
     
-    private ParseASTNode twoPhaseParse() {
-        SQLParser sqlParser = createSQLParser();
+    private ParseASTNode twoPhaseParse(final String sql) {
+        SQLParser sqlParser = DistSQLParserFactory.newInstance(sql);
         try {
-            ((Parser) sqlParser).setErrorHandler(new BailErrorStrategy());
-            ((Parser) sqlParser).getInterpreter().setPredictionMode(PredictionMode.SLL);
+            setPredictionMode((Parser) sqlParser, PredictionMode.SLL);
             return (ParseASTNode) sqlParser.parse();
         } catch (final ParseCancellationException ex) {
             ((Parser) sqlParser).reset();
-            ((Parser) sqlParser).setErrorHandler(new DefaultErrorStrategy());
-            ((Parser) sqlParser).getInterpreter().setPredictionMode(PredictionMode.LL);
+            setPredictionMode((Parser) sqlParser, PredictionMode.LL);
             return (ParseASTNode) sqlParser.parse();
         }
     }
     
-    private SQLParser createSQLParser() {
-        CodePointBuffer buffer = CodePointBuffer.withChars(CharBuffer.wrap(sql.toCharArray()));
-        CodePointCharStream codePointCharStream = CodePointCharStream.fromBuffer(buffer);
-        Lexer lexer = new DistSQLStatementLexer(codePointCharStream);
-        return new DistSQLParser(new CommonTokenStream(lexer));
+    private void setPredictionMode(final Parser sqlParser, final PredictionMode mode) {
+        sqlParser.setErrorHandler(new BailErrorStrategy());
+        sqlParser.getInterpreter().setPredictionMode(mode);
     }
 }
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/engine/DistSQLStatementParserEngine.java b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/engine/DistSQLStatementParserEngine.java
deleted file mode 100644
index f1437e3..0000000
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/engine/DistSQLStatementParserEngine.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.distsql.parser.engine.engine;
-
-import lombok.RequiredArgsConstructor;
-import org.antlr.v4.runtime.tree.ParseTree;
-import org.apache.shardingsphere.distsql.parser.engine.executor.DistSQLParserExecutor;
-import org.apache.shardingsphere.infra.parser.SQLStatementParserEngine;
-import org.apache.shardingsphere.distsql.parser.sql.visitor.DistSQLStatementVisitor;
-import org.apache.shardingsphere.sql.parser.hook.ParsingHookRegistry;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
-
-/**
- * Dist SQL statement parser engine.
- */
-@RequiredArgsConstructor
-public final class DistSQLStatementParserEngine implements SQLStatementParserEngine {
-    
-    private final ParsingHookRegistry parsingHookRegistry = ParsingHookRegistry.getInstance();
-    
-    /*
-     * To make sure SkyWalking will be available at the next release of ShardingSphere, a new plugin should be provided to SkyWalking project if this API changed.
-     *
-     * @see <a href="https://github.com/apache/skywalking/blob/master/docs/en/guides/Java-Plugin-Development-Guide.md#user-content-plugin-development-guide">Plugin Development Guide</a>
-     */
-    @SuppressWarnings("OverlyBroadCatchBlock")
-    @Override
-    public SQLStatement parse(final String sql, final boolean useCache) {
-        parsingHookRegistry.start(sql);
-        try {
-            ParseTree parseTree = new DistSQLParserExecutor(sql).execute().getRootNode();
-            SQLStatement result = (SQLStatement) new DistSQLStatementVisitor().visit(parseTree);
-            parsingHookRegistry.finishSuccess(result);
-            return result;
-            // CHECKSTYLE:OFF
-        } catch (final Exception ex) {
-            // CHECKSTYLE:ON
-            parsingHookRegistry.finishFailure(ex);
-            throw ex;
-        }
-    }
-}
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-sql/pom.xml b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-sql/pom.xml
index a85603c..175e1a7 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-sql/pom.xml
+++ b/shardingsphere-distsql-parser/shardingsphere-distsql-parser-sql/pom.xml
@@ -26,7 +26,7 @@
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>shardingsphere-distsql-parser-sql</artifactId>
-
+    
     <dependencies>
         <dependency>
             <groupId>org.apache.commons</groupId>
@@ -47,7 +47,7 @@
             <version>${project.version}</version>
         </dependency>
     </dependencies>
-
+    
     <build>
         <plugins>
             <plugin>
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
index c9567fd..3542feb 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
@@ -44,7 +44,7 @@ import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
-import org.apache.shardingsphere.infra.parser.standard.StandardSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.junit.runners.Parameterized.Parameters;
 
 import java.io.File;
@@ -75,12 +75,11 @@ public final class EncryptSQLRewriterParameterizedTest extends AbstractSQLRewrit
         YamlRootRuleConfigurations ruleConfigurations = createRuleConfigurations();
         Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build(
                 new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(ruleConfigurations.getRules()), ruleConfigurations.getDataSources().keySet());
-        StandardSQLStatementParserEngine standardSQLStatementParserEngine =
-                new StandardSQLStatementParserEngine(null == getTestParameters().getDatabaseType() ? "SQL92" : getTestParameters().getDatabaseType());
+        SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine(null == getTestParameters().getDatabaseType() ? "SQL92" : getTestParameters().getDatabaseType());
         ShardingSphereMetaData metaData = createShardingSphereMetaData();
         ConfigurationProperties props = new ConfigurationProperties(ruleConfigurations.getProps());
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaData.getSchemaMetaData().getConfiguredSchemaMetaData(), 
-                getTestParameters().getInputParameters(), standardSQLStatementParserEngine.parse(getTestParameters().getInputSQL(), false));
+                getTestParameters().getInputParameters(), sqlStatementParserEngine.parse(getTestParameters().getInputSQL(), false));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
         ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), rules, Collections.emptyMap(), metaData);
         RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL, schema);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
index 96d86e8..2e09c66 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
@@ -47,7 +47,7 @@ import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
-import org.apache.shardingsphere.infra.parser.standard.StandardSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.junit.runners.Parameterized.Parameters;
 
 import java.io.File;
@@ -82,12 +82,11 @@ public final class MixSQLRewriterParameterizedTest extends AbstractSQLRewriterPa
         YamlRootRuleConfigurations ruleConfigurations = createRuleConfigurations();
         Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build(
                 new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(ruleConfigurations.getRules()), ruleConfigurations.getDataSources().keySet());
-        StandardSQLStatementParserEngine standardSQLStatementParserEngine = new StandardSQLStatementParserEngine(null == getTestParameters().getDatabaseType() 
-                ? "SQL92" : getTestParameters().getDatabaseType());
+        SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine(null == getTestParameters().getDatabaseType() ? "SQL92" : getTestParameters().getDatabaseType());
         ShardingSphereMetaData metaData = createShardingSphereMetaData();
         ConfigurationProperties props = new ConfigurationProperties(ruleConfigurations.getProps());
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaData.getSchemaMetaData().getConfiguredSchemaMetaData(),
-                getTestParameters().getInputParameters(), standardSQLStatementParserEngine.parse(getTestParameters().getInputSQL(), false));
+                getTestParameters().getInputParameters(), sqlStatementParserEngine.parse(getTestParameters().getInputSQL(), false));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
         ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), rules, Collections.emptyMap(), metaData);
         RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL, schema);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
index 0c9276f..a75b8d2 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
@@ -47,7 +47,7 @@ import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
-import org.apache.shardingsphere.infra.parser.standard.StandardSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.junit.runners.Parameterized.Parameters;
 
 import java.io.File;
@@ -82,12 +82,11 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
         YamlRootRuleConfigurations yamlRootRuleConfigs = createYamlRootRuleConfigurations();
         Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build(
                 new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(yamlRootRuleConfigs.getRules()), yamlRootRuleConfigs.getDataSources().keySet());
-        StandardSQLStatementParserEngine standardSQLStatementParserEngine =
-                new StandardSQLStatementParserEngine(null == getTestParameters().getDatabaseType() ? "SQL92" : getTestParameters().getDatabaseType());
+        SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine(null == getTestParameters().getDatabaseType() ? "SQL92" : getTestParameters().getDatabaseType());
         ShardingSphereMetaData metaData = createShardingSphereMetaData();
         ConfigurationProperties props = new ConfigurationProperties(yamlRootRuleConfigs.getProps());
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(metaData.getSchemaMetaData().getConfiguredSchemaMetaData(), 
-                getTestParameters().getInputParameters(), standardSQLStatementParserEngine.parse(getTestParameters().getInputSQL(), false));
+                getTestParameters().getInputParameters(), sqlStatementParserEngine.parse(getTestParameters().getInputSQL(), false));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
         ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), rules, Collections.emptyMap(), metaData);
         RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL, schema);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
index 0fc3f3b..5a6076c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
@@ -36,7 +36,7 @@ import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.sharding.route.engine.fixture.AbstractRoutingEngineTest;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.parser.standard.StandardSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 
 import java.sql.Types;
 import java.util.Arrays;
@@ -57,9 +57,9 @@ public abstract class AbstractSQLRouteTest extends AbstractRoutingEngineTest {
         ShardingRule shardingRule = createAllShardingRule();
         ShardingSphereMetaData metaData = new ShardingSphereMetaData(buildDataSourceMetas(), buildLogicSchemaMetaData(), mock(TableAddressingMetaData.class), mock(CachedDatabaseMetaData.class));
         ConfigurationProperties props = new ConfigurationProperties(new Properties());
-        StandardSQLStatementParserEngine standardSQLStatementParserEngine = new StandardSQLStatementParserEngine("MySQL");
+        SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine("MySQL");
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
-                metaData.getSchemaMetaData().getConfiguredSchemaMetaData(), parameters, standardSQLStatementParserEngine.parse(sql, false));
+                metaData.getSchemaMetaData().getConfiguredSchemaMetaData(), parameters, sqlStatementParserEngine.parse(sql, false));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, sql, parameters);
         ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), Collections.singleton(shardingRule), Collections.emptyMap(), metaData);
         RouteContext result = new SQLRouteEngine(props, Collections.singletonList(shardingRule)).route(logicSQL, schema);
diff --git a/shardingsphere-infra/shardingsphere-infra-parser/pom.xml b/shardingsphere-infra/shardingsphere-infra-parser/pom.xml
index fdbc686..203f8c2 100644
--- a/shardingsphere-infra/shardingsphere-infra-parser/pom.xml
+++ b/shardingsphere-infra/shardingsphere-infra-parser/pom.xml
@@ -34,5 +34,10 @@
             <artifactId>shardingsphere-sql-parser-engine</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-distsql-parser-engine</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
 </project>
diff --git a/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/SQLStatementParserEngine.java b/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/SQLStatementParserEngine.java
deleted file mode 100644
index ebc3945..0000000
--- a/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/SQLStatementParserEngine.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.parser;
-
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
-
-/**
- * SQL statement parser engine.
- */
-public interface SQLStatementParserEngine {
-    
-    /**
-     * Parse to SQL statement.
-     * 
-     * @param sql SQL to be parsed
-     * @param useCache whether use cache
-     * @return SQL statement
-     */
-    SQLStatement parse(String sql, boolean useCache);
-}
diff --git a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/ShardingSphereSQLStatementParserEngine.java b/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/ShardingSphereSQLParserEngine.java
similarity index 56%
rename from shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/ShardingSphereSQLStatementParserEngine.java
rename to shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/ShardingSphereSQLParserEngine.java
index 1850786..9ee56f5 100644
--- a/shardingsphere-distsql-parser/shardingsphere-distsql-parser-engine/src/main/java/org/apache/shardingsphere/distsql/parser/engine/ShardingSphereSQLStatementParserEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/ShardingSphereSQLParserEngine.java
@@ -15,37 +15,42 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.parser.engine;
+package org.apache.shardingsphere.infra.parser;
 
-import org.apache.shardingsphere.distsql.parser.engine.engine.DistSQLStatementParserEngine;
-import org.apache.shardingsphere.infra.parser.SQLStatementParserEngine;
-import org.apache.shardingsphere.infra.parser.standard.StandardSQLStatementParserEngine;
-import org.apache.shardingsphere.infra.parser.standard.StandardSQLStatementParserEngineFactory;
+import org.apache.shardingsphere.distsql.parser.DistSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngineFactory;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 /**
- * ShardingSphere SQL statement parser engine.
+ * ShardingSphere SQL parser engine.
  */
-public final class ShardingSphereSQLStatementParserEngine implements SQLStatementParserEngine {
+public final class ShardingSphereSQLParserEngine {
     
-    private final StandardSQLStatementParserEngine standardSQLStatementParserEngine;
+    private final SQLStatementParserEngine sqlStatementParserEngine;
     
     private final DistSQLStatementParserEngine distSQLStatementParserEngine;
     
-    public ShardingSphereSQLStatementParserEngine(final String databaseTypeName) {
-        standardSQLStatementParserEngine = StandardSQLStatementParserEngineFactory.getSQLStatementParserEngine(databaseTypeName);
+    public ShardingSphereSQLParserEngine(final String databaseTypeName) {
+        sqlStatementParserEngine = SQLStatementParserEngineFactory.getSQLStatementParserEngine(databaseTypeName);
         distSQLStatementParserEngine = new DistSQLStatementParserEngine();
     }
     
-    @Override
+    /**
+     * Parse to SQL statement.
+     *
+     * @param sql SQL to be parsed
+     * @param useCache whether use cache
+     * @return SQL statement
+     */
     public SQLStatement parse(final String sql, final boolean useCache) {
         SQLStatement result;
         try {
-            result = standardSQLStatementParserEngine.parse(sql, useCache);
+            result = sqlStatementParserEngine.parse(sql, useCache);
             // CHECKSTYLE:OFF
         } catch (final Exception ex) {
             // CHECKSTYLE:ON
-            result = distSQLStatementParserEngine.parse(sql, useCache);
+            result = distSQLStatementParserEngine.parse(sql);
         }
         return result;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/standard/StandardSQLStatementParserEngine.java b/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/sql/SQLStatementParserEngine.java
similarity index 90%
rename from shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/standard/StandardSQLStatementParserEngine.java
rename to shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/sql/SQLStatementParserEngine.java
index 25a0484..2fe0193 100644
--- a/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/standard/StandardSQLStatementParserEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/sql/SQLStatementParserEngine.java
@@ -15,22 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.parser.standard;
+package org.apache.shardingsphere.infra.parser.sql;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.sql.parser.api.SQLParserEngine;
 import org.apache.shardingsphere.sql.parser.cache.SQLParsedResultCache;
 import org.apache.shardingsphere.sql.parser.hook.ParsingHookRegistry;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
-import org.apache.shardingsphere.infra.parser.SQLStatementParserEngine;
 
 import java.util.Optional;
 
 /**
- * Standard SQL statement parser engine.
+ * SQL statement parser engine.
  */
 @RequiredArgsConstructor
-public final class StandardSQLStatementParserEngine implements SQLStatementParserEngine {
+public final class SQLStatementParserEngine {
     
     private final String databaseTypeName;
     
@@ -43,8 +42,14 @@ public final class StandardSQLStatementParserEngine implements SQLStatementParse
      *
      * @see <a href="https://github.com/apache/skywalking/blob/master/docs/en/guides/Java-Plugin-Development-Guide.md#user-content-plugin-development-guide">Plugin Development Guide</a>
      */
+    /**
+     * Parse to SQL statement.
+     *
+     * @param sql SQL to be parsed
+     * @param useCache whether use cache
+     * @return SQL statement
+     */
     @SuppressWarnings("OverlyBroadCatchBlock")
-    @Override
     public SQLStatement parse(final String sql, final boolean useCache) {
         parsingHookRegistry.start(sql);
         try {
diff --git a/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/standard/StandardSQLStatementParserEngineFactory.java b/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/sql/SQLStatementParserEngineFactory.java
similarity index 66%
rename from shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/standard/StandardSQLStatementParserEngineFactory.java
rename to shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/sql/SQLStatementParserEngineFactory.java
index 941cf3f..ceea8c6 100644
--- a/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/standard/StandardSQLStatementParserEngineFactory.java
+++ b/shardingsphere-infra/shardingsphere-infra-parser/src/main/java/org/apache/shardingsphere/infra/parser/sql/SQLStatementParserEngineFactory.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.parser.standard;
+package org.apache.shardingsphere.infra.parser.sql;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
@@ -24,20 +24,20 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
 /**
- * Standard SQL statement parser engine factory.
+ * SQL statement parser engine factory.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class StandardSQLStatementParserEngineFactory {
+public final class SQLStatementParserEngineFactory {
     
-    private static final Map<String, StandardSQLStatementParserEngine> ENGINES = new ConcurrentHashMap<>();
+    private static final Map<String, SQLStatementParserEngine> ENGINES = new ConcurrentHashMap<>();
     
     /**
-     * Get standard SQL statement parser engine.
-     *x
+     * Get SQL statement parser engine.
+     *
      * @param databaseType name of database type
-     * @return standard SQL statement parser engine
+     * @return SQL statement parser engine
      */
-    public static StandardSQLStatementParserEngine getSQLStatementParserEngine(final String databaseType) {
-        return ENGINES.containsKey(databaseType) ? ENGINES.get(databaseType) : ENGINES.computeIfAbsent(databaseType, StandardSQLStatementParserEngine::new);
+    public static SQLStatementParserEngine getSQLStatementParserEngine(final String databaseType) {
+        return ENGINES.containsKey(databaseType) ? ENGINES.get(databaseType) : ENGINES.computeIfAbsent(databaseType, SQLStatementParserEngine::new);
     }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index da68478..768cfb9 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -58,7 +58,7 @@ import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.model.physical.model.schema.PhysicalSchemaMetaData;
 import org.apache.shardingsphere.infra.rule.DataNodeRoutedRule;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
 
@@ -138,7 +138,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         this.sql = sql;
         statements = new ArrayList<>();
         parameterSets = new ArrayList<>();
-        ShardingSphereSQLStatementParserEngine sqlStatementParserEngine = new ShardingSphereSQLStatementParserEngine(DatabaseTypeRegistry.getTrunkDatabaseTypeName(schemaContexts.getDatabaseType()));
+        ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypeRegistry.getTrunkDatabaseTypeName(schemaContexts.getDatabaseType()));
         sqlStatement = sqlStatementParserEngine.parse(sql, true);
         parameterMetaData = new ShardingSphereParameterMetaData(sqlStatement);
         statementOption = returnGeneratedKeys ? new StatementOption(true) : new StatementOption(resultSetType, resultSetConcurrency, resultSetHoldability);
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index f850bf9..21fff58 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -56,7 +56,7 @@ import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.model.physical.model.schema.PhysicalSchemaMetaData;
 import org.apache.shardingsphere.infra.rule.DataNodeRoutedRule;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
 
@@ -306,7 +306,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     
     private LogicSQL createLogicSQL(final String sql) {
         PhysicalSchemaMetaData schemaMetaData = schemaContexts.getDefaultSchema().getMetaData().getSchemaMetaData().getSchemaMetaData();
-        ShardingSphereSQLStatementParserEngine sqlStatementParserEngine = new ShardingSphereSQLStatementParserEngine(DatabaseTypeRegistry.getTrunkDatabaseTypeName(schemaContexts.getDatabaseType()));
+        ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypeRegistry.getTrunkDatabaseTypeName(schemaContexts.getDatabaseType()));
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, Collections.emptyList(), sqlStatement);
         return new LogicSQL(sqlStatementContext, sql, Collections.emptyList());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactory.java
index 6fa850c..74aa83e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/TextProtocolBackendHandlerFactory.java
@@ -29,7 +29,7 @@ import org.apache.shardingsphere.proxy.backend.text.sctl.ShardingCTLBackendHandl
 import org.apache.shardingsphere.proxy.backend.text.sctl.utils.SCTLUtils;
 import org.apache.shardingsphere.proxy.backend.text.skip.SkipBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.transaction.TransactionBackendHandlerFactory;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.RDLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
@@ -60,7 +60,7 @@ public final class TextProtocolBackendHandlerFactory {
         if (trimSQL.toUpperCase().startsWith(ShardingCTLBackendHandlerFactory.SCTL)) {
             return ShardingCTLBackendHandlerFactory.newInstance(trimSQL, backendConnection);
         }
-        SQLStatement sqlStatement = new ShardingSphereSQLStatementParserEngine(databaseType.getName()).parse(sql, false);
+        SQLStatement sqlStatement = new ShardingSphereSQLParserEngine(databaseType.getName()).parse(sql, false);
         if (sqlStatement instanceof RDLStatement || sqlStatement instanceof CreateDatabaseStatement || sqlStatement instanceof DropDatabaseStatement) {
             return new RDLBackendHandler(backendConnection, sqlStatement);
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandler.java
index 9d52971..8cac7d0 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandler.java
@@ -36,7 +36,7 @@ import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
 import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
 import org.apache.shardingsphere.proxy.backend.text.sctl.exception.InvalidShardingCTLFormatException;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.Types;
@@ -85,7 +85,7 @@ public final class ShardingCTLExplainBackendHandler implements TextProtocolBacke
     
     private LogicSQL createLogicSQL(final ShardingSphereSchema schema, final ShardingCTLExplainStatement explainStatement) {
         PhysicalSchemaMetaData schemaMetaData = schema.getMetaData().getSchemaMetaData().getSchemaMetaData();
-        ShardingSphereSQLStatementParserEngine sqlStatementParserEngine = new ShardingSphereSQLStatementParserEngine(
+        ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                 DatabaseTypeRegistry.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(explainStatement.getSql(), false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, Collections.emptyList(), sqlStatement);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
index e4dce8f..419eee1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
@@ -36,7 +36,7 @@ import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
 import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
 import org.apache.shardingsphere.proxy.frontend.command.executor.ResponseType;
 import org.apache.shardingsphere.proxy.frontend.mysql.command.query.builder.ResponsePacketBuilder;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.SQLException;
@@ -57,7 +57,7 @@ public final class MySQLComStmtExecuteExecutor implements QueryCommandExecutor {
     private int currentSequenceId;
     
     public MySQLComStmtExecuteExecutor(final MySQLComStmtExecutePacket packet, final BackendConnection backendConnection) {
-        ShardingSphereSQLStatementParserEngine sqlStatementParserEngine = new ShardingSphereSQLStatementParserEngine(
+        ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                 DatabaseTypeRegistry.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(packet.getSql(), true);
         databaseCommunicationEngine = DatabaseCommunicationEngineFactory.getInstance().newBinaryProtocolInstance(sqlStatement, packet.getSql(), packet.getParameters(), backendConnection);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareExecutor.java
index 07271a1..25e0f3a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/prepare/MySQLComStmtPrepareExecutor.java
@@ -29,7 +29,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
 import org.apache.shardingsphere.proxy.frontend.exception.UnsupportedPreparedStatementException;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 
@@ -50,7 +50,7 @@ public final class MySQLComStmtPrepareExecutor implements CommandExecutor {
     
     @Override
     public Collection<DatabasePacket<?>> execute() {
-        ShardingSphereSQLStatementParserEngine sqlStatementParserEngine = new ShardingSphereSQLStatementParserEngine(
+        ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                 DatabaseTypeRegistry.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(packet.getSql(), true);
         if (!MySQLComStmtPrepareChecker.isStatementAllowed(sqlStatement)) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/fieldlist/MySQLComFieldListPacketExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/fieldlist/MySQLComFieldListPacketExecutor.java
index 86d3b65..0ea990b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/fieldlist/MySQLComFieldListPacketExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/text/fieldlist/MySQLComFieldListPacketExecutor.java
@@ -28,7 +28,7 @@ import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicati
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.SQLException;
@@ -54,7 +54,7 @@ public final class MySQLComFieldListPacketExecutor implements CommandExecutor {
         this.packet = packet;
         schemaName = backendConnection.getSchemaName();
         String sql = String.format(SQL, packet.getTable(), schemaName);
-        ShardingSphereSQLStatementParserEngine sqlStatementParserEngine = new ShardingSphereSQLStatementParserEngine(
+        ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                 DatabaseTypeRegistry.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
         SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
         databaseCommunicationEngine = DatabaseCommunicationEngineFactory.getInstance().newTextProtocolInstance(sqlStatement, sql, backendConnection);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/bind/PostgreSQLComBindExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/bind/PostgreSQLComBindExecutor.java
index 0df2e4d..3c1e1dc 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/bind/PostgreSQLComBindExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/bind/PostgreSQLComBindExecutor.java
@@ -43,7 +43,7 @@ import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
 import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
 import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
 import org.apache.shardingsphere.proxy.frontend.command.executor.ResponseType;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.ResultSetMetaData;
@@ -71,7 +71,7 @@ public final class PostgreSQLComBindExecutor implements QueryCommandExecutor {
         this.packet = packet;
         ShardingSphereSchema schema = ProxyContext.getInstance().getSchema(backendConnection.getSchemaName());
         if (null != packet.getSql() && null != schema) {
-            ShardingSphereSQLStatementParserEngine sqlStatementParserEngine = new ShardingSphereSQLStatementParserEngine(
+            ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                     DatabaseTypeRegistry.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
             SQLStatement sqlStatement = sqlStatementParserEngine.parse(packet.getSql(), true);
             databaseCommunicationEngine =
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutor.java
index 4e5bf52..13dde4e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutor.java
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
-import org.apache.shardingsphere.distsql.parser.engine.ShardingSphereSQLStatementParserEngine;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Collection;
@@ -49,7 +49,7 @@ public final class PostgreSQLComParseExecutor implements CommandExecutor {
     @Override
     public Collection<DatabasePacket<?>> execute() {
         if (!packet.getSql().isEmpty()) {
-            ShardingSphereSQLStatementParserEngine sqlStatementParserEngine = new ShardingSphereSQLStatementParserEngine(
+            ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
                     DatabaseTypeRegistry.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
             SQLStatement sqlStatement = sqlStatementParserEngine.parse(packet.getSql(), true);
             binaryStatementRegistry.register(packet.getStatementId(), packet.getSql(), sqlStatement.getParameterCount(), packet.getBinaryStatementParameterTypes());
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
index 4b1f8d5..c1fe00d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/parser/SQLParserExecutor.java
@@ -41,7 +41,7 @@ public final class SQLParserExecutor {
     private final SQLParsedResultCache<ParseTree> cache = new SQLParsedResultCache<>();
     
     /**
-     * Parse.
+     * Parse SQL.
      *
      * @param sql SQL to be parsed
      * @param useCache whether use cache