You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2020/10/01 08:20:08 UTC

[shardingsphere] branch master updated: Move LogicSQL to infra module (#7677)

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

panjuan 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 8e36397  Move LogicSQL to infra module (#7677)
8e36397 is described below

commit 8e36397e3017dcf1c1cc87a293abaaa525d2cdf6
Author: Liang Zhang <te...@163.com>
AuthorDate: Thu Oct 1 16:19:47 2020 +0800

    Move LogicSQL to infra module (#7677)
    
    * Simplify SchemaContext
    
    * Fix test cases
    
    * Refactor AbstractStatementExecutor
    
    * Use ShardingSphereSchema in LogicSQLContext
    
    * Move ShardingSphereSchema to infra module
    
    * Move LogicSQLContext to infra module
    
    * Use LogicSQLContext on SQLLogger
    
    * Rename LogicSQLContext to LogicSQL
    
    * Use LogicSQL on SQLRouteEngine
---
 .../engine/ConsensusReplicationSQLRouter.java      | 16 +++----
 .../EncryptSQLRewriterParameterizedTest.java       |  8 +++-
 .../engine/PrimaryReplicaReplicationSQLRouter.java | 16 +++----
 .../PrimaryReplicaReplicationSQLRouterTest.java    | 34 +++++++++----
 .../shadow/route/engine/ShadowSQLRouter.java       | 18 ++++---
 .../shadow/route/engine/ShadowSQLRouterTest.java   | 43 ++++++++++-------
 .../MixSQLRewriterParameterizedTest.java           |  8 +++-
 .../ShardingSQLRewriterParameterizedTest.java      |  9 ++--
 .../sharding/route/engine/ShardingSQLRouter.java   | 23 +++++----
 .../engine/type/standard/AbstractSQLRouteTest.java |  6 ++-
 .../context/schema/GovernanceSchemaContexts.java   |  2 +-
 .../schema/GovernanceSchemaContextsTest.java       |  2 +-
 .../metadata}/schema/ShardingSphereSchema.java     | 11 ++++-
 .../apache/shardingsphere/infra/sql/LogicSQL.java} | 10 ++--
 .../fixture/datasource}/CloseableDataSource.java   |  2 +-
 .../metadata}/schema/ShardingSphereSchemaTest.java | 29 +++++++++++-
 .../infra/context/kernel/KernelProcessor.java      | 20 ++++----
 .../infra/context/schema/SchemaContext.java        | 17 +------
 .../context/schema/SchemaContextsBuilder.java      |  1 +
 .../schema/impl/StandardSchemaContexts.java        |  2 +-
 .../infra/context/schema/SchemaContextTest.java    | 55 ----------------------
 .../schema/impl/StandardSchemaContextsTest.java    |  2 +-
 .../infra/executor/sql/log/SQLLogger.java          |  9 ++--
 .../infra/executor/sql/log/SQLLoggerTest.java      | 11 +++--
 .../shardingsphere/infra/route/SQLRouter.java      | 17 ++-----
 .../infra/route/UnconfiguredSchemaSQLRouter.java   | 12 ++---
 .../infra/route/engine/SQLRouteEngine.java         | 29 +++++-------
 .../infra/route/engine/SQLRouteEngineTest.java     | 14 ++++--
 .../fixture/router/SQLRouterFailureFixture.java    | 13 ++---
 .../route/fixture/router/SQLRouterFixture.java     | 12 ++---
 .../driver/executor/AbstractStatementExecutor.java | 10 ++--
 .../driver/executor/PreparedStatementExecutor.java |  4 +-
 .../driver/executor/StatementExecutor.java         |  8 ++--
 .../statement/ShardingSpherePreparedStatement.java | 15 +++---
 .../core/statement/ShardingSphereStatement.java    | 15 +++---
 .../connection/ShardingSphereConnectionTest.java   |  2 +-
 .../DatabaseCommunicationEngineFactory.java        | 14 +++---
 .../jdbc/JDBCDatabaseCommunicationEngine.java      | 26 +++++-----
 .../text/admin/BroadcastBackendHandler.java        |  2 +-
 .../text/admin/ShowTablesBackendHandler.java       |  2 +-
 .../backend/text/admin/UnicastBackendHandler.java  |  2 +-
 .../backend/text/query/QueryBackendHandler.java    |  2 +-
 .../explain/ShardingCTLExplainBackendHandler.java  | 12 ++---
 .../executor/HintShowTableStatusExecutor.java      |  2 +-
 .../jdbc/connection/BackendConnectionTest.java     |  2 +-
 .../response/query/QueryHeaderBuilderTest.java     |  2 +-
 .../text/admin/BroadcastBackendHandlerTest.java    |  5 +-
 .../text/admin/ShowTablesBackendHandlerTest.java   |  5 +-
 .../ShardingCTLExplainBackendHandlerTest.java      |  2 +-
 .../hint/ShardingCTLHintBackendHandlerTest.java    |  7 +--
 .../frontend/mysql/MySQLFrontendEngineTest.java    |  2 +-
 .../mysql/auth/MySQLAuthenticationHandlerTest.java |  2 +-
 .../parse/PostgreSQLComParseExecutorTest.java      |  2 +-
 53 files changed, 289 insertions(+), 307 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-route/src/main/java/org/apache/shardingsphere/replication/consensus/route/engine/ConsensusReplicationSQLRouter.java b/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-route/src/main/java/org/apache/shardingsphere/replication/consensus/route/engine/ConsensusReplicationSQLRouter.java
index 9839c06..5493d80 100644
--- a/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-route/src/main/java/org/apache/shardingsphere/replication/consensus/route/engine/ConsensusReplicationSQLRouter.java
+++ b/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-route/src/main/java/org/apache/shardingsphere/replication/consensus/route/engine/ConsensusReplicationSQLRouter.java
@@ -18,20 +18,18 @@
 package org.apache.shardingsphere.replication.consensus.route.engine;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.replication.consensus.constant.ConsensusReplicationOrder;
 import org.apache.shardingsphere.replication.consensus.rule.ConsensusReplicationRule;
 import org.apache.shardingsphere.replication.consensus.rule.ConsensusReplicationTableRule;
-import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
@@ -41,20 +39,19 @@ import java.util.Optional;
 public final class ConsensusReplicationSQLRouter implements SQLRouter<ConsensusReplicationRule> {
     
     @Override
-    public RouteContext createRouteContext(final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                           final ShardingSphereMetaData metaData, final ConsensusReplicationRule rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ConsensusReplicationRule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
         ConsensusReplicationTableRule replicaRoutingRule = rule.getReplicaTableRules().iterator().next();
         ConsensusReplicationGroup replicaGroup = new ConsensusReplicationGroup(replicaRoutingRule.getPhysicsTable(), replicaRoutingRule.getReplicaGroupId(), replicaRoutingRule.getReplicaPeers(),
                 replicaRoutingRule.getDataSourceName());
         Map<String, ConsensusReplicationGroup> replicaGroups = Collections.singletonMap(ConsensusReplicationGroup.BLANK_CONSENSUS_REPLICATION_GROUP_KEY, replicaGroup);
-        result.getRouteStageContexts().put(getTypeClass(), new ConsensusReplicationRouteStageContext(metaData.getSchemaName(), replicaGroups, sqlStatementContext.isReadOnly()));
+        result.getRouteStageContexts().put(getTypeClass(), 
+                new ConsensusReplicationRouteStageContext(logicSQL.getSchema().getMetaData().getSchemaName(), replicaGroups, logicSQL.getSqlStatementContext().isReadOnly()));
         return result;
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                     final ShardingSphereMetaData metaData, final ConsensusReplicationRule rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final ConsensusReplicationRule rule, final ConfigurationProperties props) {
         Map<String, ConsensusReplicationGroup> replicaGroups = new HashMap<>();
         for (RouteUnit each : routeContext.getRouteUnits()) {
             Collection<RouteMapper> routeMappers = each.getTableMappers();
@@ -67,7 +64,8 @@ public final class ConsensusReplicationSQLRouter implements SQLRouter<ConsensusR
                 routeReplicaGroups(routeMappers, rule, replicaGroups);
             }
         }
-        routeContext.getRouteStageContexts().put(getTypeClass(), new ConsensusReplicationRouteStageContext(metaData.getSchemaName(), replicaGroups, sqlStatementContext.isReadOnly()));
+        routeContext.getRouteStageContexts().put(getTypeClass(), 
+                new ConsensusReplicationRouteStageContext(logicSQL.getSchema().getMetaData().getSchemaName(), replicaGroups, logicSQL.getSqlStatementContext().isReadOnly()));
     }
     
     private void routeReplicaGroups(final Collection<RouteMapper> routeMappers, final ConsensusReplicationRule rule, final Map<String, ConsensusReplicationGroup> replicaGroups) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
index 36c4c10..d527f8e 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.datasource.DataSourceMetaDatas;
 import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
 import org.apache.shardingsphere.infra.rewrite.engine.result.RouteSQLRewriteResult;
@@ -30,10 +31,11 @@ import org.apache.shardingsphere.infra.rewrite.engine.result.SQLRewriteUnit;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.AbstractSQLRewriterParameterizedTest;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.parameter.SQLRewriteEngineTestParameters;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.parameter.SQLRewriteEngineTestParametersBuilder;
-import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRulesBuilder;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -74,10 +76,12 @@ public final class EncryptSQLRewriterParameterizedTest extends AbstractSQLRewrit
                 new YamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(ruleConfigurations.getRules()), ruleConfigurations.getDataSources().keySet());
         StandardSQLParserEngine standardSqlParserEngine = SQLParserEngineFactory.getSQLParserEngine(null == getTestParameters().getDatabaseType() ? "SQL92" : getTestParameters().getDatabaseType());
         ShardingSphereMetaData metaData = createShardingSphereMetaData();
+        ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), rules, Collections.emptyMap(), metaData);
         ConfigurationProperties props = new ConfigurationProperties(ruleConfigurations.getProps());
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
                 metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), getTestParameters().getInputParameters(), standardSqlParserEngine.parse(getTestParameters().getInputSQL(), false));
-        RouteContext routeContext = new SQLRouteEngine(metaData, props, rules).route(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
+        RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL);
         SQLRewriteResult sqlRewriteResult = new SQLRewriteEntry(metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(),
                 props, rules).rewrite(getTestParameters().getInputSQL(), getTestParameters().getInputParameters(), sqlStatementContext, routeContext);
         return sqlRewriteResult instanceof GenericSQLRewriteResult
