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

[shardingsphere] branch master updated: Remove ShardingSphereSQLParserEngine from SchemaContexts (#7832)

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

xiaoyu 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 8131f34  Remove ShardingSphereSQLParserEngine from SchemaContexts (#7832)
8131f34 is described below

commit 8131f34f9307a0c6a7011a6f86ae6c9ce4fb3352
Author: Liang Zhang <te...@163.com>
AuthorDate: Sun Oct 18 17:03:19 2020 +0800

    Remove ShardingSphereSQLParserEngine from SchemaContexts (#7832)
    
    * Remove ShardingSphereSQLParserEngineFactory
    
    * Remove ShardingSphereSQLParserEngineFactory
    
    * Remove ShardingSphereSQLParserEngine from SchemaContextsBuilder
    
    * Remove ShardingSphereSQLParserEngine from SchemaContexts
    
    * Delete DALStatement.interp
    
    * Delete DALStatement.tokens
    
    * Delete DALStatementLexer.tokens
    
    * Delete DALStatementLexer.interp
    
    * Delete DALStatementVisitor.java
    
    * Delete DALStatementParser.java
    
    * Delete DALStatementListener.java
    
    * Delete DALStatementLexer.java
    
    * Delete DALStatementBaseVisitor.java
    
    * Delete DALStatementBaseListener.java
---
 .../context/schema/GovernanceSchemaContexts.java   | 31 ++++++----------
 .../schema/GovernanceSchemaContextsTest.java       |  4 +--
 .../infra/context/schema/SchemaContexts.java       |  8 -----
 .../context/schema/SchemaContextsBuilder.java      |  8 +----
 .../schema/impl/StandardSchemaContexts.java        |  9 ++---
 .../schema/impl/StandardSchemaContextsTest.java    |  6 ++--
 .../statement/ShardingSpherePreparedStatement.java | 19 +++++-----
 .../core/statement/ShardingSphereStatement.java    | 19 +++++-----
 .../text/TextProtocolBackendHandlerFactory.java    |  4 +--
 .../explain/ShardingCTLExplainBackendHandler.java  | 13 ++++---
 .../DatabaseCommunicationEngineFactoryTest.java    |  5 ++-
 .../jdbc/connection/BackendConnectionTest.java     |  5 ++-
 .../jdbc/datasource/JDBCBackendDataSourceTest.java |  5 ++-
 .../proxy/backend/context/ProxyContextTest.java    | 17 +++++----
 .../text/admin/BroadcastBackendHandlerTest.java    |  5 ++-
 .../backend/text/admin/RDLBackendHandlerTest.java  |  5 ++-
 .../admin/ShowDatabasesBackendHandlerTest.java     |  5 ++-
 .../text/admin/ShowTablesBackendHandlerTest.java   |  5 ++-
 .../text/admin/UnicastBackendHandlerTest.java      |  5 ++-
 .../text/admin/UseDatabaseBackendHandlerTest.java  |  5 ++-
 .../ShardingCTLExplainBackendHandlerTest.java      |  5 ++-
 .../hint/ShardingCTLHintBackendHandlerTest.java    |  5 ++-
 .../sctl/set/ShardingCTLSetBackendHandlerTest.java |  5 ++-
 .../execute/MySQLComStmtExecuteExecutor.java       |  3 +-
 .../prepare/MySQLComStmtPrepareExecutor.java       |  5 ++-
 .../fieldlist/MySQLComFieldListPacketExecutor.java |  5 ++-
 .../frontend/mysql/MySQLFrontendEngineTest.java    |  4 +--
 .../mysql/auth/MySQLAuthenticationEngineTest.java  |  3 +-
 .../mysql/auth/MySQLAuthenticationHandlerTest.java |  4 +--
 .../command/MySQLCommandExecutorFactoryTest.java   | 23 ++++++------
 .../execute/MySQLComStmtExecuteExecutorTest.java   | 19 +++++-----
 .../binary/bind/PostgreSQLComBindExecutor.java     |  5 ++-
 .../binary/parse/PostgreSQLComParseExecutor.java   |  5 ++-
 .../parse/PostgreSQLComParseExecutorTest.java      | 17 ++-------
 .../engine/ShardingSphereSQLParserEngine.java      |  7 ++--
 .../ShardingSphereSQLParserEngineFactory.java      | 41 ----------------------
 .../src/main/antlr4/imports/mysql/DALStatement.g4  |  2 +-
 37 files changed, 133 insertions(+), 208 deletions(-)

diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/main/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContexts.java b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/main/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContexts.java
index 81c21d2..0aff7e3 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/main/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContexts.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/main/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContexts.java
@@ -49,7 +49,6 @@ import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.StatusContainedRule;
 import org.apache.shardingsphere.infra.rule.event.impl.DataSourceNameDisabledEvent;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 
 import javax.sql.DataSource;
 import java.io.IOException;
@@ -125,11 +124,6 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
     }
     
     @Override
-    public ShardingSphereSQLParserEngine getSqlParserEngine() {
-        return schemaContexts.getSqlParserEngine();
-    }
-    
-    @Override
     public ExecutorKernel getExecutorKernel() {
         return schemaContexts.getExecutorKernel();
     }
@@ -165,8 +159,7 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
     public synchronized void renew(final SchemaAddedEvent event) throws SQLException {
         Map<String, ShardingSphereSchema> schemas = new HashMap<>(schemaContexts.getSchemas());
         schemas.put(event.getSchemaName(), createAddedSchemaContext(event));
-        schemaContexts = new StandardSchemaContexts(schemas, 
-                schemaContexts.getSqlParserEngine(), schemaContexts.getExecutorKernel(), schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
+        schemaContexts = new StandardSchemaContexts(schemas, schemaContexts.getExecutorKernel(), schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
         governanceFacade.getConfigCenter().persistMetaData(event.getSchemaName(), schemaContexts.getSchemas().get(event.getSchemaName()).getMetaData().getSchemaMetaData());
         GovernanceEventBus.getInstance().post(
                 new DataSourceChangeCompletedEvent(event.getSchemaName(), schemaContexts.getDatabaseType(), schemas.get(event.getSchemaName()).getDataSources()));
@@ -181,8 +174,7 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
     public synchronized void renew(final SchemaDeletedEvent event) {
         Map<String, ShardingSphereSchema> schemas = new HashMap<>(schemaContexts.getSchemas());
         schemas.remove(event.getSchemaName());
-        schemaContexts = new StandardSchemaContexts(schemas, schemaContexts.getSqlParserEngine(), schemaContexts.getExecutorKernel(), 
-                schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
+        schemaContexts = new StandardSchemaContexts(schemas, schemaContexts.getExecutorKernel(), schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
         governanceFacade.getConfigCenter().deleteSchema(event.getSchemaName());
     }
     
@@ -194,8 +186,7 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
     @Subscribe
     public synchronized void renew(final PropertiesChangedEvent event) {
         ConfigurationProperties props = new ConfigurationProperties(event.getProps());
-        schemaContexts = new StandardSchemaContexts(getChangedSchemas(), schemaContexts.getSqlParserEngine(), schemaContexts.getExecutorKernel(), 
-                schemaContexts.getAuthentication(), props, schemaContexts.getDatabaseType());
+        schemaContexts = new StandardSchemaContexts(getChangedSchemas(), schemaContexts.getExecutorKernel(), schemaContexts.getAuthentication(), props, schemaContexts.getDatabaseType());
     }
     
     /**
@@ -205,8 +196,8 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
      */
     @Subscribe
     public synchronized void renew(final AuthenticationChangedEvent event) {
-        schemaContexts = new StandardSchemaContexts(schemaContexts.getSchemas(), schemaContexts.getSqlParserEngine(), schemaContexts.getExecutorKernel(), 
-                event.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
+        schemaContexts = new StandardSchemaContexts(
+                schemaContexts.getSchemas(), schemaContexts.getExecutorKernel(), event.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
     }
     
     /**
@@ -223,8 +214,7 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
             ShardingSphereSchema newSchema = event.getSchemaName().equals(schemaName) ? getChangedShardingSphereSchema(oldSchema, event.getLogicSchemaMetaData(), schemaName) : oldSchema;
             newSchemas.put(schemaName, newSchema);
         }
-        schemaContexts = new StandardSchemaContexts(newSchemas, schemaContexts.getSqlParserEngine(), schemaContexts.getExecutorKernel(), 
-                schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
+        schemaContexts = new StandardSchemaContexts(newSchemas, schemaContexts.getExecutorKernel(), schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
     }
     
     /**
@@ -239,8 +229,8 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
         String schemaName = event.getSchemaName();
         newSchemaContexts.remove(schemaName);
         newSchemaContexts.put(schemaName, getChangedSchema(schemaContexts.getSchemas().get(schemaName), event.getRuleConfigurations()));
-        schemaContexts = new StandardSchemaContexts(newSchemaContexts, schemaContexts.getSqlParserEngine(), schemaContexts.getExecutorKernel(), 
-                schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
+        schemaContexts = new StandardSchemaContexts(
+                newSchemaContexts, schemaContexts.getExecutorKernel(), schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
         governanceFacade.getConfigCenter().persistMetaData(schemaName, newSchemaContexts.get(schemaName).getMetaData().getSchemaMetaData());
     }
     
@@ -256,8 +246,7 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
         Map<String, ShardingSphereSchema> newSchemas = new HashMap<>(schemaContexts.getSchemas());
         newSchemas.remove(schemaName);
         newSchemas.put(schemaName, getChangedSchema(schemaContexts.getSchemas().get(schemaName), event.getDataSourceConfigurations()));
-        schemaContexts = new StandardSchemaContexts(newSchemas, schemaContexts.getSqlParserEngine(), schemaContexts.getExecutorKernel(), 
-                schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
+        schemaContexts = new StandardSchemaContexts(newSchemas, schemaContexts.getExecutorKernel(), schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType());
         GovernanceEventBus.getInstance().post(
                 new DataSourceChangeCompletedEvent(event.getSchemaName(), schemaContexts.getDatabaseType(), newSchemas.get(event.getSchemaName()).getDataSources()));
     }
@@ -285,7 +274,7 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
      */
     @Subscribe
     public synchronized void renew(final CircuitStateChangedEvent event) {
-        schemaContexts = new StandardSchemaContexts(schemaContexts.getSchemas(), schemaContexts.getSqlParserEngine(), 
+        schemaContexts = new StandardSchemaContexts(schemaContexts.getSchemas(), 
                 schemaContexts.getExecutorKernel(), schemaContexts.getAuthentication(), schemaContexts.getProps(), schemaContexts.getDatabaseType(), event.isCircuitBreak());
     }
     
diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/test/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContextsTest.java b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/test/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContextsTest.java
index 7fd12cf..fd0aaea 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/test/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContextsTest.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/test/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContextsTest.java
@@ -43,7 +43,6 @@ import org.apache.shardingsphere.infra.metadata.model.logic.LogicSchemaMetaData;
 import org.apache.shardingsphere.infra.rule.event.RuleChangedEvent;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.jdbc.test.MockedDataSource;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.replication.primaryreplica.rule.PrimaryReplicaReplicationRule;
 import org.junit.Before;
 import org.junit.Test;
@@ -106,8 +105,7 @@ public final class GovernanceSchemaContextsTest {
         when(governanceFacade.getRegistryCenter()).thenReturn(registryCenter);
         when(governanceFacade.getConfigCenter()).thenReturn(configCenter);
         when(registryCenter.loadDisabledDataSources("schema")).thenReturn(Collections.singletonList("schema.ds_1"));
-        governanceSchemaContexts = new GovernanceSchemaContexts(
-                new StandardSchemaContexts(createSchemas(), mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), authentication, props, databaseType), governanceFacade);
+        governanceSchemaContexts = new GovernanceSchemaContexts(new StandardSchemaContexts(createSchemas(), mock(ExecutorKernel.class), authentication, props, databaseType), governanceFacade);
     }
     
     private Map<String, ShardingSphereSchema> createSchemas() {
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContexts.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContexts.java
index cc53ddd..a34e14f 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContexts.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContexts.java
@@ -22,7 +22,6 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 
 import java.io.Closeable;
 import java.util.Map;
@@ -54,13 +53,6 @@ public interface SchemaContexts extends Closeable {
     ShardingSphereSchema getDefaultSchema();
     
     /**
-     * Get SQL parser engine.
-     * 
-     * @return SQL parser engine
-     */
-    ShardingSphereSQLParserEngine getSqlParserEngine();
-    
-    /**
      * Get executor kernel.
      * 
      * @return executor kernel
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContextsBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContextsBuilder.java
index 663b600..94a4f0d 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContextsBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContextsBuilder.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.metadata.model.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.model.addressing.TableAddressingMetaData;
@@ -37,8 +36,6 @@ import org.apache.shardingsphere.infra.metadata.model.logic.LogicSchemaMetaDataL
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRulesBuilder;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngineFactory;
 
 import javax.sql.DataSource;
 import java.sql.Connection;
@@ -67,8 +64,6 @@ public final class SchemaContextsBuilder {
     
     private final ConfigurationProperties props;
     
-    private final ShardingSphereSQLParserEngine sqlParserEngine;
-    
     private final ExecutorKernel executorKernel;
     
     public SchemaContextsBuilder(final DatabaseType databaseType, final Map<String, Map<String, DataSource>> dataSources,
@@ -83,7 +78,6 @@ public final class SchemaContextsBuilder {
         this.ruleConfigs = ruleConfigs;
         this.authentication = authentication;
         this.props = new ConfigurationProperties(null == props ? new Properties() : props);
-        sqlParserEngine = ShardingSphereSQLParserEngineFactory.getSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(databaseType));
         executorKernel = new ExecutorKernel(this.props.<Integer>getValue(ConfigurationPropertyKey.EXECUTOR_SIZE));
     }
     
@@ -98,7 +92,7 @@ public final class SchemaContextsBuilder {
         for (String each : ruleConfigs.keySet()) {
             schemas.put(each, createSchema(each));
         }
-        return new StandardSchemaContexts(schemas, sqlParserEngine, executorKernel, authentication, props, databaseType);
+        return new StandardSchemaContexts(schemas, executorKernel, authentication, props, databaseType);
     }
     
     private ShardingSphereSchema createSchema(final String schemaName) throws SQLException {
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/impl/StandardSchemaContexts.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/impl/StandardSchemaContexts.java
index 5f050da..39e2d17 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/impl/StandardSchemaContexts.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/impl/StandardSchemaContexts.java
@@ -27,7 +27,6 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -42,8 +41,6 @@ public final class StandardSchemaContexts implements SchemaContexts {
     
     private final Map<String, ShardingSphereSchema> schemas;
     
-    private final ShardingSphereSQLParserEngine sqlParserEngine;
-    
     private final ExecutorKernel executorKernel;
     
     private final Authentication authentication;
@@ -56,12 +53,12 @@ public final class StandardSchemaContexts implements SchemaContexts {
     
     public StandardSchemaContexts() {
         // TODO MySQLDatabaseType is invalid because it can not update again
-        this(new HashMap<>(), null, null, new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType(), false);
+        this(new HashMap<>(), null, new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType(), false);
     }
     
-    public StandardSchemaContexts(final Map<String, ShardingSphereSchema> schemas, final ShardingSphereSQLParserEngine sqlParserEngine, final ExecutorKernel executorKernel, 
+    public StandardSchemaContexts(final Map<String, ShardingSphereSchema> schemas, final ExecutorKernel executorKernel, 
                                   final Authentication authentication, final ConfigurationProperties props, final DatabaseType databaseType) {
-        this(schemas, sqlParserEngine, executorKernel, authentication, props, databaseType, false);
+        this(schemas, executorKernel, authentication, props, databaseType, false);
     }
     
     @Override
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/impl/StandardSchemaContextsTest.java b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/impl/StandardSchemaContextsTest.java
index b5c74b6..eb64635 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/impl/StandardSchemaContextsTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/impl/StandardSchemaContextsTest.java
@@ -19,10 +19,9 @@ package org.apache.shardingsphere.infra.context.schema.impl;
 
 import org.apache.shardingsphere.infra.auth.Authentication;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.junit.Test;
 
 import java.util.Collections;
@@ -45,11 +44,10 @@ public final class StandardSchemaContextsTest {
     
     @Test
     public void assertClose() {
-        ShardingSphereSQLParserEngine sqlParserEngine = mock(ShardingSphereSQLParserEngine.class);
         ExecutorKernel executorKernel = mock(ExecutorKernel.class);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
         StandardSchemaContexts standardSchemaContexts = new StandardSchemaContexts(
-                Collections.singletonMap("logic_db", schema), sqlParserEngine, executorKernel, new Authentication(), new ConfigurationProperties(new Properties()), mock(DatabaseType.class));
+                Collections.singletonMap("logic_db", schema), executorKernel, new Authentication(), new ConfigurationProperties(new Properties()), DatabaseTypes.getTrunkDatabaseType("SQL92"));
         standardSchemaContexts.close();
         verify(executorKernel).close();
     }
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 8606c02..ec968c1 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
@@ -28,9 +28,16 @@ import org.apache.shardingsphere.driver.jdbc.core.constant.SQLExceptionConstant;
 import org.apache.shardingsphere.driver.jdbc.core.resultset.GeneratedKeysResultSet;
 import org.apache.shardingsphere.driver.jdbc.core.resultset.ShardingSphereResultSet;
 import org.apache.shardingsphere.driver.jdbc.core.statement.metadata.ShardingSphereParameterMetaData;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
+import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.kernel.InputGroup;
 import org.apache.shardingsphere.infra.executor.sql.ExecutorConstant;
@@ -48,15 +55,10 @@ import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.group.Stateme
 import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.queryresult.StreamQueryResult;
 import org.apache.shardingsphere.infra.merge.MergeEngine;
 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.infra.binder.LogicSQL;
-import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
-import org.apache.shardingsphere.infra.metadata.model.physical.model.schema.PhysicalSchemaMetaData;
-import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
 
@@ -136,7 +138,8 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         this.sql = sql;
         statements = new ArrayList<>();
         parameterSets = new ArrayList<>();
-        sqlStatement = schemaContexts.getSqlParserEngine().parse(sql, true);
+        ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(schemaContexts.getDatabaseType()));
+        sqlStatement = sqlParserEngine.parse(sql, true);
         parameterMetaData = new ShardingSphereParameterMetaData(sqlStatement);
         statementOption = returnGeneratedKeys ? new StatementOption(true) : new StatementOption(resultSetType, resultSetConcurrency, resultSetHoldability);
         SQLExecutor sqlExecutor = new SQLExecutor(schemaContexts.getExecutorKernel(), connection.isHoldTransaction());
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 997ebbc..15789ff 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
@@ -25,10 +25,17 @@ import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConne
 import org.apache.shardingsphere.driver.jdbc.core.constant.SQLExceptionConstant;
 import org.apache.shardingsphere.driver.jdbc.core.resultset.GeneratedKeysResultSet;
 import org.apache.shardingsphere.driver.jdbc.core.resultset.ShardingSphereResultSet;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
+import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.kernel.InputGroup;
 import org.apache.shardingsphere.infra.executor.sql.ExecutorConstant;
@@ -46,15 +53,10 @@ import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.group.Stateme
 import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.queryresult.StreamQueryResult;
 import org.apache.shardingsphere.infra.merge.MergeEngine;
 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.infra.binder.LogicSQL;
-import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
-import org.apache.shardingsphere.infra.metadata.model.physical.model.schema.PhysicalSchemaMetaData;
-import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
 
@@ -304,7 +306,8 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     
     private LogicSQL createLogicSQL(final String sql) {
         PhysicalSchemaMetaData schemaMetaData = schemaContexts.getDefaultSchema().getMetaData().getSchemaMetaData().getSchemaMetaData();
-        SQLStatement sqlStatement = schemaContexts.getSqlParserEngine().parse(sql, false);
+        ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(schemaContexts.getDatabaseType()));
+        SQLStatement sqlStatement = sqlParserEngine.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 074bfcd..2d27657 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.rdl.parser.engine.ShardingSphereSQLParserEngineFactory;
+import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.rdl.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 = ShardingSphereSQLParserEngineFactory.getSQLParserEngine(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 983323c..b5b7148 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
@@ -18,11 +18,15 @@
 package org.apache.shardingsphere.proxy.backend.text.sctl.explain;
 
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.raw.execute.result.query.QueryHeader;
+import org.apache.shardingsphere.infra.metadata.model.physical.model.schema.PhysicalSchemaMetaData;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.NoDatabaseSelectedException;
@@ -32,9 +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.infra.binder.SQLStatementContextFactory;
-import org.apache.shardingsphere.infra.metadata.model.physical.model.schema.PhysicalSchemaMetaData;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.Types;
@@ -83,7 +85,8 @@ public final class ShardingCTLExplainBackendHandler implements TextProtocolBacke
     
     private LogicSQL createLogicSQL(final ShardingSphereSchema schema, final ShardingCTLExplainStatement explainStatement) {
         PhysicalSchemaMetaData schemaMetaData = schema.getMetaData().getSchemaMetaData().getSchemaMetaData();
-        SQLStatement sqlStatement = ProxyContext.getInstance().getSchemaContexts().getSqlParserEngine().parse(explainStatement.getSql(), false);
+        ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
+        SQLStatement sqlStatement = sqlParserEngine.parse(explainStatement.getSql(), false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, Collections.emptyList(), sqlStatement);
         return new LogicSQL(sqlStatementContext, explainStatement.getSql(), Collections.emptyList());
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/DatabaseCommunicationEngineFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/DatabaseCommunicationEngineFactoryTest.java
index 23ed856..afe4208 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/DatabaseCommunicationEngineFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/DatabaseCommunicationEngineFactoryTest.java
@@ -27,7 +27,6 @@ import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicati
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngineFactory;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,8 +49,8 @@ public final class DatabaseCommunicationEngineFactoryTest {
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
         BackendConnection backendConnection = mock(BackendConnection.class, RETURNS_DEEP_STUBS);
         when(backendConnection.getSchemaName()).thenReturn("schema");
         when(backendConnection.isSerialExecute()).thenReturn(true);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnectionTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnectionTest.java
index e332c4d..a5b6517 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnectionTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/BackendConnectionTest.java
@@ -30,7 +30,6 @@ import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCBackendDataSource;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.BackendTransactionManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 import org.apache.shardingsphere.transaction.core.TransactionType;
@@ -101,8 +100,8 @@ public final class BackendConnectionTest {
     private void setSchemaContexts() throws ReflectiveOperationException {
         Field field = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         field.setAccessible(true);
-        field.set(ProxyContext.getInstance(), new StandardSchemaContexts(createSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        field.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(createSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     private Map<String, ShardingSphereSchema> createSchemas() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
index 5ad0e29..10f195e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/datasource/JDBCBackendDataSourceTest.java
@@ -28,7 +28,6 @@ import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.fixture.CallTimeRecordDataSource;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 import org.junit.Before;
@@ -74,8 +73,8 @@ public final class JDBCBackendDataSourceTest {
     private void setSchemaContexts() {
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(createSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(createSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     private Map<String, ShardingSphereSchema> createSchemas() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
index 29cad21..40766b7 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/context/ProxyContextTest.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.jdbc.test.MockedDataSource;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 import org.junit.Test;
 
@@ -59,8 +58,8 @@ public final class ProxyContextTest {
         mockDataSourceMap.put("ds_2", new MockedDataSource());
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(mockSchemas(mockDataSourceMap),
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(mockSchemas(mockDataSourceMap), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
         Optional<DataSource> actual = ProxyContext.getInstance().getDataSourceSample();
         assertThat(actual, is(Optional.of(mockDataSourceMap.get("ds_1"))));
     }
@@ -80,8 +79,8 @@ public final class ProxyContextTest {
         Map<String, ShardingSphereSchema> schemas = mockSchemas(Collections.emptyMap());
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(schemas, 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(schemas, mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
         assertTrue(ProxyContext.getInstance().schemaExists("schema"));
         assertFalse(ProxyContext.getInstance().schemaExists("schema_2"));
     }
@@ -91,8 +90,8 @@ public final class ProxyContextTest {
         Map<String, ShardingSphereSchema> schemas = mockSchemas(Collections.emptyMap());
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(schemas, 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(),
+                new StandardSchemaContexts(schemas, mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
         assertNull(ProxyContext.getInstance().getSchema(null));
         assertNull(ProxyContext.getInstance().getSchema(""));
         assertNull(ProxyContext.getInstance().getSchema("schema1"));
@@ -104,8 +103,8 @@ public final class ProxyContextTest {
         Map<String, ShardingSphereSchema> schemas = createSchemas();
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(schemas, 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(schemas, mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
         assertThat(new LinkedHashSet<>(ProxyContext.getInstance().getAllSchemaNames()), is(schemas.keySet()));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/BroadcastBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/BroadcastBackendHandlerTest.java
index 12d083f..7e7d1bb 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/BroadcastBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/BroadcastBackendHandlerTest.java
@@ -30,7 +30,6 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
 import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Before;
 import org.junit.Test;
@@ -73,8 +72,8 @@ public final class BroadcastBackendHandlerTest {
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
         when(backendConnection.getSchemaName()).thenReturn(String.format(SCHEMA_PATTERN, 0));
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/RDLBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/RDLBackendHandlerTest.java
index 754f116..fe81ca9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/RDLBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/RDLBackendHandlerTest.java
@@ -30,7 +30,6 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.DBCreateExistsException;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
 import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.rdl.parser.statement.rdl.CreateDataSourcesStatement;
 import org.apache.shardingsphere.rdl.parser.statement.rdl.CreateShardingRuleStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateDatabaseStatement;
@@ -62,8 +61,8 @@ public final class RDLBackendHandlerTest {
     public void setUp() throws IllegalAccessException, NoSuchFieldException {
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowDatabasesBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowDatabasesBackendHandlerTest.java
index b071744..1615357 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowDatabasesBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowDatabasesBackendHandlerTest.java
@@ -28,7 +28,6 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -62,8 +61,8 @@ public final class ShowDatabasesBackendHandlerTest {
         showDatabasesBackendHandler = new ShowDatabasesBackendHandler(backendConnection);
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(
+                getSchemas(), mock(ExecutorKernel.class), getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     private Map<String, ShardingSphereSchema> getSchemas() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandlerTest.java
index 083f294..19851f09 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandlerTest.java
@@ -28,7 +28,6 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Before;
 import org.junit.Test;
@@ -63,8 +62,8 @@ public final class ShowTablesBackendHandlerTest {
         when(backendConnection.getSchemaName()).thenReturn(String.format(SCHEMA_PATTERN, 0));
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(schemas, 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(schemas, mock(ExecutorKernel.class), getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     private Map<String, ShardingSphereSchema> getSchemas() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/UnicastBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/UnicastBackendHandlerTest.java
index 64d0bd0..6c27a54 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/UnicastBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/UnicastBackendHandlerTest.java
@@ -30,7 +30,6 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
 import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.transaction.core.TransactionType;
 import org.junit.Before;
@@ -68,8 +67,8 @@ public final class UnicastBackendHandlerTest {
     public void setUp() throws SQLException, IllegalAccessException, NoSuchFieldException {
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
         setUnderlyingHandler(new UpdateResponse());
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/UseDatabaseBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/UseDatabaseBackendHandlerTest.java
index 056d4af..cfe81f9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/UseDatabaseBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/admin/UseDatabaseBackendHandlerTest.java
@@ -29,7 +29,6 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.UnknownDatabaseException;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
 import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLUseStatement;
 import org.junit.Before;
 import org.junit.Test;
@@ -62,8 +61,8 @@ public final class UseDatabaseBackendHandlerTest {
         when(backendConnection.getUsername()).thenReturn("root");
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     private Map<String, ShardingSphereSchema> getSchemas() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandlerTest.java
index 21bb991..c97f616 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/explain/ShardingCTLExplainBackendHandlerTest.java
@@ -10,7 +10,6 @@ import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -55,8 +54,8 @@ public final class ShardingCTLExplainBackendHandlerTest {
         handler = new ShardingCTLExplainBackendHandler("sctl:explain select 1", connection);
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     private Map<String, ShardingSphereSchema> getSchemas() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
index c27e82d..46e6572 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
@@ -43,7 +43,6 @@ import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
 import org.apache.shardingsphere.proxy.backend.text.sctl.exception.InvalidShardingCTLFormatException;
 import org.apache.shardingsphere.proxy.backend.text.sctl.exception.UnsupportedShardingCTLTypeException;
 import org.apache.shardingsphere.proxy.backend.text.sctl.hint.internal.HintManagerHolder;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 import org.junit.Before;
 import org.junit.Test;
@@ -175,8 +174,8 @@ public final class ShardingCTLHintBackendHandlerTest {
         schemaContexts.setAccessible(true);
         Properties props = new Properties();
         props.setProperty(ConfigurationPropertyKey.PROXY_HINT_ENABLED.getKey(), Boolean.TRUE.toString());
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(props), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(props), new MySQLDatabaseType()));
         String sql = "sctl:hint show table status";
         ShardingCTLHintBackendHandler defaultHintBackendHandler = new ShardingCTLHintBackendHandler(sql, backendConnection);
         BackendResponse backendResponse = defaultHintBackendHandler.execute();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/set/ShardingCTLSetBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/set/ShardingCTLSetBackendHandlerTest.java
index 184ee1f..5e1b003 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/set/ShardingCTLSetBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/set/ShardingCTLSetBackendHandlerTest.java
@@ -29,7 +29,6 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
 import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
 import org.apache.shardingsphere.proxy.backend.text.sctl.exception.InvalidShardingCTLFormatException;
 import org.apache.shardingsphere.proxy.backend.text.sctl.exception.UnsupportedShardingCTLTypeException;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.transaction.core.TransactionType;
 import org.junit.Before;
 import org.junit.Test;
@@ -54,8 +53,8 @@ public final class ShardingCTLSetBackendHandlerTest {
     public void setUp() throws NoSuchFieldException, IllegalAccessException {
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     private Map<String, ShardingSphereSchema> getSchemas() {
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 9150912..671d00e 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
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.MySQLPacket;
 import org.apache.shardingsphere.db.protocol.mysql.packet.command.query.binary.execute.MySQLBinaryResultSetRowPacket;
 import org.apache.shardingsphere.db.protocol.mysql.packet.command.query.binary.execute.MySQLComStmtExecutePacket;
 import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngineFactory;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -56,7 +57,7 @@ public final class MySQLComStmtExecuteExecutor implements QueryCommandExecutor {
     private int currentSequenceId;
     
     public MySQLComStmtExecuteExecutor(final MySQLComStmtExecutePacket packet, final BackendConnection backendConnection) {
-        ShardingSphereSQLParserEngine sqlParserEngine = ProxyContext.getInstance().getSchemaContexts().getSqlParserEngine();
+        ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
         SQLStatement sqlStatement = sqlParserEngine.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 c8aed68..0bef4a8 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
@@ -25,9 +25,11 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.command.query.binary.p
 import org.apache.shardingsphere.db.protocol.mysql.packet.command.query.binary.prepare.MySQLComStmtPreparePacket;
 import org.apache.shardingsphere.db.protocol.mysql.packet.generic.MySQLEofPacket;
 import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 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.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 
@@ -48,7 +50,8 @@ public final class MySQLComStmtPrepareExecutor implements CommandExecutor {
     
     @Override
     public Collection<DatabasePacket<?>> execute() {
-        SQLStatement sqlStatement = ProxyContext.getInstance().getSchemaContexts().getSqlParserEngine().parse(packet.getSql(), true);
+        ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
+        SQLStatement sqlStatement = sqlParserEngine.parse(packet.getSql(), true);
         if (!MySQLComStmtPrepareChecker.isStatementAllowed(sqlStatement)) {
             throw new UnsupportedPreparedStatementException();
         }
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 9f958e1..0ce2ba3 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
@@ -22,11 +22,13 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.command.query.MySQLCol
 import org.apache.shardingsphere.db.protocol.mysql.packet.command.query.text.fieldlist.MySQLComFieldListPacket;
 import org.apache.shardingsphere.db.protocol.mysql.packet.generic.MySQLEofPacket;
 import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngineFactory;
 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.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.SQLException;
@@ -52,7 +54,8 @@ public final class MySQLComFieldListPacketExecutor implements CommandExecutor {
         this.packet = packet;
         schemaName = backendConnection.getSchemaName();
         String sql = String.format(SQL, packet.getTable(), schemaName);
-        SQLStatement sqlStatement = ProxyContext.getInstance().getSchemaContexts().getSqlParserEngine().parse(sql, false);
+        ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
+        SQLStatement sqlStatement = sqlParserEngine.parse(sql, false);
         databaseCommunicationEngine = DatabaseCommunicationEngineFactory.getInstance().newTextProtocolInstance(sqlStatement, sql, backendConnection);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
index c5b65be..d60b018 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/MySQLFrontendEngineTest.java
@@ -37,7 +37,6 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationResult;
 import org.apache.shardingsphere.proxy.frontend.connection.ConnectionIdGenerator;
 import org.apache.shardingsphere.proxy.frontend.mysql.auth.MySQLAuthenticationEngine;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -165,8 +164,7 @@ public final class MySQLFrontendEngineTest {
     }
     
     private SchemaContexts getSchemaContexts(final Authentication authentication) {
-        return new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), authentication, new ConfigurationProperties(new Properties()), new MySQLDatabaseType());
+        return new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), authentication, new ConfigurationProperties(new Properties()), new MySQLDatabaseType());
     }
     
     private Map<String, ShardingSphereSchema> getSchemas() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationEngineTest.java
index 9c1a08f..89ff673 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationEngineTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationEngineTest.java
@@ -35,7 +35,6 @@ import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationResultBuilder;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -142,7 +141,7 @@ public final class MySQLAuthenticationEngineTest {
         Field field = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         field.setAccessible(true);
         field.set(ProxyContext.getInstance(), new StandardSchemaContexts(Collections.singletonMap("sharding_db", mock(ShardingSphereSchema.class)),
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+                mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     private MySQLPacketPayload getPayload(final String username, final String database, final byte[] authResponse) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationHandlerTest.java
index 387ebcb..25e371a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/auth/MySQLAuthenticationHandlerTest.java
@@ -30,7 +30,6 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -122,8 +121,7 @@ public final class MySQLAuthenticationHandlerTest {
     }
     
     private SchemaContexts getSchemaContexts(final Authentication authentication) {
-        return new StandardSchemaContexts(getSchemas(), 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), authentication, new ConfigurationProperties(new Properties()), new MySQLDatabaseType());
+        return new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), authentication, new ConfigurationProperties(new Properties()), new MySQLDatabaseType());
     }
     
     private Map<String, ShardingSphereSchema> getSchemas() {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
index f650d26..9422e71 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/MySQLCommandExecutorFactoryTest.java
@@ -44,7 +44,6 @@ import org.apache.shardingsphere.proxy.frontend.mysql.command.query.binary.prepa
 import org.apache.shardingsphere.proxy.frontend.mysql.command.query.binary.reset.MySQLComStmtResetExecutor;
 import org.apache.shardingsphere.proxy.frontend.mysql.command.query.text.fieldlist.MySQLComFieldListPacketExecutor;
 import org.apache.shardingsphere.proxy.frontend.mysql.command.query.text.query.MySQLComQueryPacketExecutor;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -70,12 +69,12 @@ public final class MySQLCommandExecutorFactoryTest {
     
     @Before
     public void setUp() throws ReflectiveOperationException {
-        when(backendConnection.getSchemaName()).thenReturn("schema");
+        when(backendConnection.getSchemaName()).thenReturn("logic_db");
         Field field = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         field.setAccessible(true);
-        Map<String, ShardingSphereSchema> schemas = Collections.singletonMap("schema", mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS));
-        field.set(ProxyContext.getInstance(), new StandardSchemaContexts(schemas, 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        Map<String, ShardingSphereSchema> schemas = Collections.singletonMap("logic_db", mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS));
+        field.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(schemas, mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     @Test
@@ -90,8 +89,9 @@ public final class MySQLCommandExecutorFactoryTest {
     
     @Test
     public void assertNewInstanceWithComFieldList() {
-        assertThat(MySQLCommandExecutorFactory.newInstance(MySQLCommandPacketType.COM_FIELD_LIST,
-                mock(MySQLComFieldListPacket.class), backendConnection), instanceOf(MySQLComFieldListPacketExecutor.class));
+        MySQLComFieldListPacket packet = mock(MySQLComFieldListPacket.class);
+        when(packet.getTable()).thenReturn("test");
+        assertThat(MySQLCommandExecutorFactory.newInstance(MySQLCommandPacketType.COM_FIELD_LIST, packet, backendConnection), instanceOf(MySQLComFieldListPacketExecutor.class));
     }
     
     @Test
@@ -106,14 +106,15 @@ public final class MySQLCommandExecutorFactoryTest {
     
     @Test
     public void assertNewInstanceWithComStmtPrepare() {
-        assertThat(MySQLCommandExecutorFactory.newInstance(MySQLCommandPacketType.COM_STMT_PREPARE,
-                mock(MySQLComStmtPreparePacket.class), backendConnection), instanceOf(MySQLComStmtPrepareExecutor.class));
+        assertThat(MySQLCommandExecutorFactory.newInstance(
+                MySQLCommandPacketType.COM_STMT_PREPARE, mock(MySQLComStmtPreparePacket.class), backendConnection), instanceOf(MySQLComStmtPrepareExecutor.class));
     }
     
     @Test
     public void assertNewInstanceWithComStmtExecute() {
-        assertThat(MySQLCommandExecutorFactory.newInstance(MySQLCommandPacketType.COM_STMT_EXECUTE,
-                mock(MySQLComStmtExecutePacket.class), backendConnection), instanceOf(MySQLComStmtExecuteExecutor.class));
+        MySQLComStmtExecutePacket packet = mock(MySQLComStmtExecutePacket.class);
+        when(packet.getSql()).thenReturn("SELECT 1");
+        assertThat(MySQLCommandExecutorFactory.newInstance(MySQLCommandPacketType.COM_STMT_EXECUTE, packet, backendConnection), instanceOf(MySQLComStmtExecuteExecutor.class));
     }
     
     @Test
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
index 6032352..95bb61f 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
@@ -31,7 +31,6 @@ import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 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.ResponseType;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -61,16 +60,18 @@ public final class MySQLComStmtExecuteExecutorTest {
     public void setUp() throws ReflectiveOperationException {
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        Map<String, ShardingSphereSchema> schemas = Collections.singletonMap("schema", mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS));
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(schemas, 
-                mock(ShardingSphereSQLParserEngine.class), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        Map<String, ShardingSphereSchema> schemas = Collections.singletonMap("logic_db", mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(schemas, mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
     }
     
     @Test
     public void assertIsQueryResponse() throws NoSuchFieldException, SQLException {
         BackendConnection backendConnection = mock(BackendConnection.class);
-        when(backendConnection.getSchemaName()).thenReturn("schema");
-        MySQLComStmtExecuteExecutor mysqlComStmtExecuteExecutor = new MySQLComStmtExecuteExecutor(mock(MySQLComStmtExecutePacket.class), backendConnection);
+        when(backendConnection.getSchemaName()).thenReturn("logic_db");
+        MySQLComStmtExecutePacket packet = mock(MySQLComStmtExecutePacket.class);
+        when(packet.getSql()).thenReturn("SELECT 1");
+        MySQLComStmtExecuteExecutor mysqlComStmtExecuteExecutor = new MySQLComStmtExecuteExecutor(packet, backendConnection);
         FieldSetter.setField(mysqlComStmtExecuteExecutor, MySQLComStmtExecuteExecutor.class.getDeclaredField("databaseCommunicationEngine"), databaseCommunicationEngine);
         when(databaseCommunicationEngine.execute()).thenReturn(new QueryResponse(Collections.singletonList(mock(QueryHeader.class))));
         mysqlComStmtExecuteExecutor.execute();
@@ -80,8 +81,10 @@ public final class MySQLComStmtExecuteExecutorTest {
     @Test
     public void assertIsUpdateResponse() throws NoSuchFieldException, SQLException {
         BackendConnection backendConnection = mock(BackendConnection.class);
-        when(backendConnection.getSchemaName()).thenReturn("schema");
-        MySQLComStmtExecuteExecutor mysqlComStmtExecuteExecutor = new MySQLComStmtExecuteExecutor(mock(MySQLComStmtExecutePacket.class), backendConnection);
+        when(backendConnection.getSchemaName()).thenReturn("logic_db");
+        MySQLComStmtExecutePacket packet = mock(MySQLComStmtExecutePacket.class);
+        when(packet.getSql()).thenReturn("SELECT 1");
+        MySQLComStmtExecuteExecutor mysqlComStmtExecuteExecutor = new MySQLComStmtExecuteExecutor(packet, backendConnection);
         FieldSetter.setField(mysqlComStmtExecuteExecutor, MySQLComStmtExecuteExecutor.class.getDeclaredField("databaseCommunicationEngine"), databaseCommunicationEngine);
         when(databaseCommunicationEngine.execute()).thenReturn(new UpdateResponse());
         mysqlComStmtExecuteExecutor.execute();
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 ee93d64..3faefb5 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
@@ -29,6 +29,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.bin
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.text.PostgreSQLDataRowPacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.generic.PostgreSQLCommandCompletePacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.generic.PostgreSQLErrorResponsePacket;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.executor.sql.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.raw.execute.result.query.QueryHeader;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
@@ -42,6 +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.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.sql.ResultSetMetaData;
@@ -69,7 +71,8 @@ public final class PostgreSQLComBindExecutor implements QueryCommandExecutor {
         this.packet = packet;
         ShardingSphereSchema schema = ProxyContext.getInstance().getSchema(backendConnection.getSchemaName());
         if (null != packet.getSql() && null != schema) {
-            SQLStatement sqlStatement = ProxyContext.getInstance().getSchemaContexts().getSqlParserEngine().parse(packet.getSql(), true);
+            ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
+            SQLStatement sqlStatement = sqlParserEngine.parse(packet.getSql(), true);
             databaseCommunicationEngine =
                     DatabaseCommunicationEngineFactory.getInstance().newBinaryProtocolInstance(sqlStatement, packet.getSql(), packet.getParameters(), backendConnection);
         } else {
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 5053463..9de4a3f 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
@@ -22,9 +22,11 @@ import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.bin
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.ConnectionScopeBinaryStatementRegistry;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.parse.PostgreSQLComParsePacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.parse.PostgreSQLParseCompletePacket;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 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.rdl.parser.engine.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Collection;
@@ -47,7 +49,8 @@ public final class PostgreSQLComParseExecutor implements CommandExecutor {
     @Override
     public Collection<DatabasePacket<?>> execute() {
         if (!packet.getSql().isEmpty()) {
-            SQLStatement sqlStatement = ProxyContext.getInstance().getSchemaContexts().getSqlParserEngine().parse(packet.getSql(), true);
+            ShardingSphereSQLParserEngine sqlParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(ProxyContext.getInstance().getSchemaContexts().getDatabaseType()));
+            SQLStatement sqlStatement = sqlParserEngine.parse(packet.getSql(), true);
             binaryStatementRegistry.register(packet.getStatementId(), packet.getSql(), sqlStatement.getParameterCount(), packet.getBinaryStatementParameterTypes());
         }
         return Collections.singletonList(new PostgreSQLParseCompletePacket());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutorTest.java
index 790e141..977c640 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/parse/PostgreSQLComParseExecutorTest.java
@@ -28,8 +28,6 @@ import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
@@ -42,7 +40,6 @@ import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.assertThat;
-import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -57,13 +54,13 @@ public final class PostgreSQLComParseExecutorTest {
     
     @Test
     public void assertNewInstance() throws NoSuchFieldException, IllegalAccessException {
-        when(parsePacket.getSql()).thenReturn("sql");
+        when(parsePacket.getSql()).thenReturn("SELECT 1");
         when(parsePacket.getStatementId()).thenReturn("2");
         when(backendConnection.getConnectionId()).thenReturn(1);
         Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
         schemaContexts.setAccessible(true);
-        schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts(getSchemas(),
-                mockSQLParserEngine(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
+        schemaContexts.set(ProxyContext.getInstance(), 
+                new StandardSchemaContexts(getSchemas(), mock(ExecutorKernel.class), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
         BinaryStatementRegistry.getInstance().register(1);
         PostgreSQLComParseExecutor actual = new PostgreSQLComParseExecutor(parsePacket, backendConnection);
         assertThat(actual.execute().iterator().next(), instanceOf(PostgreSQLParseCompletePacket.class));
@@ -74,14 +71,6 @@ public final class PostgreSQLComParseExecutorTest {
         return Collections.singletonMap("schema", schema);
     }
     
-    private ShardingSphereSQLParserEngine mockSQLParserEngine() {
-        ShardingSphereSQLParserEngine result = mock(ShardingSphereSQLParserEngine.class);
-        SQLStatement sqlStatement = mock(SQLStatement.class);
-        when(result.parse(eq("sql"), eq(true))).thenReturn(sqlStatement);
-        when(sqlStatement.getParameterCount()).thenReturn(1);
-        return result;
-    }
-    
     @Test
     public void assertGetSqlWithNull() {
         when(parsePacket.getSql()).thenReturn("");
diff --git a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-engine/src/main/java/org/apache/shardingsphere/rdl/parser/engine/ShardingSphereSQLParserEngine.java b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-engine/src/main/java/org/apache/shardingsphere/rdl/parser/engine/ShardingSphereSQLParserEngine.java
index 799d028..f6ec76d 100644
--- a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-engine/src/main/java/org/apache/shardingsphere/rdl/parser/engine/ShardingSphereSQLParserEngine.java
+++ b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-engine/src/main/java/org/apache/shardingsphere/rdl/parser/engine/ShardingSphereSQLParserEngine.java
@@ -17,8 +17,9 @@
 
 package org.apache.shardingsphere.rdl.parser.engine;
 
-import org.apache.shardingsphere.sql.parser.engine.SQLParserEngine;
 import org.apache.shardingsphere.rdl.parser.engine.engine.RDLSQLParserEngine;
+import org.apache.shardingsphere.sql.parser.engine.SQLParserEngine;
+import org.apache.shardingsphere.sql.parser.engine.SQLParserEngineFactory;
 import org.apache.shardingsphere.sql.parser.engine.StandardSQLParserEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
@@ -31,8 +32,8 @@ public final class ShardingSphereSQLParserEngine implements SQLParserEngine {
 
     private final RDLSQLParserEngine rdlsqlParserEngine;
     
-    public ShardingSphereSQLParserEngine(final StandardSQLParserEngine standardSqlParserEngine) {
-        this.standardSqlParserEngine = standardSqlParserEngine;
+    public ShardingSphereSQLParserEngine(final String databaseTypeName) {
+        standardSqlParserEngine = SQLParserEngineFactory.getSQLParserEngine(databaseTypeName);
         rdlsqlParserEngine = new RDLSQLParserEngine();
     }
     
diff --git a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-engine/src/main/java/org/apache/shardingsphere/rdl/parser/engine/ShardingSphereSQLParserEngineFactory.java b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-engine/src/main/java/org/apache/shardingsphere/rdl/parser/engine/ShardingSphereSQLParserEngineFactory.java
deleted file mode 100644
index 946d153..0000000
--- a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-engine/src/main/java/org/apache/shardingsphere/rdl/parser/engine/ShardingSphereSQLParserEngineFactory.java
+++ /dev/null
@@ -1,41 +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.rdl.parser.engine;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.sql.parser.engine.StandardSQLParserEngine;
-import org.apache.shardingsphere.sql.parser.engine.SQLParserEngineFactory;
-
-/**
- * SQL parser engine factory.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ShardingSphereSQLParserEngineFactory {
-    
-    /**
-     * Get SQL parser engine.
-     *
-     * @param databaseTypeName name of database type
-     * @return SQL parser engine
-     */
-    public static ShardingSphereSQLParserEngine getSQLParserEngine(final String databaseTypeName) {
-        StandardSQLParserEngine standardSqlParserEngine = SQLParserEngineFactory.getSQLParserEngine(databaseTypeName);
-        return new ShardingSphereSQLParserEngine(standardSqlParserEngine);
-    }
-}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DALStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DALStatement.g4
index 3cd5d88..7c0ba6d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DALStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DALStatement.g4
@@ -47,7 +47,7 @@ showTableStatus
     ;
 
 showColumns
-    : SHOW EXTENDED? FULL? (COLUMNS | FIELDS) fromTable fromSchema?  (showColumnLike_ | showWhereClause_)?
+    : SHOW EXTENDED? FULL? (COLUMNS | FIELDS) fromTable fromSchema? (showColumnLike_ | showWhereClause_)?
     ;
 
 showIndex