diff --git a/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/main/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationSQLRouter.java b/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/main/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationSQLRouter.java
index 883ffac..afeeb88 100644
--- a/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/main/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationSQLRouter.java
+++ b/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/main/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationSQLRouter.java
@@ -19,21 +19,19 @@ package org.apache.shardingsphere.replication.primaryreplica.route.engine;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.DefaultSchema;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.route.SQLRouter;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.replication.primaryreplica.constant.PrimaryReplicaReplicationOrder;
 import org.apache.shardingsphere.replication.primaryreplica.route.engine.impl.PrimaryReplicaReplicationDataSourceRouter;
 import org.apache.shardingsphere.replication.primaryreplica.rule.PrimaryReplicaReplicationDataSourceRule;
 import org.apache.shardingsphere.replication.primaryreplica.rule.PrimaryReplicaReplicationRule;
-import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
-import java.util.List;
 import java.util.Optional;
 
 /**
@@ -42,17 +40,15 @@ import java.util.Optional;
 public final class PrimaryReplicaReplicationSQLRouter implements SQLRouter<PrimaryReplicaReplicationRule> {
     
     @Override
-    public RouteContext createRouteContext(final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                           final ShardingSphereMetaData metaData, final PrimaryReplicaReplicationRule rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final PrimaryReplicaReplicationRule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
-        String dataSourceName = new PrimaryReplicaReplicationDataSourceRouter(rule.getSingleDataSourceRule()).route(sqlStatementContext.getSqlStatement());
+        String dataSourceName = new PrimaryReplicaReplicationDataSourceRouter(rule.getSingleDataSourceRule()).route(logicSQL.getSqlStatementContext().getSqlStatement());
         result.getRouteUnits().add(new RouteUnit(new RouteMapper(DefaultSchema.LOGIC_NAME, dataSourceName), Collections.emptyList()));
         return result;
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                     final ShardingSphereMetaData metaData, final PrimaryReplicaReplicationRule rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final PrimaryReplicaReplicationRule rule, final ConfigurationProperties props) {
         Collection<RouteUnit> toBeRemoved = new LinkedList<>();
         Collection<RouteUnit> toBeAdded = new LinkedList<>();
         for (RouteUnit each : routeContext.getRouteUnits()) {
@@ -60,7 +56,7 @@ public final class PrimaryReplicaReplicationSQLRouter implements SQLRouter<Prima
             Optional<PrimaryReplicaReplicationDataSourceRule> dataSourceRule = rule.findDataSourceRule(dataSourceName);
             if (dataSourceRule.isPresent() && dataSourceRule.get().getName().equalsIgnoreCase(each.getDataSourceMapper().getActualName())) {
                 toBeRemoved.add(each);
-                String actualDataSourceName = new PrimaryReplicaReplicationDataSourceRouter(dataSourceRule.get()).route(sqlStatementContext.getSqlStatement());
+                String actualDataSourceName = new PrimaryReplicaReplicationDataSourceRouter(dataSourceRule.get()).route(logicSQL.getSqlStatementContext().getSqlStatement());
                 toBeAdded.add(new RouteUnit(new RouteMapper(each.getDataSourceMapper().getLogicName(), actualDataSourceName), each.getTableMappers()));
             }
         }
diff --git a/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/test/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationSQLRouterTest.java b/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/test/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationSQLRouterTest.java
index 5ff091c..e10f764 100644
--- a/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/test/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationSQLRouterTest.java
+++ b/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/test/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationSQLRouterTest.java
@@ -19,12 +19,14 @@ package org.apache.shardingsphere.replication.primaryreplica.route.engine;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.order.OrderedSPIRegistry;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.replication.primaryreplica.api.config.PrimaryReplicaReplicationRuleConfiguration;
 import org.apache.shardingsphere.replication.primaryreplica.api.config.rule.PrimaryReplicaReplicationDataSourceRuleConfiguration;
 import org.apache.shardingsphere.replication.primaryreplica.route.engine.impl.PrimaryVisitedManager;
@@ -87,8 +89,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
     
     @Test
     public void assertCreateRouteContextToPrimaryWithoutRouteUnits() {
-        RouteContext actual = sqlRouter.createRouteContext(mock(SQLStatementContext.class), 
-                Collections.emptyList(), mock(ShardingSphereMetaData.class), rule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mock(SQLStatementContext.class), "", Collections.emptyList());
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATASOURCE));
     }
@@ -96,7 +99,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
     @Test
     public void assertDecorateRouteContextToPrimaryDataSource() {
         RouteContext actual = mockRouteContext();
-        sqlRouter.decorateRouteContext(actual, mock(SQLStatementContext.class), Collections.emptyList(), mock(ShardingSphereMetaData.class), rule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mock(SQLStatementContext.class), "", Collections.emptyList());
+        sqlRouter.decorateRouteContext(actual, logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(NON_PRIMARY_REPLICA_DATASOURCE_NAME));
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATASOURCE));
@@ -107,7 +112,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
         MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.empty());
-        RouteContext actual = sqlRouter.createRouteContext(sqlStatementContext, Collections.emptyList(), mock(ShardingSphereMetaData.class), rule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, "", Collections.emptyList());
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(REPLICA_DATASOURCE));
     }
@@ -118,7 +125,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
         MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.empty());
-        sqlRouter.decorateRouteContext(actual, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereMetaData.class), rule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, "", Collections.emptyList());
+        sqlRouter.decorateRouteContext(actual, logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(NON_PRIMARY_REPLICA_DATASOURCE_NAME));
         assertThat(routedDataSourceNames.next(), is(REPLICA_DATASOURCE));
@@ -129,7 +138,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
         MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.of(mock(LockSegment.class)));
-        RouteContext actual = sqlRouter.createRouteContext(sqlStatementContext, Collections.emptyList(), mock(ShardingSphereMetaData.class), rule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, "", Collections.emptyList());
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATASOURCE));
     }
@@ -140,7 +151,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
         MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.of(mock(LockSegment.class)));
-        sqlRouter.decorateRouteContext(actual, sqlStatementContext, Collections.emptyList(), mock(ShardingSphereMetaData.class), rule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, "", Collections.emptyList());
+        sqlRouter.decorateRouteContext(actual, logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(NON_PRIMARY_REPLICA_DATASOURCE_NAME));
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATASOURCE));
@@ -149,8 +162,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
     @Test
     public void assertCreateRouteContextToPrimaryDataSource() {
         when(sqlStatementContext.getSqlStatement()).thenReturn(mock(InsertStatement.class));
-        RouteContext actual = sqlRouter.createRouteContext(
-                sqlStatementContext, Collections.singletonList("true"), mock(ShardingSphereMetaData.class), rule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, "", Collections.emptyList());
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATASOURCE));
     }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/main/java/org/apache/shardingsphere/shadow/route/engine/ShadowSQLRouter.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/main/java/org/apache/shardingsphere/shadow/route/engine/ShadowSQLRouter.java
index 6ff6cde..c8f3ff4 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/main/java/org/apache/shardingsphere/shadow/route/engine/ShadowSQLRouter.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/main/java/org/apache/shardingsphere/shadow/route/engine/ShadowSQLRouter.java
@@ -18,11 +18,11 @@
 package org.apache.shardingsphere.shadow.route.engine;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.route.SQLRouter;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 import org.apache.shardingsphere.shadow.route.engine.judge.ShadowDataSourceJudgeEngine;
 import org.apache.shardingsphere.shadow.route.engine.judge.impl.PreparedShadowDataSourceJudgeEngine;
@@ -42,17 +42,16 @@ import java.util.List;
 public final class ShadowSQLRouter implements SQLRouter<ShadowRule> {
     
     @Override
-    public RouteContext createRouteContext(final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                           final ShardingSphereMetaData metaData, final ShadowRule rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShadowRule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
-        if (!(sqlStatementContext.getSqlStatement() instanceof DMLStatement)) {
+        if (!(logicSQL.getSqlStatementContext().getSqlStatement() instanceof DMLStatement)) {
             rule.getShadowMappings().forEach((key, value) -> {
                 result.getRouteUnits().add(new RouteUnit(new RouteMapper(key, key), Collections.emptyList()));
                 result.getRouteUnits().add(new RouteUnit(new RouteMapper(value, value), Collections.emptyList()));
             });
             return result;
         }
-        if (isShadow(sqlStatementContext, parameters, rule)) {
+        if (isShadow(logicSQL.getSqlStatementContext(), logicSQL.getParameters(), rule)) {
             rule.getShadowMappings().values().forEach(each -> result.getRouteUnits().add(new RouteUnit(new RouteMapper(each, each), Collections.emptyList())));
         } else {
             rule.getShadowMappings().keySet().forEach(each -> result.getRouteUnits().add(new RouteUnit(new RouteMapper(each, each), Collections.emptyList())));
@@ -61,10 +60,9 @@ public final class ShadowSQLRouter implements SQLRouter<ShadowRule> {
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                     final ShardingSphereMetaData metaData, final ShadowRule rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final ShadowRule rule, final ConfigurationProperties props) {
         Collection<RouteUnit> toBeAdded = new LinkedList<>();
-        if (!(sqlStatementContext.getSqlStatement() instanceof DMLStatement)) {
+        if (!(logicSQL.getSqlStatementContext().getSqlStatement() instanceof DMLStatement)) {
             for (RouteUnit each : routeContext.getRouteUnits()) {
                 String shadowDataSourceName = rule.getShadowMappings().get(each.getDataSourceMapper().getActualName());
                 toBeAdded.add(new RouteUnit(new RouteMapper(each.getDataSourceMapper().getLogicName(), shadowDataSourceName), each.getTableMappers()));
@@ -73,7 +71,7 @@ public final class ShadowSQLRouter implements SQLRouter<ShadowRule> {
             return;
         }
         Collection<RouteUnit> toBeRemoved = new LinkedList<>();
-        if (isShadow(sqlStatementContext, parameters, rule)) {
+        if (isShadow(logicSQL.getSqlStatementContext(), logicSQL.getParameters(), rule)) {
             for (RouteUnit each : routeContext.getRouteUnits()) {
                 toBeRemoved.add(each);
                 String shadowDataSourceName = rule.getShadowMappings().get(each.getDataSourceMapper().getActualName());
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/test/java/org/apache/shardingsphere/shadow/route/engine/ShadowSQLRouterTest.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/test/java/org/apache/shardingsphere/shadow/route/engine/ShadowSQLRouterTest.java
index b86d78b..b68b3db 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/test/java/org/apache/shardingsphere/shadow/route/engine/ShadowSQLRouterTest.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/test/java/org/apache/shardingsphere/shadow/route/engine/ShadowSQLRouterTest.java
@@ -19,9 +19,11 @@ package org.apache.shardingsphere.shadow.route.engine;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.shadow.api.config.ShadowRuleConfiguration;
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
 import org.apache.shardingsphere.sql.parser.binder.segment.insert.values.InsertValueContext;
@@ -56,35 +58,38 @@ public final class ShadowSQLRouterTest {
     
     private ShadowSQLRouter sqlRouter;
     
-    private ShadowRule shadowRule;
+    private ShadowRule rule;
     
     @Before
     public void setUp() {
         sqlRouter = new ShadowSQLRouter();
-        ShadowRuleConfiguration shadowRuleConfiguration = new ShadowRuleConfiguration(SHADOW_COLUMN, Collections.singletonList(ACTUAL_DATASOURCE), Collections.singletonList(SHADOW_DATASOURCE));
-        shadowRule = new ShadowRule(shadowRuleConfiguration);
+        ShadowRuleConfiguration shadowRuleConfig = new ShadowRuleConfiguration(SHADOW_COLUMN, Collections.singletonList(ACTUAL_DATASOURCE), Collections.singletonList(SHADOW_DATASOURCE));
+        rule = new ShadowRule(shadowRuleConfig);
     }
     
     @Test
     public void assertCreateRouteContextToShadowDataSource() {
-        RouteContext actual = sqlRouter.createRouteContext(mockSQLStatementContextForShadow(), 
-                Collections.emptyList(), mock(ShardingSphereMetaData.class), shadowRule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mockSQLStatementContextForShadow(), "", Collections.emptyList());
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(SHADOW_DATASOURCE));
     }
     
     @Test
     public void assertCreateRouteContextToActualDataSource() {
-        RouteContext actual = sqlRouter.createRouteContext(mockSQLStatementContext(), 
-                Collections.emptyList(), mock(ShardingSphereMetaData.class), shadowRule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mockSQLStatementContext(), "", Collections.emptyList());
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(ACTUAL_DATASOURCE));
     }
     
     @Test
     public void assertCreateRouteContextForNonDMLStatement() {
-        RouteContext actual = sqlRouter.createRouteContext(mockNonDMLSQLStatementContext(), 
-                Collections.emptyList(), mock(ShardingSphereMetaData.class), shadowRule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mockNonDMLSQLStatementContext(), "", Collections.emptyList());
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, rule, new ConfigurationProperties(new Properties()));
         assertThat(actual.getRouteUnits().size(), is(2));
         assertTrue(actual.getActualDataSourceNames().contains(SHADOW_DATASOURCE));
         assertTrue(actual.getActualDataSourceNames().contains(ACTUAL_DATASOURCE));
@@ -94,8 +99,9 @@ public final class ShadowSQLRouterTest {
     public void assertDecorateRouteContextToShadowDataSource() {
         RouteContext actual = new RouteContext();
         actual.getRouteUnits().add(mockRouteUnit());
-        sqlRouter.decorateRouteContext(actual, 
-                mockSQLStatementContextForShadow(), Collections.emptyList(), mock(ShardingSphereMetaData.class), shadowRule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mockSQLStatementContextForShadow(), "", Collections.emptyList());
+        sqlRouter.decorateRouteContext(actual, logicSQL, rule, new ConfigurationProperties(new Properties()));
         assertThat(actual.getRouteUnits().size(), is(1));
         assertTrue(actual.getActualDataSourceNames().contains(SHADOW_DATASOURCE));
     }
@@ -104,8 +110,9 @@ public final class ShadowSQLRouterTest {
     public void assertDecorateRouteContextToActualDataSource() {
         RouteContext actual = new RouteContext();
         actual.getRouteUnits().add(mockRouteUnit());
-        sqlRouter.decorateRouteContext(actual, 
-                mockSQLStatementContext(), Collections.emptyList(), mock(ShardingSphereMetaData.class), shadowRule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mockSQLStatementContext(), "", Collections.emptyList());
+        sqlRouter.decorateRouteContext(actual, logicSQL, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(ACTUAL_DATASOURCE));
     }
@@ -114,8 +121,9 @@ public final class ShadowSQLRouterTest {
     public void assertDecorateRouteContextNonDMLStatement() {
         RouteContext actual = new RouteContext();
         actual.getRouteUnits().add(mockRouteUnit());
-        sqlRouter.decorateRouteContext(actual, 
-                mockNonDMLSQLStatementContext(), Collections.emptyList(), mock(ShardingSphereMetaData.class), shadowRule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mockNonDMLSQLStatementContext(), "", Collections.emptyList());
+        sqlRouter.decorateRouteContext(actual, logicSQL, rule, new ConfigurationProperties(new Properties()));
         assertThat(actual.getRouteUnits().size(), is(2));
         assertTrue(actual.getActualDataSourceNames().contains(SHADOW_DATASOURCE));
         assertTrue(actual.getActualDataSourceNames().contains(ACTUAL_DATASOURCE));
@@ -125,8 +133,9 @@ public final class ShadowSQLRouterTest {
     public void assertDecorateRouteContextWithTableMapper() {
         RouteContext actual = new RouteContext();
         actual.getRouteUnits().add(mockRouteUnit());
-        sqlRouter.decorateRouteContext(actual, 
-                mockSQLStatementContextForShadow(), Collections.emptyList(), mock(ShardingSphereMetaData.class), shadowRule, new ConfigurationProperties(new Properties()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(rule), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        LogicSQL logicSQL = new LogicSQL(schema, mockSQLStatementContextForShadow(), "", Collections.emptyList());
+        sqlRouter.decorateRouteContext(actual, logicSQL, rule, new ConfigurationProperties(new Properties()));
         assertThat(actual.getRouteUnits().size(), is(1));
         assertTrue(actual.getActualDataSourceNames().contains(SHADOW_DATASOURCE));
         Collection<RouteMapper> tableMappers = actual.getRouteUnits().iterator().next().getTableMappers();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
index d40ba42..e2b00b9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.datasource.DataSourceMetaDatas;
 import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
 import org.apache.shardingsphere.infra.rewrite.engine.result.RouteSQLRewriteResult;
@@ -30,10 +31,11 @@ import org.apache.shardingsphere.infra.rewrite.engine.result.SQLRewriteUnit;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.AbstractSQLRewriterParameterizedTest;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.parameter.SQLRewriteEngineTestParameters;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.parameter.SQLRewriteEngineTestParametersBuilder;
-import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRulesBuilder;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -84,7 +86,9 @@ public final class MixSQLRewriterParameterizedTest extends AbstractSQLRewriterPa
         ConfigurationProperties props = new ConfigurationProperties(ruleConfigurations.getProps());
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
                 metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), getTestParameters().getInputParameters(), standardSqlParserEngine.parse(getTestParameters().getInputSQL(), false));
-        RouteContext routeContext = new SQLRouteEngine(metaData, props, rules).route(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
+        ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), rules, Collections.emptyMap(), metaData);
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
+        RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL);
         SQLRewriteResult sqlRewriteResult = new SQLRewriteEntry(metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(),
                 props, rules).rewrite(getTestParameters().getInputSQL(), getTestParameters().getInputParameters(), sqlStatementContext, routeContext);
         return sqlRewriteResult instanceof GenericSQLRewriteResult
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
index 2474730..fd68341 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.datasource.DataSourceMetaDatas;
 import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
 import org.apache.shardingsphere.infra.rewrite.engine.result.RouteSQLRewriteResult;
@@ -30,10 +31,11 @@ import org.apache.shardingsphere.infra.rewrite.engine.result.SQLRewriteUnit;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.AbstractSQLRewriterParameterizedTest;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.parameter.SQLRewriteEngineTestParameters;
 import org.apache.shardingsphere.infra.rewrite.parameterized.engine.parameter.SQLRewriteEngineTestParametersBuilder;
-import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRulesBuilder;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -84,8 +86,9 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
         ConfigurationProperties props = new ConfigurationProperties(yamlRootRuleConfigs.getProps());
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
                 metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), getTestParameters().getInputParameters(), standardSqlParserEngine.parse(getTestParameters().getInputSQL(), false));
-        RouteContext routeContext = new SQLRouteEngine(metaData, props, rules).route(
-                sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
+        ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), rules, Collections.emptyMap(), metaData);
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
+        RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL);
         SQLRewriteResult sqlRewriteResult = new SQLRewriteEntry(metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(),
                 props, rules).rewrite(getTestParameters().getInputSQL(), getTestParameters().getInputParameters(), sqlStatementContext, routeContext);
         return sqlRewriteResult instanceof GenericSQLRewriteResult
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
index 9f5c6c6..1fbf496 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
@@ -20,9 +20,9 @@ package org.apache.shardingsphere.sharding.route.engine;
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.hint.HintManager;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.SQLRouter;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditions;
@@ -56,19 +56,19 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
     
     @SuppressWarnings({"rawtypes", "unchecked"})
     @Override
-    public RouteContext createRouteContext(final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                           final ShardingSphereMetaData metaData, final ShardingRule rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingRule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
-        SQLStatement sqlStatement = sqlStatementContext.getSqlStatement();
+        SQLStatement sqlStatement = logicSQL.getSqlStatementContext().getSqlStatement();
         Optional<ShardingStatementValidator> shardingStatementValidator = ShardingStatementValidatorFactory.newInstance(sqlStatement);
-        shardingStatementValidator.ifPresent(validator -> validator.preValidate(rule, sqlStatementContext, parameters, metaData));
-        ShardingConditions shardingConditions = getShardingConditions(parameters, sqlStatementContext, metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), rule);
-        boolean needMergeShardingValues = isNeedMergeShardingValues(sqlStatementContext, rule);
+        shardingStatementValidator.ifPresent(validator -> validator.preValidate(rule, logicSQL.getSqlStatementContext(), logicSQL.getParameters(), logicSQL.getSchema().getMetaData()));
+        ShardingConditions shardingConditions = getShardingConditions(
+                logicSQL.getParameters(), logicSQL.getSqlStatementContext(), logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData(), rule);
+        boolean needMergeShardingValues = isNeedMergeShardingValues(logicSQL.getSqlStatementContext(), rule);
         if (sqlStatement instanceof DMLStatement && needMergeShardingValues) {
-            checkSubqueryShardingValues(sqlStatementContext, rule, shardingConditions);
+            checkSubqueryShardingValues(logicSQL.getSqlStatementContext(), rule, shardingConditions);
             mergeShardingConditions(shardingConditions);
         }
-        ShardingRouteEngineFactory.newInstance(rule, metaData, sqlStatementContext, shardingConditions, props).route(result, rule);
+        ShardingRouteEngineFactory.newInstance(rule, logicSQL.getSchema().getMetaData(), logicSQL.getSqlStatementContext(), shardingConditions, props).route(result, rule);
         shardingStatementValidator.ifPresent(validator -> validator.postValidate(sqlStatement, result));
         return result;
     }
@@ -156,8 +156,7 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, 
-                                     final List<Object> parameters, final ShardingSphereMetaData metaData, final ShardingRule rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final ShardingRule rule, final ConfigurationProperties props) {
         // TODO
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
index b352647..1dacd22 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
@@ -23,8 +23,10 @@ import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.datasource.DataSourceMetaDatas;
 import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.sharding.route.fixture.AbstractRoutingEngineTest;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.binder.SQLStatementContextFactory;
@@ -56,7 +58,9 @@ public abstract class AbstractSQLRouteTest extends AbstractRoutingEngineTest {
         StandardSQLParserEngine standardSqlParserEngine = SQLParserEngineFactory.getSQLParserEngine("MySQL");
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
                 metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), parameters, standardSqlParserEngine.parse(sql, false));
-        RouteContext result = new SQLRouteEngine(metaData, props, Collections.singletonList(shardingRule)).route(sqlStatementContext, sql, parameters);
+        ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), Collections.singleton(shardingRule), Collections.emptyMap(), metaData);
+        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, sql, parameters);
+        RouteContext result = new SQLRouteEngine(props, Collections.singletonList(shardingRule)).route(logicSQL);
         assertThat(result.getRouteUnits().size(), is(1));
         return result;
     }
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 6f683c5..a97a042 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
@@ -43,7 +43,7 @@ import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.SchemaContextsBuilder;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
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 53e5f7b..0fbe879 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
@@ -36,7 +36,7 @@ import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
 import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/ShardingSphereSchema.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchema.java
similarity index 89%
rename from shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/ShardingSphereSchema.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchema.java
index 226360c..30f1b9c 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/ShardingSphereSchema.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchema.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.context.schema;
+package org.apache.shardingsphere.infra.metadata.schema;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
@@ -46,6 +46,15 @@ public final class ShardingSphereSchema {
     private final ShardingSphereMetaData metaData;
     
     /**
+     * Is complete schema context.
+     *
+     * @return is complete schema context or not
+     */
+    public boolean isComplete() {
+        return !rules.isEmpty() && !dataSources.isEmpty();
+    }
+    
+    /**
      * Close data sources.
      * @param dataSources data sources
      * @throws SQLException exception
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/sql/LogicSQLContext.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/sql/LogicSQL.java
similarity index 83%
rename from shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/sql/LogicSQLContext.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/sql/LogicSQL.java
index 05c84e4..f064fb8 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/sql/LogicSQLContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/sql/LogicSQL.java
@@ -15,23 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.context.sql;
+package org.apache.shardingsphere.infra.sql;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.context.schema.SchemaContext;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 
 import java.util.List;
 
 /**
- * Logic SQL context.
+ * Logic SQL.
  */
 @RequiredArgsConstructor
 @Getter
-public final class LogicSQLContext {
+public final class LogicSQL {
     
-    private final SchemaContext schemaContext;
+    private final ShardingSphereSchema schema;
     
     private final SQLStatementContext<?> sqlStatementContext;
     
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/CloseableDataSource.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/fixture/datasource/CloseableDataSource.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/CloseableDataSource.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/fixture/datasource/CloseableDataSource.java
index dc55a2f..7453a41 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/fixture/CloseableDataSource.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/fixture/datasource/CloseableDataSource.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.context.fixture;
+package org.apache.shardingsphere.infra.metadata.fixture.datasource;
 
 import javax.sql.DataSource;
 import java.io.Closeable;
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/ShardingSphereSchemaTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchemaTest.java
similarity index 60%
rename from shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/ShardingSphereSchemaTest.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchemaTest.java
index 107f617..37ec4ce 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/ShardingSphereSchemaTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/ShardingSphereSchemaTest.java
@@ -15,10 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.context.schema;
+package org.apache.shardingsphere.infra.metadata.schema;
 
-import org.apache.shardingsphere.infra.context.fixture.CloseableDataSource;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.metadata.fixture.datasource.CloseableDataSource;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.Mock;
@@ -32,6 +34,8 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -49,6 +53,27 @@ public final class ShardingSphereSchemaTest {
     private DataSource dataSource2;
     
     @Test
+    public void assertIsComplete() {
+        ShardingSphereSchema schema = new ShardingSphereSchema("name", Collections.singleton(mock(RuleConfiguration.class)),
+                Collections.singleton(mock(ShardingSphereRule.class)), Collections.singletonMap("ds", mock(DataSource.class)), mock(ShardingSphereMetaData.class));
+        assertTrue(schema.isComplete());
+    }
+    
+    @Test
+    public void assertIsNotCompleteWithoutRule() {
+        ShardingSphereSchema schema = new ShardingSphereSchema("name", Collections.emptyList(),
+                Collections.emptyList(), Collections.singletonMap("ds", mock(DataSource.class)), mock(ShardingSphereMetaData.class));
+        assertFalse(schema.isComplete());
+    }
+    
+    @Test
+    public void assertIsNotCompleteWithoutDataSource() {
+        ShardingSphereSchema schema = new ShardingSphereSchema("name", Collections.singleton(mock(RuleConfiguration.class)),
+                Collections.singleton(mock(ShardingSphereRule.class)), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
+        assertFalse(schema.isComplete());
+    }
+    
+    @Test
     public void assertCloseDataSources() throws SQLException, IOException {
         new ShardingSphereSchema("name", Collections.emptyList(), Collections.emptyList(), createDataSources(), mock(ShardingSphereMetaData.class)).closeDataSources(Arrays.asList("ds_0", "ds_2"));
         verify(dataSource0).close();
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessor.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessor.java
index e7bd6be..37bdc19 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessor.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/kernel/KernelProcessor.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.infra.context.kernel;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.context.sql.LogicSQLContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContextBuilder;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
@@ -39,18 +39,18 @@ public final class KernelProcessor {
     /**
      * Generate execution context.
      *
-     * @param logicSQLContext logic SQL context
+     * @param logicSQL logic SQL
      * @param props configuration properties
      * @return execution context
      */
-    public ExecutionContext generateExecutionContext(final LogicSQLContext logicSQLContext, final ConfigurationProperties props) {
-        Collection<ShardingSphereRule> rules = logicSQLContext.getSchemaContext().getSchema().getRules();
-        SQLRouteEngine sqlRouteEngine = new SQLRouteEngine(logicSQLContext.getSchemaContext().getSchema().getMetaData(), props, rules);
-        SQLStatementContext<?> sqlStatementContext = logicSQLContext.getSqlStatementContext();
-        RouteContext routeContext = sqlRouteEngine.route(sqlStatementContext, logicSQLContext.getSql(), logicSQLContext.getParameters());
-        SQLRewriteEntry rewriteEntry = new SQLRewriteEntry(logicSQLContext.getSchemaContext().getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData(), props, rules);
-        SQLRewriteResult rewriteResult = rewriteEntry.rewrite(logicSQLContext.getSql(), logicSQLContext.getParameters(), sqlStatementContext, routeContext);
-        Collection<ExecutionUnit> executionUnits = ExecutionContextBuilder.build(logicSQLContext.getSchemaContext().getSchema().getMetaData(), rewriteResult, sqlStatementContext);
+    public ExecutionContext generateExecutionContext(final LogicSQL logicSQL, final ConfigurationProperties props) {
+        Collection<ShardingSphereRule> rules = logicSQL.getSchema().getRules();
+        SQLRouteEngine sqlRouteEngine = new SQLRouteEngine(props, rules);
+        SQLStatementContext<?> sqlStatementContext = logicSQL.getSqlStatementContext();
+        RouteContext routeContext = sqlRouteEngine.route(logicSQL);
+        SQLRewriteEntry rewriteEntry = new SQLRewriteEntry(logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData(), props, rules);
+        SQLRewriteResult rewriteResult = rewriteEntry.rewrite(logicSQL.getSql(), logicSQL.getParameters(), sqlStatementContext, routeContext);
+        Collection<ExecutionUnit> executionUnits = ExecutionContextBuilder.build(logicSQL.getSchema().getMetaData(), rewriteResult, sqlStatementContext);
         return new ExecutionContext(sqlStatementContext, executionUnits, routeContext);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContext.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContext.java
index 6b682e6..34feb17 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContext.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.infra.context.schema;
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 
 /**
  * Schema context.
@@ -31,20 +32,4 @@ public final class SchemaContext {
     private final ShardingSphereSchema schema;
     
     private final RuntimeContext runtimeContext;
-    
-    /**
-     * Is complete schema context.
-     *
-     * @return is complete schema context or not
-     */
-    public boolean isComplete() {
-        return !schema.getRules().isEmpty() && !schema.getDataSources().isEmpty();
-    }
-    
-    /**
-     * Close executorKernel in runtimeContext.
-     */
-    public void close() {
-        runtimeContext.getExecutorKernel().close();
-    }
 }
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 b8d06f3..4e29028 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
@@ -30,6 +30,7 @@ import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.datasource.DataSourceMetaDatas;
 import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaDataLoader;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRulesBuilder;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
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 316c16e..5d435d6 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
@@ -64,6 +64,6 @@ public final class StandardSchemaContexts implements SchemaContexts {
     
     @Override
     public void close() {
-        schemaContextMap.values().forEach(SchemaContext::close);
+        schemaContextMap.values().forEach(schemaContext -> schemaContext.getRuntimeContext().getExecutorKernel().close());
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/SchemaContextTest.java b/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/SchemaContextTest.java
deleted file mode 100644
index b9bd0f2..0000000
--- a/shardingsphere-infra/shardingsphere-infra-context/src/test/java/org/apache/shardingsphere/infra/context/schema/SchemaContextTest.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.context.schema;
-
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
-import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.junit.Test;
-
-import javax.sql.DataSource;
-import java.util.Collections;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-public final class SchemaContextTest {
-    
-    @Test
-    public void assertIsComplete() {
-        ShardingSphereSchema schema = new ShardingSphereSchema("name", Collections.singleton(mock(RuleConfiguration.class)), 
-                Collections.singleton(mock(ShardingSphereRule.class)), Collections.singletonMap("ds", mock(DataSource.class)), mock(ShardingSphereMetaData.class));
-        assertTrue(new SchemaContext(schema, mock(RuntimeContext.class)).isComplete());
-    }
-    
-    @Test
-    public void assertIsNotCompleteWithoutRule() {
-        ShardingSphereSchema schema = new ShardingSphereSchema("name", Collections.emptyList(), 
-                Collections.emptyList(), Collections.singletonMap("ds", mock(DataSource.class)), mock(ShardingSphereMetaData.class));
-        assertFalse(new SchemaContext(schema, mock(RuntimeContext.class)).isComplete());
-    }
-    
-    @Test
-    public void assertIsNotCompleteWithoutDataSource() {
-        ShardingSphereSchema schema = new ShardingSphereSchema("name", Collections.singleton(mock(RuleConfiguration.class)),
-                Collections.singleton(mock(ShardingSphereRule.class)), Collections.emptyMap(), mock(ShardingSphereMetaData.class));
-        assertFalse(new SchemaContext(schema, mock(RuntimeContext.class)).isComplete());
-    }
-}
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 064f391..0e65870 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
@@ -21,7 +21,7 @@ import org.apache.shardingsphere.infra.auth.Authentication;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.junit.Test;
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLogger.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLogger.java
index b836ec9..d8e9794 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLogger.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLogger.java
@@ -22,6 +22,7 @@ import lombok.NoArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 
 import java.util.Collection;
 import java.util.HashSet;
@@ -35,15 +36,15 @@ import java.util.Set;
 public final class SQLLogger {
     
     /**
-     * Print SQL log.
+     * Log SQL.
      * 
      * @param logicSQL logic SQL
      * @param showSimple whether show SQL in simple style
      * @param executionContext Execution context
      */
-    public static void logSQL(final String logicSQL, final boolean showSimple, final ExecutionContext executionContext) {
-        log("Logic SQL: {}", logicSQL);
-        log("SQLStatement: {}", executionContext.getSqlStatementContext());
+    public static void logSQL(final LogicSQL logicSQL, final boolean showSimple, final ExecutionContext executionContext) {
+        log("Logic SQL: {}", logicSQL.getSql());
+        log("SQLStatement: {}", logicSQL.getSqlStatementContext());
         if (showSimple) {
             logSimpleMode(executionContext.getExecutionUnits());
         } else {
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLoggerTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLoggerTest.java
index f5c1670..d87f7bc 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLoggerTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/log/SQLLoggerTest.java
@@ -20,7 +20,9 @@ package org.apache.shardingsphere.infra.executor.sql.log;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionUnit;
 import org.apache.shardingsphere.infra.executor.sql.context.SQLUnit;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -34,6 +36,7 @@ import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.stream.Collectors;
@@ -46,6 +49,8 @@ public final class SQLLoggerTest {
     
     private static final String SQL = "SELECT * FROM t_user";
     
+    private final LogicSQL logicSQL = new LogicSQL(mock(ShardingSphereSchema.class), null, SQL, Collections.emptyList());
+    
     private Collection<ExecutionUnit> executionUnits;
     
     @Mock
@@ -63,7 +68,7 @@ public final class SQLLoggerTest {
     
     @Test
     public void assertLogNormalSQLWithoutParameter() {
-        SQLLogger.logSQL(SQL, false, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
+        SQLLogger.logSQL(logicSQL, false, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
         inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
@@ -76,7 +81,7 @@ public final class SQLLoggerTest {
     public void assertLogNormalSQLWithParameters() {
         List<Object> parameters = executionUnits.iterator().next().getSqlUnit().getParameters();
         parameters.add("parameter");
-        SQLLogger.logSQL(SQL, false, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
+        SQLLogger.logSQL(logicSQL, false, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
         inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
@@ -87,7 +92,7 @@ public final class SQLLoggerTest {
     
     @Test
     public void assertLogSimpleSQL() {
-        SQLLogger.logSQL(SQL, true, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
+        SQLLogger.logSQL(logicSQL, true, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
         inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/SQLRouter.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/SQLRouter.java
index 1d31cbd..ac70189 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/SQLRouter.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/SQLRouter.java
@@ -18,13 +18,10 @@
 package org.apache.shardingsphere.infra.route;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.spi.order.OrderedSPI;
-import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
-
-import java.util.List;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 
 /**
  * SQL Router.
@@ -36,24 +33,20 @@ public interface SQLRouter<T extends ShardingSphereRule> extends OrderedSPI<T> {
     /**
      * Create route context.
      *
-     * @param sqlStatementContext SQL statement context
-     * @param parameters SQL parameters
-     * @param metaData meta data of ShardingSphere
+     * @param logicSQL logic SQL
      * @param rule rule
      * @param props configuration properties
      * @return route context
      */
-    RouteContext createRouteContext(SQLStatementContext<?> sqlStatementContext, List<Object> parameters, ShardingSphereMetaData metaData, T rule, ConfigurationProperties props);
+    RouteContext createRouteContext(LogicSQL logicSQL, T rule, ConfigurationProperties props);
     
     /**
      * Decorate route context.
      * 
      * @param routeContext route context
-     * @param sqlStatementContext SQL statement context
-     * @param parameters SQL parameters
-     * @param metaData meta data of ShardingSphere
+     * @param logicSQL logic SQL
      * @param rule rule
      * @param props configuration properties
      */
-    void decorateRouteContext(RouteContext routeContext, SQLStatementContext<?> sqlStatementContext, List<Object> parameters, ShardingSphereMetaData metaData, T rule, ConfigurationProperties props);
+    void decorateRouteContext(RouteContext routeContext, LogicSQL logicSQL, T rule, ConfigurationProperties props);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/UnconfiguredSchemaSQLRouter.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/UnconfiguredSchemaSQLRouter.java
index 8508f3e..27e3177 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/UnconfiguredSchemaSQLRouter.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/UnconfiguredSchemaSQLRouter.java
@@ -17,11 +17,10 @@
 
 package org.apache.shardingsphere.infra.route;
 
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowTablesStatement;
 
@@ -36,12 +35,11 @@ public final class UnconfiguredSchemaSQLRouter {
      * Decorate route context.
      *
      * @param routeContext route context
-     * @param sqlStatementContext SQL statement context
-     * @param metaData meta data of ShardingSphere
+     * @param logicSQL logic SQL
      */
-    public void decorate(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, final ShardingSphereMetaData metaData) {
-        if (isNeedUnconfiguredSchema(sqlStatementContext.getSqlStatement())) {
-            for (String each : metaData.getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().keySet()) {
+    public void decorate(final RouteContext routeContext, final LogicSQL logicSQL) {
+        if (isNeedUnconfiguredSchema(logicSQL.getSqlStatementContext().getSqlStatement())) {
+            for (String each : logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().keySet()) {
                 routeContext.getRouteUnits().add(new RouteUnit(new RouteMapper(each, each), Collections.emptyList()));
             }
         }
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngine.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngine.java
index 25b0667..0d35d35 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngine.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.infra.route.engine;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.UnconfiguredSchemaSQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
@@ -26,10 +25,9 @@ import org.apache.shardingsphere.infra.route.hook.SPIRoutingHook;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.order.OrderedSPIRegistry;
-import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 
 import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -42,8 +40,6 @@ public final class SQLRouteEngine {
         ShardingSphereServiceLoader.register(SQLRouter.class);
     }
     
-    private final ShardingSphereMetaData metaData;
-    
     private final ConfigurationProperties props;
     
     @SuppressWarnings("rawtypes")
@@ -51,8 +47,7 @@ public final class SQLRouteEngine {
     
     private final SPIRoutingHook routingHook;
     
-    public SQLRouteEngine(final ShardingSphereMetaData metaData, final ConfigurationProperties props, final Collection<ShardingSphereRule> rules) {
-        this.metaData = metaData;
+    public SQLRouteEngine(final ConfigurationProperties props, final Collection<ShardingSphereRule> rules) {
         this.props = props;
         decorators = OrderedSPIRegistry.getRegisteredServices(rules, SQLRouter.class);
         routingHook = new SPIRoutingHook();
@@ -61,16 +56,14 @@ public final class SQLRouteEngine {
     /**
      * Route SQL.
      *
-     * @param sqlStatementContext SQL statement context
-     * @param sql SQL
-     * @param parameters SQL parameters
+     * @param logicSQL logic SQL
      * @return route context
      */
-    public RouteContext route(final SQLStatementContext<?> sqlStatementContext, final String sql, final List<Object> parameters) {
-        routingHook.start(sql);
+    public RouteContext route(final LogicSQL logicSQL) {
+        routingHook.start(logicSQL.getSql());
         try {
-            RouteContext result = doRoute(sqlStatementContext, parameters);
-            routingHook.finishSuccess(result, metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData());
+            RouteContext result = doRoute(logicSQL);
+            routingHook.finishSuccess(result, logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData());
             return result;
             // CHECKSTYLE:OFF
         } catch (final Exception ex) {
@@ -81,19 +74,19 @@ public final class SQLRouteEngine {
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private RouteContext doRoute(final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters) {
+    private RouteContext doRoute(final LogicSQL logicSQL) {
         RouteContext result = null;
         for (Entry<ShardingSphereRule, SQLRouter> entry : decorators.entrySet()) {
             if (null == result) {
-                result = entry.getValue().createRouteContext(sqlStatementContext, parameters, metaData, entry.getKey(), props);
+                result = entry.getValue().createRouteContext(logicSQL, entry.getKey(), props);
             } else {
-                entry.getValue().decorateRouteContext(result, sqlStatementContext, parameters, metaData, entry.getKey(), props);
+                entry.getValue().decorateRouteContext(result, logicSQL, entry.getKey(), props);
             }
         }
         if (null == result) {
             result = new RouteContext();
         }
-        new UnconfiguredSchemaSQLRouter().decorate(result, sqlStatementContext, metaData);
+        new UnconfiguredSchemaSQLRouter().decorate(result, logicSQL);
         return result;
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngineTest.java b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngineTest.java
index 29c7e7e..6927f9d 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngineTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/engine/SQLRouteEngineTest.java
@@ -20,11 +20,13 @@ package org.apache.shardingsphere.infra.route.engine;
 import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.infra.route.fixture.rule.RouteFailureRuleFixture;
 import org.apache.shardingsphere.infra.route.fixture.rule.RouteRuleFixture;
 import org.apache.shardingsphere.infra.route.hook.SPIRoutingHook;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.sql.parser.binder.metadata.schema.SchemaMetaData;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 import org.junit.Before;
@@ -63,9 +65,11 @@ public final class SQLRouteEngineTest {
     
     @Test
     public void assertRouteSuccess() {
-        SQLRouteEngine sqlRouteEngine = new SQLRouteEngine(metaData, props, Collections.singletonList(new RouteRuleFixture()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(new RouteRuleFixture()), Collections.emptyMap(), metaData);
+        LogicSQL logicSQL = new LogicSQL(schema, mock(SQLStatementContext.class), "SELECT 1", Collections.emptyList());
+        SQLRouteEngine sqlRouteEngine = new SQLRouteEngine(props, Collections.singleton(new RouteRuleFixture()));
         setSPIRoutingHook(sqlRouteEngine);
-        RouteContext actual = sqlRouteEngine.route(mock(SQLStatementContext.class), "SELECT 1", Collections.emptyList());
+        RouteContext actual = sqlRouteEngine.route(logicSQL);
         assertThat(actual.getRouteUnits().size(), is(1));
         RouteUnit routeUnit = actual.getRouteUnits().iterator().next();
         assertThat(routeUnit.getDataSourceMapper().getLogicName(), is("ds"));
@@ -77,10 +81,12 @@ public final class SQLRouteEngineTest {
     
     @Test(expected = UnsupportedOperationException.class)
     public void assertRouteFailure() {
-        SQLRouteEngine sqlRouteEngine = new SQLRouteEngine(metaData, props, Collections.singletonList(new RouteFailureRuleFixture()));
+        ShardingSphereSchema schema = new ShardingSphereSchema("logic_schema", Collections.emptyList(), Collections.singleton(new RouteRuleFixture()), Collections.emptyMap(), metaData);
+        LogicSQL logicSQL = new LogicSQL(schema, mock(SQLStatementContext.class), "SELECT 1", Collections.emptyList());
+        SQLRouteEngine sqlRouteEngine = new SQLRouteEngine(props, Collections.singleton(new RouteFailureRuleFixture()));
         setSPIRoutingHook(sqlRouteEngine);
         try {
-            sqlRouteEngine.route(mock(SQLStatementContext.class), "SELECT 1", Collections.emptyList());
+            sqlRouteEngine.route(logicSQL);
         } catch (final UnsupportedOperationException ex) {
             verify(routingHook).start("SELECT 1");
             verify(routingHook).finishFailure(ex);
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/router/SQLRouterFailureFixture.java b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/router/SQLRouterFailureFixture.java
index 7df2e15..50fb721 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/router/SQLRouterFailureFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/router/SQLRouterFailureFixture.java
@@ -18,25 +18,20 @@
 package org.apache.shardingsphere.infra.route.fixture.router;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.SQLRouter;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.fixture.rule.RouteFailureRuleFixture;
-import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
-
-import java.util.List;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 
 public final class SQLRouterFailureFixture implements SQLRouter<RouteFailureRuleFixture> {
     
     @Override
-    public RouteContext createRouteContext(final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                           final ShardingSphereMetaData metaData, final RouteFailureRuleFixture rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final RouteFailureRuleFixture rule, final ConfigurationProperties props) {
         throw new UnsupportedOperationException("Route failure.");
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                     final ShardingSphereMetaData metaData, final RouteFailureRuleFixture rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final RouteFailureRuleFixture rule, final ConfigurationProperties props) {
         throw new UnsupportedOperationException("Route failure.");
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/router/SQLRouterFixture.java b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/router/SQLRouterFixture.java
index 603fae1..25d8850 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/router/SQLRouterFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/router/SQLRouterFixture.java
@@ -18,30 +18,26 @@
 package org.apache.shardingsphere.infra.route.fixture.router;
 
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.fixture.rule.RouteRuleFixture;
-import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 
 import java.util.Collections;
-import java.util.List;
 
 public final class SQLRouterFixture implements SQLRouter<RouteRuleFixture> {
     
     @Override
-    public RouteContext createRouteContext(final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                           final ShardingSphereMetaData metaData, final RouteRuleFixture rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final RouteRuleFixture rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
         result.getRouteUnits().add(new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.emptyList()));
         return result;
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, 
-                                     final ShardingSphereMetaData metaData, final RouteRuleFixture rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final RouteRuleFixture rule, final ConfigurationProperties props) {
         routeContext.getRouteUnits().add(new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.emptyList()));
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
index 22db49f..0aae2a7 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/AbstractStatementExecutor.java
@@ -19,8 +19,8 @@ package org.apache.shardingsphere.driver.executor;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.executor.kernel.InputGroup;
 import org.apache.shardingsphere.infra.executor.sql.QueryResult;
@@ -75,17 +75,17 @@ public abstract class AbstractStatementExecutor {
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    protected final void refreshTableMetaData(final SchemaContext schemaContext, final SQLStatementContext<?> sqlStatementContext) throws SQLException {
+    protected final void refreshTableMetaData(final ShardingSphereSchema schema, final SQLStatementContext<?> sqlStatementContext) throws SQLException {
         if (null == sqlStatementContext) {
             return;
         }
         Optional<MetaDataRefreshStrategy> refreshStrategy = MetaDataRefreshStrategyFactory.newInstance(sqlStatementContext);
         if (refreshStrategy.isPresent()) {
-            RuleSchemaMetaDataLoader metaDataLoader = new RuleSchemaMetaDataLoader(schemaContext.getSchema().getRules());
-            refreshStrategy.get().refreshMetaData(schemaContext.getSchema().getMetaData(), schemaContexts.getDatabaseType(),
+            RuleSchemaMetaDataLoader metaDataLoader = new RuleSchemaMetaDataLoader(schema.getRules());
+            refreshStrategy.get().refreshMetaData(schema.getMetaData(), schemaContexts.getDatabaseType(),
                     dataSourceMap, sqlStatementContext, tableName -> metaDataLoader.load(schemaContexts.getDatabaseType(),
                             dataSourceMap, tableName, schemaContexts.getProps()));
-            notifyPersistRuleMetaData(DefaultSchema.LOGIC_NAME, schemaContext.getSchema().getMetaData().getRuleSchemaMetaData());
+            notifyPersistRuleMetaData(DefaultSchema.LOGIC_NAME, schema.getMetaData().getRuleSchemaMetaData());
         }
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java
index 56d8019..11bfd51 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/PreparedStatementExecutor.java
@@ -78,7 +78,7 @@ public final class PreparedStatementExecutor extends AbstractStatementExecutor {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback<Integer> sqlExecutorCallback = createDefaultSQLExecutorCallbackWithInteger(isExceptionThrown);
         List<Integer> results = getSqlExecutor().execute(inputGroups, sqlExecutorCallback);
-        refreshTableMetaData(getSchemaContexts().getDefaultSchemaContext(), sqlStatementContext);
+        refreshTableMetaData(getSchemaContexts().getDefaultSchemaContext().getSchema(), sqlStatementContext);
         return isNeedAccumulate(
                 getSchemaContexts().getDefaultSchemaContext().getSchema().getRules().stream().filter(rule -> rule instanceof DataNodeRoutedRule).collect(Collectors.toList()), sqlStatementContext)
                 ? accumulate(results) : results.get(0);
@@ -99,7 +99,7 @@ public final class PreparedStatementExecutor extends AbstractStatementExecutor {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback<Boolean> sqlExecutorCallback = createDefaultSQLExecutorCallbackWithBoolean(isExceptionThrown);
         List<Boolean> result = getSqlExecutor().execute(inputGroups, sqlExecutorCallback);
-        refreshTableMetaData(getSchemaContexts().getDefaultSchemaContext(), sqlStatementContext);
+        refreshTableMetaData(getSchemaContexts().getDefaultSchemaContext().getSchema(), sqlStatementContext);
         if (null == result || result.isEmpty() || null == result.get(0)) {
             return false;
         }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java
index fdbe9de..7f411be 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/executor/StatementExecutor.java
@@ -111,7 +111,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
         return executeUpdate(inputGroups, (statement, sql) -> statement.executeUpdate(sql, columnNames), sqlStatementContext);
     }
     
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"unchecked", "rawtypes"})
     private int executeUpdate(final Collection<InputGroup<StatementExecuteUnit>> inputGroups, final Updater updater, final SQLStatementContext<?> sqlStatementContext) throws SQLException {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback sqlExecutorCallback = new DefaultSQLExecutorCallback<Integer>(getSchemaContexts().getDatabaseType(), isExceptionThrown) {
@@ -122,7 +122,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
             }
         };
         List<Integer> results = getSqlExecutor().execute(inputGroups, sqlExecutorCallback);
-        refreshTableMetaData(getSchemaContexts().getDefaultSchemaContext(), sqlStatementContext);
+        refreshTableMetaData(getSchemaContexts().getDefaultSchemaContext().getSchema(), sqlStatementContext);
         if (isNeedAccumulate(
                 getSchemaContexts().getDefaultSchemaContext().getSchema().getRules().stream().filter(rule -> rule instanceof DataNodeRoutedRule).collect(Collectors.toList()), sqlStatementContext)) {
             return accumulate(results);
@@ -174,7 +174,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
         return execute(inputGroups, (statement, sql) -> statement.execute(sql, columnNames), sqlStatementContext);
     }
     
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"unchecked", "rawtypes"})
     private boolean execute(final Collection<InputGroup<StatementExecuteUnit>> inputGroups, final Executor executor, final SQLStatementContext<?> sqlStatementContext) throws SQLException {
         boolean isExceptionThrown = ExecutorExceptionHandler.isExceptionThrown();
         SQLExecutorCallback sqlExecutorCallback = new DefaultSQLExecutorCallback<Boolean>(getSchemaContexts().getDatabaseType(), isExceptionThrown) {
@@ -185,7 +185,7 @@ public final class StatementExecutor extends AbstractStatementExecutor {
             }
         };
         List<Boolean> result = getSqlExecutor().execute(inputGroups, sqlExecutorCallback);
-        refreshTableMetaData(getSchemaContexts().getDefaultSchemaContext(), sqlStatementContext);
+        refreshTableMetaData(getSchemaContexts().getDefaultSchemaContext().getSchema(), sqlStatementContext);
         if (null == result || result.isEmpty() || null == result.get(0)) {
             return false;
         }
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 bf935b7..4e90e88 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
@@ -32,7 +32,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKe
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
-import org.apache.shardingsphere.infra.context.sql.LogicSQLContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.kernel.InputGroup;
 import org.apache.shardingsphere.infra.executor.sql.ExecutorConstant;
@@ -256,17 +256,18 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     }
     
     private ExecutionContext createExecutionContext() {
-        ExecutionContext result = new KernelProcessor().generateExecutionContext(createLogicSQLContext(), schemaContexts.getProps());
+        LogicSQL logicSQL = createLogicSQL();
+        ExecutionContext result = new KernelProcessor().generateExecutionContext(logicSQL, schemaContexts.getProps());
         findGeneratedKey(result).ifPresent(generatedKey -> generatedValues.addAll(generatedKey.getGeneratedValues()));
-        logSQL(result);
+        logSQL(logicSQL, result);
         return result;
     }
     
-    private LogicSQLContext createLogicSQLContext() {
+    private LogicSQL createLogicSQL() {
         List<Object> parameters = new ArrayList<>(getParameters());
         SchemaMetaData schemaMetaData = schemaContexts.getDefaultSchemaContext().getSchema().getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, parameters, sqlStatement);
-        return new LogicSQLContext(schemaContexts.getDefaultSchemaContext(), sqlStatementContext, sql, parameters);
+        return new LogicSQL(schemaContexts.getDefaultSchemaContext().getSchema(), sqlStatementContext, sql, parameters);
     }
     
     private MergedResult mergeQuery(final List<QueryResult> queryResults) throws SQLException {
@@ -308,9 +309,9 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
                 ? ((InsertStatementContext) executionContext.getSqlStatementContext()).getGeneratedKeyContext() : Optional.empty();
     }
     
-    private void logSQL(final ExecutionContext executionContext) {
+    private void logSQL(final LogicSQL logicSQL, final ExecutionContext executionContext) {
         if (schemaContexts.getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)) {
-            SQLLogger.logSQL(sql, schemaContexts.getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SIMPLE), executionContext);
+            SQLLogger.logSQL(logicSQL, schemaContexts.getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SIMPLE), executionContext);
         }
     }
     
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 3f348b2..e2e06c9 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
@@ -30,7 +30,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKe
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
-import org.apache.shardingsphere.infra.context.sql.LogicSQLContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.kernel.InputGroup;
 import org.apache.shardingsphere.infra.executor.sql.ExecutorConstant;
@@ -281,8 +281,9 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     
     private ExecutionContext createExecutionContext(final String sql) throws SQLException {
         clearStatements();
-        ExecutionContext result = new KernelProcessor().generateExecutionContext(createLogicSQLContext(sql), schemaContexts.getProps());
-        logSQL(sql, schemaContexts.getProps(), result);
+        LogicSQL logicSQL = createLogicSQL(sql);
+        ExecutionContext result = new KernelProcessor().generateExecutionContext(logicSQL, schemaContexts.getProps());
+        logSQL(logicSQL, schemaContexts.getProps(), result);
         return result;
     }
     
@@ -293,18 +294,18 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
         statements.clear();
     }
     
-    private void logSQL(final String sql, final ConfigurationProperties props, final ExecutionContext executionContext) {
+    private void logSQL(final LogicSQL logicSQL, final ConfigurationProperties props, final ExecutionContext executionContext) {
         if (props.<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)) {
-            SQLLogger.logSQL(sql, props.<Boolean>getValue(ConfigurationPropertyKey.SQL_SIMPLE), executionContext);
+            SQLLogger.logSQL(logicSQL, props.<Boolean>getValue(ConfigurationPropertyKey.SQL_SIMPLE), executionContext);
         }
     }
     
-    private LogicSQLContext createLogicSQLContext(final String sql) {
+    private LogicSQL createLogicSQL(final String sql) {
         SchemaContext schemaContext = schemaContexts.getDefaultSchemaContext();
         SchemaMetaData schemaMetaData = schemaContext.getSchema().getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
         SQLStatement sqlStatement = schemaContext.getRuntimeContext().getSqlParserEngine().parse(sql, false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, Collections.emptyList(), sqlStatement);
-        return new LogicSQLContext(schemaContext, sqlStatementContext, sql, Collections.emptyList());
+        return new LogicSQL(schemaContext.getSchema(), sqlStatementContext, sql, Collections.emptyList());
     }
     
     private Collection<InputGroup<StatementExecuteUnit>> getInputGroups() throws SQLException {
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
index 7551a26..fb33504 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/test/java/org/apache/shardingsphere/driver/jdbc/core/connection/ShardingSphereConnectionTest.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.driver.jdbc.core.connection;
 import org.apache.shardingsphere.driver.jdbc.core.fixture.BASEShardingTransactionManagerFixture;
 import org.apache.shardingsphere.driver.jdbc.core.fixture.XAShardingTransactionManagerFixture;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
index 891a33a..0d99085 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngineFactory.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.proxy.backend.communication;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
-import org.apache.shardingsphere.infra.context.sql.LogicSQLContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.engine.jdbc.JDBCExecuteEngine;
@@ -62,8 +62,8 @@ public final class DatabaseCommunicationEngineFactory {
      * @return text protocol backend handler
      */
     public DatabaseCommunicationEngine newTextProtocolInstance(final SQLStatement sqlStatement, final String sql, final BackendConnection backendConnection) {
-        LogicSQLContext logicSQLContext = createLogicSQLContext(sqlStatement, sql, Collections.emptyList(), backendConnection);
-        return new JDBCDatabaseCommunicationEngine(logicSQLContext, new JDBCExecuteEngine(backendConnection, new StatementAccessor()));
+        LogicSQL logicSQL = createLogicSQL(sqlStatement, sql, Collections.emptyList(), backendConnection);
+        return new JDBCDatabaseCommunicationEngine(logicSQL, new JDBCExecuteEngine(backendConnection, new StatementAccessor()));
     }
     
     /**
@@ -76,14 +76,14 @@ public final class DatabaseCommunicationEngineFactory {
      * @return binary protocol backend handler
      */
     public DatabaseCommunicationEngine newBinaryProtocolInstance(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final BackendConnection backendConnection) {
-        LogicSQLContext logicSQLContext = createLogicSQLContext(sqlStatement, sql, new ArrayList<>(parameters), backendConnection);
-        return new JDBCDatabaseCommunicationEngine(logicSQLContext, new JDBCExecuteEngine(backendConnection, new PreparedStatementAccessor()));
+        LogicSQL logicSQL = createLogicSQL(sqlStatement, sql, new ArrayList<>(parameters), backendConnection);
+        return new JDBCDatabaseCommunicationEngine(logicSQL, new JDBCExecuteEngine(backendConnection, new PreparedStatementAccessor()));
     }
     
-    private LogicSQLContext createLogicSQLContext(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final BackendConnection backendConnection) {
+    private LogicSQL createLogicSQL(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final BackendConnection backendConnection) {
         SchemaContext schemaContext = ProxyContext.getInstance().getSchema(backendConnection.getSchemaName());
         SchemaMetaData schemaMetaData = schemaContext.getSchema().getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, parameters, sqlStatement);
-        return new LogicSQLContext(schemaContext, sqlStatementContext, sql, parameters);
+        return new LogicSQL(schemaContext.getSchema(), sqlStatementContext, sql, parameters);
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
index a1c78b6..e819698 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
@@ -34,7 +34,7 @@ import org.apache.shardingsphere.infra.rule.DataNodeRoutedRule;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.SQLExecuteEngine;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.infra.context.sql.LogicSQLContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
 import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
@@ -54,7 +54,7 @@ import java.util.Optional;
 @RequiredArgsConstructor
 public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicationEngine {
     
-    private final LogicSQLContext logicSQLContext;
+    private final LogicSQL logicSQL;
     
     private final SQLExecuteEngine sqlExecuteEngine;
     
@@ -64,14 +64,14 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
     
     @Override
     public BackendResponse execute() throws SQLException {
-        ExecutionContext executionContext = new KernelProcessor().generateExecutionContext(logicSQLContext, ProxyContext.getInstance().getSchemaContexts().getProps());
+        ExecutionContext executionContext = new KernelProcessor().generateExecutionContext(logicSQL, ProxyContext.getInstance().getSchemaContexts().getProps());
         logSQL(executionContext);
         return doExecute(executionContext);
     }
     
     private void logSQL(final ExecutionContext executionContext) {
         if (ProxyContext.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)) {
-            SQLLogger.logSQL(logicSQLContext.getSql(), ProxyContext.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SIMPLE), executionContext);
+            SQLLogger.logSQL(logicSQL, ProxyContext.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SIMPLE), executionContext);
         }
     }
     
@@ -85,24 +85,24 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
         return merge(executionContext.getSqlStatementContext());
     }
     
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"unchecked", "rawtypes"})
     private void refreshTableMetaData(final SQLStatementContext<?> sqlStatementContext) throws SQLException {
         if (null == sqlStatementContext) {
             return;
         }
         Optional<MetaDataRefreshStrategy> refreshStrategy = MetaDataRefreshStrategyFactory.newInstance(sqlStatementContext);
         if (refreshStrategy.isPresent()) {
-            refreshStrategy.get().refreshMetaData(logicSQLContext.getSchemaContext().getSchema().getMetaData(), ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
-                    logicSQLContext.getSchemaContext().getSchema().getDataSources(), sqlStatementContext, this::loadTableMetaData);
+            refreshStrategy.get().refreshMetaData(logicSQL.getSchema().getMetaData(), ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
+                    logicSQL.getSchema().getDataSources(), sqlStatementContext, this::loadTableMetaData);
             GovernanceEventBus.getInstance().post(
-                    new MetaDataPersistEvent(logicSQLContext.getSchemaContext().getSchema().getName(), logicSQLContext.getSchemaContext().getSchema().getMetaData().getRuleSchemaMetaData()));
+                    new MetaDataPersistEvent(logicSQL.getSchema().getName(), logicSQL.getSchema().getMetaData().getRuleSchemaMetaData()));
         }
     }
     
     private Optional<TableMetaData> loadTableMetaData(final String tableName) throws SQLException {
-        RuleSchemaMetaDataLoader loader = new RuleSchemaMetaDataLoader(logicSQLContext.getSchemaContext().getSchema().getRules());
+        RuleSchemaMetaDataLoader loader = new RuleSchemaMetaDataLoader(logicSQL.getSchema().getRules());
         return loader.load(ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
-                logicSQLContext.getSchemaContext().getSchema().getDataSources(), tableName, ProxyContext.getInstance().getSchemaContexts().getProps());
+                logicSQL.getSchema().getDataSources(), tableName, ProxyContext.getInstance().getSchemaContexts().getProps());
     }
     
     private BackendResponse merge(final SQLStatementContext<?> sqlStatementContext) throws SQLException {
@@ -122,14 +122,14 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
     
     private boolean isNeedAccumulate(final SQLStatementContext<?> sqlStatementContext) {
         Optional<DataNodeRoutedRule> dataNodeRoutedRule =
-                logicSQLContext.getSchemaContext().getSchema().getRules().stream().filter(each -> each instanceof DataNodeRoutedRule).findFirst().map(rule -> (DataNodeRoutedRule) rule);
+                logicSQL.getSchema().getRules().stream().filter(each -> each instanceof DataNodeRoutedRule).findFirst().map(rule -> (DataNodeRoutedRule) rule);
         return dataNodeRoutedRule.isPresent() && dataNodeRoutedRule.get().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames());
     }
     
     private MergedResult mergeQuery(final SQLStatementContext<?> sqlStatementContext, final List<QueryResult> queryResults) throws SQLException {
         MergeEngine mergeEngine = new MergeEngine(ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
-                logicSQLContext.getSchemaContext().getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData(), 
-                ProxyContext.getInstance().getSchemaContexts().getProps(), logicSQLContext.getSchemaContext().getSchema().getRules());
+                logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData(), 
+                ProxyContext.getInstance().getSchemaContexts().getProps(), logicSQL.getSchema().getRules());
         return mergeEngine.merge(queryResults, sqlStatementContext);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/BroadcastBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/BroadcastBackendHandler.java
index 64d0dd5..4299e20 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/BroadcastBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/BroadcastBackendHandler.java
@@ -49,7 +49,7 @@ public final class BroadcastBackendHandler implements TextProtocolBackendHandler
         String originalSchema = backendConnection.getSchemaName();
         for (String each : ProxyContext.getInstance().getAllSchemaNames()) {
             backendConnection.setCurrentSchema(each);
-            if (!ProxyContext.getInstance().getSchema(each).isComplete()) {
+            if (!ProxyContext.getInstance().getSchema(each).getSchema().isComplete()) {
                 throw new RuleNotExistsException();
             }
             databaseCommunicationEngineFactory.newTextProtocolInstance(sqlStatement, sql, backendConnection).execute();
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandler.java
index 2eadd62..4113d50 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandler.java
@@ -57,7 +57,7 @@ public final class ShowTablesBackendHandler implements TextProtocolBackendHandle
         if (null == schemaContext) {
             throw new NoDatabaseSelectedException();
         }
-        if (!schemaContext.isComplete()) {
+        if (!schemaContext.getSchema().isComplete()) {
             return getDefaultQueryResponse(backendConnection.getSchemaName());
         }
         // TODO Get all tables from meta data.
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/UnicastBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/UnicastBackendHandler.java
index a19a6c6..9758fc8 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/UnicastBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/UnicastBackendHandler.java
@@ -56,7 +56,7 @@ public final class UnicastBackendHandler implements TextProtocolBackendHandler {
             if (schemaContexts.isEmpty()) {
                 throw new NoDatabaseSelectedException();
             }
-            if (!schemaContexts.values().iterator().next().isComplete()) {
+            if (!schemaContexts.values().iterator().next().getSchema().isComplete()) {
                 throw new RuleNotExistsException();
             }
             // TODO we should remove set default ShardingSphere schema after parser can recognize all DAL broadcast SQL.
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/query/QueryBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/query/QueryBackendHandler.java
index df51b96..0578797 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/query/QueryBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/query/QueryBackendHandler.java
@@ -54,7 +54,7 @@ public final class QueryBackendHandler implements TextProtocolBackendHandler {
         if (null == schemaContext) {
             throw new NoDatabaseSelectedException();
         }
-        if (!schemaContext.isComplete()) {
+        if (!schemaContext.getSchema().isComplete()) {
             throw new RuleNotExistsException();
         }
         databaseCommunicationEngine = databaseCommunicationEngineFactory.newTextProtocolInstance(sqlStatement, sql, backendConnection);
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 4f7a316..a4a38e1 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
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.sctl.explain;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
-import org.apache.shardingsphere.infra.context.sql.LogicSQLContext;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 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.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -68,22 +68,22 @@ public final class ShardingCTLExplainBackendHandler implements TextProtocolBacke
         if (null == schemaContext) {
             throw new NoDatabaseSelectedException();
         }
-        if (!schemaContext.isComplete()) {
+        if (!schemaContext.getSchema().isComplete()) {
             throw new RuleNotExistsException();
         }
-        LogicSQLContext logicSQLContext = createLogicSQLContext(schemaContext, explainStatement.get());
-        executionUnits = new KernelProcessor().generateExecutionContext(logicSQLContext, ProxyContext.getInstance().getSchemaContexts().getProps()).getExecutionUnits().iterator();
+        LogicSQL logicSQL = createLogicSQL(schemaContext, explainStatement.get());
+        executionUnits = new KernelProcessor().generateExecutionContext(logicSQL, ProxyContext.getInstance().getSchemaContexts().getProps()).getExecutionUnits().iterator();
         queryHeaders = new ArrayList<>(2);
         queryHeaders.add(new QueryHeader("", "", "datasource_name", "", 255, Types.CHAR, 0, false, false, false, false));
         queryHeaders.add(new QueryHeader("", "", "sql", "", 255, Types.CHAR, 0, false, false, false, false));
         return new QueryResponse(queryHeaders);
     }
     
-    private LogicSQLContext createLogicSQLContext(final SchemaContext schemaContext, final ShardingCTLExplainStatement explainStatement) {
+    private LogicSQL createLogicSQL(final SchemaContext schemaContext, final ShardingCTLExplainStatement explainStatement) {
         SchemaMetaData schemaMetaData = schemaContext.getSchema().getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
         SQLStatement sqlStatement = schemaContext.getRuntimeContext().getSqlParserEngine().parse(explainStatement.getSql(), false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, Collections.emptyList(), sqlStatement);
-        return new LogicSQLContext(schemaContext, sqlStatementContext, explainStatement.getSql(), Collections.emptyList());
+        return new LogicSQL(schemaContext.getSchema(), sqlStatementContext, explainStatement.getSql(), Collections.emptyList());
     }
     
     @Override
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/internal/executor/HintShowTableStatusExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/internal/executor/HintShowTableStatusExecutor.java
index 388c93c..8af0b13 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/internal/executor/HintShowTableStatusExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/internal/executor/HintShowTableStatusExecutor.java
@@ -63,7 +63,7 @@ public final class HintShowTableStatusExecutor extends AbstractHintQueryExecutor
         if (null == schemaContext) {
             throw new NoDatabaseSelectedException();
         }
-        if (!schemaContext.isComplete()) {
+        if (!schemaContext.getSchema().isComplete()) {
             throw new RuleNotExistsException();
         }
         Collection<String> tableNames = schemaContext.getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData().getAllTableNames();
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 3d485c3..70f4dff 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
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilderTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilderTest.java
index 61bd099..066c5d2 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilderTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilderTest.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.proxy.backend.response.query;
 
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
 import org.apache.shardingsphere.infra.executor.sql.raw.execute.result.query.QueryHeader;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
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 67f9f42..1aefbf4 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
@@ -47,6 +47,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -91,8 +92,8 @@ public final class BroadcastBackendHandlerTest {
     private Map<String, SchemaContext> getSchemaContextMap() {
         Map<String, SchemaContext> result = new HashMap<>(10);
         for (int i = 0; i < 10; i++) {
-            SchemaContext schemaContext = mock(SchemaContext.class);
-            when(schemaContext.isComplete()).thenReturn(true);
+            SchemaContext schemaContext = mock(SchemaContext.class, RETURNS_DEEP_STUBS);
+            when(schemaContext.getSchema().isComplete()).thenReturn(true);
             result.put(String.format(SCHEMA_PATTERN, i), schemaContext);
         }
         return result;
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 3674ef7..3be2af2 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
@@ -43,6 +43,7 @@ import java.util.Properties;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -68,8 +69,8 @@ public final class ShowTablesBackendHandlerTest {
     private Map<String, SchemaContext> getSchemaContextMap() {
         Map<String, SchemaContext> result = new HashMap<>(10);
         for (int i = 0; i < 10; i++) {
-            SchemaContext context = mock(SchemaContext.class);
-            when(context.isComplete()).thenReturn(false);
+            SchemaContext context = mock(SchemaContext.class, RETURNS_DEEP_STUBS);
+            when(context.getSchema().isComplete()).thenReturn(false);
             result.put(String.format(SCHEMA_PATTERN, i), context);
         }
         return result;
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 625eb1d..9e51123 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
@@ -6,7 +6,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
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 af77495..0801f7d 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
@@ -24,14 +24,13 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.datasource.DataSourceMetaDatas;
 import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaData;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
@@ -50,7 +49,6 @@ import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.junit.MockitoJUnitRunner;
 
-import javax.sql.DataSource;
 import java.lang.reflect.Field;
 import java.sql.SQLException;
 import java.sql.Types;
@@ -211,8 +209,7 @@ public final class ShardingCTLHintBackendHandlerTest {
         ShardingSphereSchema shardingSphereSchema = mock(ShardingSphereSchema.class);
         when(shardingSphereSchema.getMetaData()).thenReturn(new ShardingSphereMetaData(mock(DataSourceMetaDatas.class),
                 new RuleSchemaMetaData(new SchemaMetaData(ImmutableMap.of("user", mock(TableMetaData.class))), Collections.emptyMap()), "sharding_db"));
-        when(shardingSphereSchema.getRules()).thenReturn(Collections.singleton(mock(ShardingSphereRule.class)));
-        when(shardingSphereSchema.getDataSources()).thenReturn(Collections.singletonMap("ds", mock(DataSource.class)));
+        when(shardingSphereSchema.isComplete()).thenReturn(true);
         SchemaContext result = new SchemaContext(shardingSphereSchema, null);
         return Collections.singletonMap("schema", result);
     }
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 e0c10fc..c3c56f4 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
@@ -32,7 +32,7 @@ import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.frontend.connection.ConnectionIdGenerator;
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 6007674..4c70ab5 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
@@ -28,7 +28,7 @@ import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.SchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.junit.Before;
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 dbc3baa..a5daf23 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
@@ -25,7 +25,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.context.schema.SchemaContext;
 import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.runtime.RuntimeContext;
-import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;