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/12 08:20:09 UTC

[shardingsphere] branch master updated: Decouple LogicSQL and schema (#7753)

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 61fb19d  Decouple LogicSQL and schema (#7753)
61fb19d is described below

commit 61fb19d54071592f0e4e70ea78bc7cd33ff8d1df
Author: Liang Zhang <te...@163.com>
AuthorDate: Mon Oct 12 16:19:42 2020 +0800

    Decouple LogicSQL and schema (#7753)
---
 .../engine/ConsensusReplicationSQLRouter.java      | 10 ++++----
 .../EncryptSQLRewriterParameterizedTest.java       |  6 ++---
 .../engine/PrimaryReplicaReplicationSQLRouter.java |  6 +++--
 .../PrimaryReplicaReplicationSQLRouterTest.java    | 28 +++++++++++-----------
 .../shadow/route/engine/ShadowSQLRouter.java       |  8 ++++---
 .../shadow/route/engine/ShadowSQLRouterTest.java   | 28 +++++++++++-----------
 .../MixSQLRewriterParameterizedTest.java           |  4 ++--
 .../ShardingSQLRewriterParameterizedTest.java      |  4 ++--
 .../sharding/route/engine/ShardingSQLRouter.java   | 26 ++++++++++----------
 .../engine/ShardingConditionEngineFactory.java     | 12 ++++++----
 .../engine/ShardingConditionEngineFactoryTest.java | 13 +++++-----
 .../engine/type/standard/AbstractSQLRouteTest.java |  4 ++--
 .../apache/shardingsphere/infra/sql/LogicSQL.java  |  3 ---
 .../infra/context/kernel/KernelProcessor.java      | 16 +++++++------
 .../infra/executor/sql/log/SQLLoggerTest.java      | 23 +++++++++---------
 .../shardingsphere/infra/route/SQLRouter.java      |  7 ++++--
 .../infra/route/UnconfiguredSchemaSQLRouter.java   |  6 +++--
 .../infra/route/engine/SQLRouteEngine.java         | 16 +++++++------
 .../infra/route/engine/SQLRouteEngineTest.java     |  8 +++----
 .../fixture/router/SQLRouterFailureFixture.java    |  6 +++--
 .../route/fixture/router/SQLRouterFixture.java     |  6 +++--
 .../statement/ShardingSpherePreparedStatement.java |  4 ++--
 .../core/statement/ShardingSphereStatement.java    |  7 +++---
 .../DatabaseCommunicationEngineFactory.java        |  8 ++++---
 .../jdbc/JDBCDatabaseCommunicationEngine.java      | 23 +++++++++---------
 .../explain/ShardingCTLExplainBackendHandler.java  |  4 ++--
 26 files changed, 154 insertions(+), 132 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 b024977..7149dc7 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
@@ -22,6 +22,7 @@ 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.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.replication.consensus.constant.ConsensusReplicationOrder;
 import org.apache.shardingsphere.replication.consensus.rule.ConsensusReplicationRule;
@@ -40,19 +41,20 @@ import java.util.Optional;
 public final class ConsensusReplicationSQLRouter implements SQLRouter<ConsensusReplicationRule> {
     
     @Override
-    public RouteContext createRouteContext(final LogicSQL logicSQL, final ConsensusReplicationRule rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingSphereSchema schema, final ConsensusReplicationRule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
         ConsensusReplicationTableRule tableRule = rule.getReplicaTableRules().iterator().next();
         ConsensusReplicationGroup replicaGroup = new ConsensusReplicationGroup(
                 tableRule.getPhysicsTable(), tableRule.getReplicaGroupId(), tableRule.getReplicaPeers(), tableRule.getDataSourceName());
         Map<String, ConsensusReplicationGroup> replicaGroups = Collections.singletonMap(ConsensusReplicationGroup.BLANK_CONSENSUS_REPLICATION_GROUP_KEY, replicaGroup);
         boolean isReadOnly = SQLUtil.isReadOnly(logicSQL.getSqlStatementContext().getSqlStatement());
-        result.getRouteStageContexts().put(getTypeClass(), new ConsensusReplicationRouteStageContext(logicSQL.getSchema().getName(), replicaGroups, isReadOnly));
+        result.getRouteStageContexts().put(getTypeClass(), new ConsensusReplicationRouteStageContext(schema.getName(), replicaGroups, isReadOnly));
         return result;
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final ConsensusReplicationRule rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, 
+                                     final LogicSQL logicSQL, final ShardingSphereSchema schema, final ConsensusReplicationRule rule, final ConfigurationProperties props) {
         Map<String, ConsensusReplicationGroup> replicaGroups = new HashMap<>();
         for (RouteUnit each : routeContext.getRouteUnits()) {
             Collection<RouteMapper> routeMappers = each.getTableMappers();
@@ -66,7 +68,7 @@ public final class ConsensusReplicationSQLRouter implements SQLRouter<ConsensusR
             }
         }
         boolean isReadOnly = SQLUtil.isReadOnly(logicSQL.getSqlStatementContext().getSqlStatement());
-        routeContext.getRouteStageContexts().put(getTypeClass(), new ConsensusReplicationRouteStageContext(logicSQL.getSchema().getName(), replicaGroups, isReadOnly));
+        routeContext.getRouteStageContexts().put(getTypeClass(), new ConsensusReplicationRouteStageContext(schema.getName(), replicaGroups, 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 35abb30..45acefc 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
@@ -77,12 +77,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));
-        LogicSQL logicSQL = new LogicSQL(schema, sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
-        RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL);
+        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
+        ShardingSphereSchema schema = new ShardingSphereSchema("sharding_db", Collections.emptyList(), rules, Collections.emptyMap(), metaData);
+        RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL, schema);
         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 afeeb88..fcd3f99 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
@@ -23,6 +23,7 @@ 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.schema.ShardingSphereSchema;
 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;
@@ -40,7 +41,7 @@ import java.util.Optional;
 public final class PrimaryReplicaReplicationSQLRouter implements SQLRouter<PrimaryReplicaReplicationRule> {
     
     @Override
-    public RouteContext createRouteContext(final LogicSQL logicSQL, final PrimaryReplicaReplicationRule rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingSphereSchema schema, final PrimaryReplicaReplicationRule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
         String dataSourceName = new PrimaryReplicaReplicationDataSourceRouter(rule.getSingleDataSourceRule()).route(logicSQL.getSqlStatementContext().getSqlStatement());
         result.getRouteUnits().add(new RouteUnit(new RouteMapper(DefaultSchema.LOGIC_NAME, dataSourceName), Collections.emptyList()));
@@ -48,7 +49,8 @@ public final class PrimaryReplicaReplicationSQLRouter implements SQLRouter<Prima
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final PrimaryReplicaReplicationRule rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, 
+                                     final LogicSQL logicSQL, final ShardingSphereSchema schema, final PrimaryReplicaReplicationRule rule, final ConfigurationProperties props) {
         Collection<RouteUnit> toBeRemoved = new LinkedList<>();
         Collection<RouteUnit> toBeAdded = new LinkedList<>();
         for (RouteUnit each : routeContext.getRouteUnits()) {
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 798158f..dd79926 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
@@ -89,9 +89,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
     
     @Test
     public void assertCreateRouteContextToPrimaryWithoutRouteUnits() {
+        LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "", Collections.emptyList());
         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()));
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATASOURCE));
     }
@@ -99,9 +99,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
     @Test
     public void assertDecorateRouteContextToPrimaryDataSource() {
         RouteContext actual = mockRouteContext();
+        LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "", Collections.emptyList());
         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()));
+        sqlRouter.decorateRouteContext(actual, logicSQL, schema, 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));
@@ -112,9 +112,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
         MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.empty());
+        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         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()));
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(REPLICA_DATASOURCE));
     }
@@ -125,9 +125,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
         MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
         when(selectStatement.getLock()).thenReturn(Optional.empty());
+        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         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()));
+        sqlRouter.decorateRouteContext(actual, logicSQL, schema, 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));
@@ -138,9 +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)));
+        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         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()));
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(PRIMARY_DATASOURCE));
     }
@@ -151,9 +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)));
+        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         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()));
+        sqlRouter.decorateRouteContext(actual, logicSQL, schema, 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));
@@ -162,9 +162,9 @@ public final class PrimaryReplicaReplicationSQLRouterTest {
     @Test
     public void assertCreateRouteContextToPrimaryDataSource() {
         when(sqlStatementContext.getSqlStatement()).thenReturn(mock(InsertStatement.class));
+        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         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()));
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, schema, 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 84318c7..ff57b99 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
@@ -17,18 +17,19 @@
 
 package org.apache.shardingsphere.shadow.route.engine;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 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.schema.ShardingSphereSchema;
 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;
 import org.apache.shardingsphere.shadow.route.engine.judge.impl.SimpleShadowDataSourceJudgeEngine;
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatement;
 
 import java.util.Collection;
@@ -42,7 +43,7 @@ import java.util.List;
 public final class ShadowSQLRouter implements SQLRouter<ShadowRule> {
     
     @Override
-    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShadowRule rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingSphereSchema schema, final ShadowRule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
         if (!(logicSQL.getSqlStatementContext().getSqlStatement() instanceof DMLStatement)) {
             rule.getShadowMappings().forEach((key, value) -> {
@@ -60,7 +61,8 @@ public final class ShadowSQLRouter implements SQLRouter<ShadowRule> {
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final ShadowRule rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, 
+                                     final LogicSQL logicSQL, final ShardingSphereSchema schema, final ShadowRule rule, final ConfigurationProperties props) {
         Collection<RouteUnit> toBeAdded = new LinkedList<>();
         if (!(logicSQL.getSqlStatementContext().getSqlStatement() instanceof DMLStatement)) {
             for (RouteUnit each : routeContext.getRouteUnits()) {
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 8a3a77d..fa8e4cb 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
@@ -76,27 +76,27 @@ public final class ShadowSQLRouterTest {
     
     @Test
     public void assertCreateRouteContextToShadowDataSource() {
+        LogicSQL logicSQL = new LogicSQL(mockSQLStatementContextForShadow(), "", Collections.emptyList());
         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()));
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(SHADOW_DATASOURCE));
     }
     
     @Test
     public void assertCreateRouteContextToActualDataSource() {
+        LogicSQL logicSQL = new LogicSQL(mockSQLStatementContext(), "", Collections.emptyList());
         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()));
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(ACTUAL_DATASOURCE));
     }
     
     @Test
     public void assertCreateRouteContextForNonDMLStatement() {
+        LogicSQL logicSQL = new LogicSQL(mockNonDMLSQLStatementContext(), "", Collections.emptyList());
         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()));
+        RouteContext actual = sqlRouter.createRouteContext(logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         assertThat(actual.getRouteUnits().size(), is(2));
         assertTrue(actual.getActualDataSourceNames().contains(SHADOW_DATASOURCE));
         assertTrue(actual.getActualDataSourceNames().contains(ACTUAL_DATASOURCE));
@@ -106,9 +106,9 @@ public final class ShadowSQLRouterTest {
     public void assertDecorateRouteContextToShadowDataSource() {
         RouteContext actual = new RouteContext();
         actual.getRouteUnits().add(mockRouteUnit());
+        LogicSQL logicSQL = new LogicSQL(mockSQLStatementContextForShadow(), "", Collections.emptyList());
         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()));
+        sqlRouter.decorateRouteContext(actual, logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         assertThat(actual.getRouteUnits().size(), is(1));
         assertTrue(actual.getActualDataSourceNames().contains(SHADOW_DATASOURCE));
     }
@@ -117,9 +117,9 @@ public final class ShadowSQLRouterTest {
     public void assertDecorateRouteContextToActualDataSource() {
         RouteContext actual = new RouteContext();
         actual.getRouteUnits().add(mockRouteUnit());
+        LogicSQL logicSQL = new LogicSQL(mockSQLStatementContext(), "", Collections.emptyList());
         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()));
+        sqlRouter.decorateRouteContext(actual, logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
         assertThat(routedDataSourceNames.next(), is(ACTUAL_DATASOURCE));
     }
@@ -128,9 +128,9 @@ public final class ShadowSQLRouterTest {
     public void assertDecorateRouteContextNonDMLStatement() {
         RouteContext actual = new RouteContext();
         actual.getRouteUnits().add(mockRouteUnit());
+        LogicSQL logicSQL = new LogicSQL(mockNonDMLSQLStatementContext(), "", Collections.emptyList());
         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()));
+        sqlRouter.decorateRouteContext(actual, logicSQL, schema, rule, new ConfigurationProperties(new Properties()));
         assertThat(actual.getRouteUnits().size(), is(2));
         assertTrue(actual.getActualDataSourceNames().contains(SHADOW_DATASOURCE));
         assertTrue(actual.getActualDataSourceNames().contains(ACTUAL_DATASOURCE));
@@ -140,9 +140,9 @@ public final class ShadowSQLRouterTest {
     public void assertDecorateRouteContextWithTableMapper() {
         RouteContext actual = new RouteContext();
         actual.getRouteUnits().add(mockRouteUnit());
+        LogicSQL logicSQL = new LogicSQL(mockSQLStatementContextForShadow(), "", Collections.emptyList());
         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()));
+        sqlRouter.decorateRouteContext(actual, logicSQL, schema, 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 b304773..6d996b5 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
@@ -87,9 +87,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));
+        LogicSQL logicSQL = new LogicSQL(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);
+        RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL, schema);
         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 cd80c7f..b328c43 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
@@ -87,9 +87,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));
+        LogicSQL logicSQL = new LogicSQL(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);
+        RouteContext routeContext = new SQLRouteEngine(props, rules).route(logicSQL, schema);
         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 8323e2b..ad4188b 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
@@ -18,10 +18,14 @@
 package org.apache.shardingsphere.sharding.route.engine;
 
 import com.google.common.base.Preconditions;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.sharding.api.config.strategy.sharding.HintShardingStrategyConfiguration;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
@@ -29,17 +33,14 @@ import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditi
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingConditions;
 import org.apache.shardingsphere.sharding.route.engine.condition.engine.ShardingConditionEngine;
 import org.apache.shardingsphere.sharding.route.engine.condition.engine.ShardingConditionEngineFactory;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
+import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
 import org.apache.shardingsphere.sharding.route.engine.type.ShardingRouteEngineFactory;
 import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidatorFactory;
 import org.apache.shardingsphere.sharding.rule.BindingTableRule;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sharding.route.engine.condition.value.ListShardingConditionValue;
-import org.apache.shardingsphere.sharding.route.engine.condition.value.ShardingConditionValue;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.util.SafeNumberOperationUtils;
@@ -55,27 +56,27 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
     
     @SuppressWarnings({"rawtypes", "unchecked"})
     @Override
-    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingRule rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingSphereSchema schema, final ShardingRule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
         SQLStatement sqlStatement = logicSQL.getSqlStatementContext().getSqlStatement();
         Optional<ShardingStatementValidator> validator = ShardingStatementValidatorFactory.newInstance(sqlStatement);
-        validator.ifPresent(v -> v.preValidate(rule, logicSQL.getSqlStatementContext(), logicSQL.getParameters(), logicSQL.getSchema().getMetaData()));
-        ShardingConditions shardingConditions = createShardingConditions(logicSQL, rule);
+        validator.ifPresent(v -> v.preValidate(rule, logicSQL.getSqlStatementContext(), logicSQL.getParameters(), schema.getMetaData()));
+        ShardingConditions shardingConditions = createShardingConditions(logicSQL, schema, rule);
         boolean needMergeShardingValues = isNeedMergeShardingValues(logicSQL.getSqlStatementContext(), rule);
         if (sqlStatement instanceof DMLStatement && needMergeShardingValues) {
             checkSubqueryShardingValues(logicSQL.getSqlStatementContext(), rule, shardingConditions);
             mergeShardingConditions(shardingConditions);
         }
-        ShardingRouteEngineFactory.newInstance(rule, logicSQL.getSchema().getMetaData(), logicSQL.getSqlStatementContext(), shardingConditions, props).route(result, rule);
+        ShardingRouteEngineFactory.newInstance(rule, schema.getMetaData(), logicSQL.getSqlStatementContext(), shardingConditions, props).route(result, rule);
         validator.ifPresent(v -> v.postValidate(sqlStatement, result));
         return result;
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
-    private ShardingConditions createShardingConditions(final LogicSQL logicSQL, final ShardingRule rule) {
+    private ShardingConditions createShardingConditions(final LogicSQL logicSQL, final ShardingSphereSchema schema, final ShardingRule rule) {
         List<ShardingCondition> shardingConditions;
         if (logicSQL.getSqlStatementContext().getSqlStatement() instanceof DMLStatement) {
-            ShardingConditionEngine shardingConditionEngine = ShardingConditionEngineFactory.createShardingConditionEngine(logicSQL, rule);
+            ShardingConditionEngine shardingConditionEngine = ShardingConditionEngineFactory.createShardingConditionEngine(logicSQL, schema, rule);
             shardingConditions = shardingConditionEngine.createShardingConditions(logicSQL.getSqlStatementContext(), logicSQL.getParameters());
         } else {
             shardingConditions = Collections.emptyList();
@@ -156,7 +157,8 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final ShardingRule rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, 
+                                     final LogicSQL logicSQL, final ShardingSphereSchema schema, final ShardingRule rule, final ConfigurationProperties props) {
         // TODO
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/ShardingConditionEngineFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/ShardingConditionEngineFactory.java
index 528b8cd..476f323 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/ShardingConditionEngineFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/ShardingConditionEngineFactory.java
@@ -19,14 +19,15 @@ package org.apache.shardingsphere.sharding.route.engine.condition.engine;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.binder.metadata.schema.SchemaMetaData;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.datetime.DatetimeService;
+import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
-import org.apache.shardingsphere.infra.datetime.DatetimeService;
 import org.apache.shardingsphere.sharding.route.engine.condition.engine.impl.InsertClauseShardingConditionEngine;
 import org.apache.shardingsphere.sharding.route.engine.condition.engine.impl.WhereClauseShardingConditionEngine;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.infra.binder.metadata.schema.SchemaMetaData;
-import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 
 /**
  * Sharding condition engine factory.
@@ -42,11 +43,12 @@ public final class ShardingConditionEngineFactory {
      * Create new instance of sharding condition engine.
      *
      * @param logicSQL logic SQL
+     * @param schema ShardingSphere schema
      * @param rule sharding rule 
      * @return sharding condition engine
      */
-    public static ShardingConditionEngine<?> createShardingConditionEngine(final LogicSQL logicSQL, final ShardingRule rule) {
-        SchemaMetaData schemaMetaData = logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData();
+    public static ShardingConditionEngine<?> createShardingConditionEngine(final LogicSQL logicSQL, final ShardingSphereSchema schema, final ShardingRule rule) {
+        SchemaMetaData schemaMetaData = schema.getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData();
         return logicSQL.getSqlStatementContext() instanceof InsertStatementContext
                 ? new InsertClauseShardingConditionEngine(rule, schemaMetaData) : new WhereClauseShardingConditionEngine(rule, schemaMetaData);
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/ShardingConditionEngineFactoryTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/ShardingConditionEngineFactoryTest.java
index f5f4760..ea6934b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/ShardingConditionEngineFactoryTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/condition/engine/ShardingConditionEngineFactoryTest.java
@@ -44,12 +44,13 @@ public final class ShardingConditionEngineFactoryTest {
     private LogicSQL logicSQL;
     
     @Mock
+    private ShardingSphereSchema shardingSphereSchema;
+    
+    @Mock
     private ShardingRule shardingRule;
     
     @Before
     public void setUp() {
-        ShardingSphereSchema shardingSphereSchema = mock(ShardingSphereSchema.class);
-        when(logicSQL.getSchema()).thenReturn(shardingSphereSchema);
         ShardingSphereMetaData shardingSphereMetaData = mock(ShardingSphereMetaData.class);
         when(shardingSphereSchema.getMetaData()).thenReturn(shardingSphereMetaData);
         RuleSchemaMetaData ruleSchemaMetaData = mock(RuleSchemaMetaData.class);
@@ -62,13 +63,13 @@ public final class ShardingConditionEngineFactoryTest {
     public void assertCreateInsertClauseShardingConditionEngine() {
         SQLStatementContext insertStatementContext = mock(InsertStatementContext.class);
         when(logicSQL.getSqlStatementContext()).thenReturn(insertStatementContext);
-        ShardingConditionEngine<?> actualInsertClauseShardingConditionEngine = ShardingConditionEngineFactory.createShardingConditionEngine(logicSQL, shardingRule);
-        assertTrue(actualInsertClauseShardingConditionEngine instanceof InsertClauseShardingConditionEngine);
+        ShardingConditionEngine<?> actual = ShardingConditionEngineFactory.createShardingConditionEngine(logicSQL, shardingSphereSchema, shardingRule);
+        assertTrue(actual instanceof InsertClauseShardingConditionEngine);
     }
     
     @Test
     public void assertCreateWhereClauseShardingConditionEngine() {
-        ShardingConditionEngine<?> actualWhereClauseShardingConditionEngine = ShardingConditionEngineFactory.createShardingConditionEngine(logicSQL, shardingRule);
-        assertTrue(actualWhereClauseShardingConditionEngine instanceof WhereClauseShardingConditionEngine);
+        ShardingConditionEngine<?> actual = ShardingConditionEngineFactory.createShardingConditionEngine(logicSQL, shardingSphereSchema, shardingRule);
+        assertTrue(actual instanceof WhereClauseShardingConditionEngine);
     }
 }
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 fb31802..15e0a14 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
@@ -60,9 +60,9 @@ public abstract class AbstractSQLRouteTest extends AbstractRoutingEngineTest {
         StandardSQLParserEngine standardSqlParserEngine = SQLParserEngineFactory.getSQLParserEngine("MySQL");
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
                 metaData.getRuleSchemaMetaData().getConfiguredSchemaMetaData(), parameters, standardSqlParserEngine.parse(sql, false));
+        LogicSQL logicSQL = new LogicSQL(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);
+        RouteContext result = new SQLRouteEngine(props, Collections.singletonList(shardingRule)).route(logicSQL, schema);
         assertThat(result.getRouteUnits().size(), is(1));
         return result;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/sql/LogicSQL.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/sql/LogicSQL.java
index 9624154..7189249 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/sql/LogicSQL.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/sql/LogicSQL.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.infra.sql;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 
 import java.util.List;
@@ -31,8 +30,6 @@ import java.util.List;
 @Getter
 public final class LogicSQL {
     
-    private final ShardingSphereSchema schema;
-    
     private final SQLStatementContext<?> sqlStatementContext;
     
     private final String sql;
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 8aaeb33..e6091c2 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
@@ -17,8 +17,8 @@
 
 package org.apache.shardingsphere.infra.context.kernel;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
-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;
@@ -27,7 +27,8 @@ import org.apache.shardingsphere.infra.rewrite.engine.result.SQLRewriteResult;
 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.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.sql.LogicSQL;
 
 import java.util.Collection;
 
@@ -40,17 +41,18 @@ public final class KernelProcessor {
      * Generate execution context.
      *
      * @param logicSQL logic SQL
+     * @param schema ShardingSphere schema
      * @param props configuration properties
      * @return execution context
      */
-    public ExecutionContext generateExecutionContext(final LogicSQL logicSQL, final ConfigurationProperties props) {
-        Collection<ShardingSphereRule> rules = logicSQL.getSchema().getRules();
+    public ExecutionContext generateExecutionContext(final LogicSQL logicSQL, final ShardingSphereSchema schema, final ConfigurationProperties props) {
+        Collection<ShardingSphereRule> rules = schema.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);
+        RouteContext routeContext = sqlRouteEngine.route(logicSQL, schema);
+        SQLRewriteEntry rewriteEntry = new SQLRewriteEntry(schema.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);
+        Collection<ExecutionUnit> executionUnits = ExecutionContextBuilder.build(schema.getMetaData(), rewriteResult, sqlStatementContext);
         return new ExecutionContext(sqlStatementContext, executionUnits, routeContext);
     }
 }
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 35e8aa5..2e7bdd6 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
@@ -22,7 +22,6 @@ 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.route.context.RouteContext;
-import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,7 +49,7 @@ public final class SQLLoggerTest {
     
     private static final String SQL = "SELECT * FROM t_user";
     
-    private final LogicSQL logicSQL = new LogicSQL(mock(ShardingSphereSchema.class), mock(SQLStatementContext.class), SQL, Collections.emptyList());
+    private final LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), SQL, Collections.emptyList());
     
     private Collection<ExecutionUnit> executionUnits;
     
@@ -71,11 +70,11 @@ public final class SQLLoggerTest {
     public void assertLogNormalSQLWithoutParameter() {
         SQLLogger.logSQL(logicSQL, false, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
-        inOrder.verify(logger).info("Logic SQL: {}", SQL);
+        inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
-        inOrder.verify(logger).info("Actual SQL: {} ::: {}", "db1", SQL);
-        inOrder.verify(logger).info("Actual SQL: {} ::: {}", "db2", SQL);
-        inOrder.verify(logger).info("Actual SQL: {} ::: {}", "db3", SQL);
+        inOrder.verify(logger).info("Actual SQL: {} ::: {}", new Object[]{"db1", SQL});
+        inOrder.verify(logger).info("Actual SQL: {} ::: {}", new Object[]{"db2", SQL});
+        inOrder.verify(logger).info("Actual SQL: {} ::: {}", new Object[]{"db3", SQL});
     }
     
     @Test
@@ -84,20 +83,20 @@ public final class SQLLoggerTest {
         parameters.add("parameter");
         SQLLogger.logSQL(logicSQL, false, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
-        inOrder.verify(logger).info("Logic SQL: {}", SQL);
+        inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
-        inOrder.verify(logger).info("Actual SQL: {} ::: {} ::: {}", "db1", SQL, parameters);
-        inOrder.verify(logger).info("Actual SQL: {} ::: {}", "db2", SQL);
-        inOrder.verify(logger).info("Actual SQL: {} ::: {}", "db3", SQL);
+        inOrder.verify(logger).info("Actual SQL: {} ::: {} ::: {}", new Object[]{"db1", SQL, parameters});
+        inOrder.verify(logger).info("Actual SQL: {} ::: {}", new Object[]{"db2", SQL});
+        inOrder.verify(logger).info("Actual SQL: {} ::: {}", new Object[]{"db3", SQL});
     }
     
     @Test
     public void assertLogSimpleSQL() {
         SQLLogger.logSQL(logicSQL, true, new ExecutionContext(null, executionUnits, mock(RouteContext.class)));
         InOrder inOrder = inOrder(logger);
-        inOrder.verify(logger).info("Logic SQL: {}", SQL);
+        inOrder.verify(logger).info("Logic SQL: {}", new Object[]{SQL});
         inOrder.verify(logger).info("SQLStatement: {}", new Object[]{null});
-        inOrder.verify(logger).info("Actual SQL(simple): {} ::: {}", buildDataSourceNames(), executionUnits.size());
+        inOrder.verify(logger).info("Actual SQL(simple): {} ::: {}", new Object[]{buildDataSourceNames(), executionUnits.size()});
     }
     
     private Collection<String> buildDataSourceNames() {
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 75006fc..1e0e7a2 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
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.infra.route;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPI;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
 
@@ -34,19 +35,21 @@ public interface SQLRouter<T extends ShardingSphereRule> extends OrderedSPI<T> {
      * Create route context.
      *
      * @param logicSQL logic SQL
+     * @param schema ShardingSphere schema
      * @param rule rule
      * @param props configuration properties
      * @return route context
      */
-    RouteContext createRouteContext(LogicSQL logicSQL, T rule, ConfigurationProperties props);
+    RouteContext createRouteContext(LogicSQL logicSQL, ShardingSphereSchema schema, T rule, ConfigurationProperties props);
     
     /**
      * Decorate route context.
      * 
      * @param routeContext route context
      * @param logicSQL logic SQL
+     * @param schema ShardingSphere schema
      * @param rule rule
      * @param props configuration properties
      */
-    void decorateRouteContext(RouteContext routeContext, LogicSQL logicSQL, T rule, ConfigurationProperties props);
+    void decorateRouteContext(RouteContext routeContext, LogicSQL logicSQL, ShardingSphereSchema schema, 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 27e3177..9fa7c7e 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
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.infra.route;
 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.schema.ShardingSphereSchema;
 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,10 +37,11 @@ public final class UnconfiguredSchemaSQLRouter {
      *
      * @param routeContext route context
      * @param logicSQL logic SQL
+     * @param schema ShardingSphere schema
      */
-    public void decorate(final RouteContext routeContext, final LogicSQL logicSQL) {
+    public void decorate(final RouteContext routeContext, final LogicSQL logicSQL, final ShardingSphereSchema schema) {
         if (isNeedUnconfiguredSchema(logicSQL.getSqlStatementContext().getSqlStatement())) {
-            for (String each : logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getUnconfiguredSchemaMetaDataMap().keySet()) {
+            for (String each : schema.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 d956bc3..7cc84fa 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
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.route.UnconfiguredSchemaSQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.hook.SPIRoutingHook;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
@@ -57,13 +58,14 @@ public final class SQLRouteEngine {
      * Route SQL.
      *
      * @param logicSQL logic SQL
+     * @param schema ShardingSphere schema
      * @return route context
      */
-    public RouteContext route(final LogicSQL logicSQL) {
+    public RouteContext route(final LogicSQL logicSQL, final ShardingSphereSchema schema) {
         routingHook.start(logicSQL.getSql());
         try {
-            RouteContext result = doRoute(logicSQL);
-            routingHook.finishSuccess(result, logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData());
+            RouteContext result = doRoute(logicSQL, schema);
+            routingHook.finishSuccess(result, schema.getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData());
             return result;
             // CHECKSTYLE:OFF
         } catch (final Exception ex) {
@@ -74,16 +76,16 @@ public final class SQLRouteEngine {
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private RouteContext doRoute(final LogicSQL logicSQL) {
+    private RouteContext doRoute(final LogicSQL logicSQL, final ShardingSphereSchema schema) {
         RouteContext result = new RouteContext();
         for (Entry<ShardingSphereRule, SQLRouter> entry : routers.entrySet()) {
             if (result.getRouteUnits().isEmpty()) {
-                result = entry.getValue().createRouteContext(logicSQL, entry.getKey(), props);
+                result = entry.getValue().createRouteContext(logicSQL, schema, entry.getKey(), props);
             } else {
-                entry.getValue().decorateRouteContext(result, logicSQL, entry.getKey(), props);
+                entry.getValue().decorateRouteContext(result, logicSQL, schema, entry.getKey(), props);
             }
         }
-        new UnconfiguredSchemaSQLRouter().decorate(result, logicSQL);
+        new UnconfiguredSchemaSQLRouter().decorate(result, logicSQL, schema);
         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 beb8228..79c6cfb 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
@@ -65,11 +65,11 @@ public final class SQLRouteEngineTest {
     
     @Test
     public void assertRouteSuccess() {
+        LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "SELECT 1", Collections.emptyList());
         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(logicSQL);
+        RouteContext actual = sqlRouteEngine.route(logicSQL, schema);
         assertThat(actual.getRouteUnits().size(), is(1));
         RouteUnit routeUnit = actual.getRouteUnits().iterator().next();
         assertThat(routeUnit.getDataSourceMapper().getLogicName(), is("ds"));
@@ -81,12 +81,12 @@ public final class SQLRouteEngineTest {
     
     @Test(expected = UnsupportedOperationException.class)
     public void assertRouteFailure() {
+        LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "SELECT 1", Collections.emptyList());
         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(logicSQL);
+            sqlRouteEngine.route(logicSQL, schema);
         } 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 50fb721..7e9fa05 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
@@ -21,17 +21,19 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 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.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
 
 public final class SQLRouterFailureFixture implements SQLRouter<RouteFailureRuleFixture> {
     
     @Override
-    public RouteContext createRouteContext(final LogicSQL logicSQL, final RouteFailureRuleFixture rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingSphereSchema schema, final RouteFailureRuleFixture rule, final ConfigurationProperties props) {
         throw new UnsupportedOperationException("Route failure.");
     }
     
     @Override
-    public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final RouteFailureRuleFixture rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, 
+                                     final LogicSQL logicSQL, final ShardingSphereSchema schema, 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 25d8850..dbcd338 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
@@ -23,6 +23,7 @@ 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.fixture.rule.RouteRuleFixture;
+import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
 
 import java.util.Collections;
@@ -30,14 +31,15 @@ import java.util.Collections;
 public final class SQLRouterFixture implements SQLRouter<RouteRuleFixture> {
     
     @Override
-    public RouteContext createRouteContext(final LogicSQL logicSQL, final RouteRuleFixture rule, final ConfigurationProperties props) {
+    public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingSphereSchema schema, 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 LogicSQL logicSQL, final RouteRuleFixture rule, final ConfigurationProperties props) {
+    public void decorateRouteContext(final RouteContext routeContext, 
+                                     final LogicSQL logicSQL, final ShardingSphereSchema schema, 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/jdbc/core/statement/ShardingSpherePreparedStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 265626c..c353af3 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
@@ -260,7 +260,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     
     private ExecutionContext createExecutionContext() {
         LogicSQL logicSQL = createLogicSQL();
-        ExecutionContext result = kernelProcessor.generateExecutionContext(logicSQL, schemaContexts.getProps());
+        ExecutionContext result = kernelProcessor.generateExecutionContext(logicSQL, schemaContexts.getDefaultSchema(), schemaContexts.getProps());
         findGeneratedKey(result).ifPresent(generatedKey -> generatedValues.addAll(generatedKey.getGeneratedValues()));
         logSQL(logicSQL, result);
         return result;
@@ -270,7 +270,7 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         List<Object> parameters = new ArrayList<>(getParameters());
         SchemaMetaData schemaMetaData = schemaContexts.getDefaultSchema().getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, parameters, sqlStatement);
-        return new LogicSQL(schemaContexts.getDefaultSchema(), sqlStatementContext, sql, parameters);
+        return new LogicSQL(sqlStatementContext, sql, parameters);
     }
     
     private MergedResult mergeQuery(final List<QueryResult> queryResults) throws SQLException {
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 284cc21..e9e967f 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
@@ -284,7 +284,7 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     private ExecutionContext createExecutionContext(final String sql) throws SQLException {
         clearStatements();
         LogicSQL logicSQL = createLogicSQL(sql);
-        ExecutionContext result = kernelProcessor.generateExecutionContext(logicSQL, schemaContexts.getProps());
+        ExecutionContext result = kernelProcessor.generateExecutionContext(logicSQL, schemaContexts.getDefaultSchema(), schemaContexts.getProps());
         logSQL(logicSQL, schemaContexts.getProps(), result);
         return result;
     }
@@ -303,11 +303,10 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     }
     
     private LogicSQL createLogicSQL(final String sql) {
-        ShardingSphereSchema schema = schemaContexts.getDefaultSchema();
-        SchemaMetaData schemaMetaData = schema.getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
+        SchemaMetaData schemaMetaData = schemaContexts.getDefaultSchema().getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
         SQLStatement sqlStatement = schemaContexts.getSqlParserEngine().parse(sql, false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, Collections.emptyList(), sqlStatement);
-        return new LogicSQL(schema, sqlStatementContext, sql, Collections.emptyList());
+        return new LogicSQL(sqlStatementContext, sql, Collections.emptyList());
     }
     
     private Collection<InputGroup<StatementExecuteUnit>> getInputGroups() throws SQLException {
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 92ab4c4..2c8de4e 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
@@ -63,7 +63,8 @@ public final class DatabaseCommunicationEngineFactory {
      */
     public DatabaseCommunicationEngine newTextProtocolInstance(final SQLStatement sqlStatement, final String sql, final BackendConnection backendConnection) {
         LogicSQL logicSQL = createLogicSQL(sqlStatement, sql, Collections.emptyList(), backendConnection);
-        return new JDBCDatabaseCommunicationEngine(logicSQL, new JDBCExecuteEngine(backendConnection, new StatementAccessor()));
+        return new JDBCDatabaseCommunicationEngine(
+                logicSQL, ProxyContext.getInstance().getSchema(backendConnection.getSchemaName()), new JDBCExecuteEngine(backendConnection, new StatementAccessor()));
     }
     
     /**
@@ -77,13 +78,14 @@ public final class DatabaseCommunicationEngineFactory {
      */
     public DatabaseCommunicationEngine newBinaryProtocolInstance(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final BackendConnection backendConnection) {
         LogicSQL logicSQL = createLogicSQL(sqlStatement, sql, new ArrayList<>(parameters), backendConnection);
-        return new JDBCDatabaseCommunicationEngine(logicSQL, new JDBCExecuteEngine(backendConnection, new PreparedStatementAccessor()));
+        return new JDBCDatabaseCommunicationEngine(
+                logicSQL, ProxyContext.getInstance().getSchema(backendConnection.getSchemaName()), new JDBCExecuteEngine(backendConnection, new PreparedStatementAccessor()));
     }
     
     private LogicSQL createLogicSQL(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final BackendConnection backendConnection) {
         ShardingSphereSchema schema = ProxyContext.getInstance().getSchema(backendConnection.getSchemaName());
         SchemaMetaData schemaMetaData = schema.getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, parameters, sqlStatement);
-        return new LogicSQL(schema, sqlStatementContext, sql, parameters);
+        return new LogicSQL(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 4265cd8..82eebf9 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,6 +34,7 @@ import org.apache.shardingsphere.infra.metadata.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.infra.metadata.refresh.MetaDataRefreshStrategyFactory;
 import org.apache.shardingsphere.infra.metadata.schema.RuleSchemaMetaDataLoader;
 import org.apache.shardingsphere.infra.rule.DataNodeRoutedRule;
+import org.apache.shardingsphere.infra.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.sql.LogicSQL;
 import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.SQLExecuteEngine;
@@ -57,6 +58,8 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
     
     private final LogicSQL logicSQL;
     
+    private final ShardingSphereSchema schema;
+    
     private final SQLExecuteEngine sqlExecuteEngine;
     
     private final KernelProcessor kernelProcessor = new KernelProcessor();
@@ -67,7 +70,7 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
     
     @Override
     public BackendResponse execute() throws SQLException {
-        ExecutionContext executionContext = kernelProcessor.generateExecutionContext(logicSQL, ProxyContext.getInstance().getSchemaContexts().getProps());
+        ExecutionContext executionContext = kernelProcessor.generateExecutionContext(logicSQL, schema, ProxyContext.getInstance().getSchemaContexts().getProps());
         logSQL(executionContext);
         return doExecute(executionContext);
     }
@@ -95,17 +98,15 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
         }
         Optional<MetaDataRefreshStrategy> refreshStrategy = MetaDataRefreshStrategyFactory.newInstance(sqlStatement);
         if (refreshStrategy.isPresent()) {
-            refreshStrategy.get().refreshMetaData(logicSQL.getSchema().getMetaData(), ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
-                    logicSQL.getSchema().getDataSources(), sqlStatement, this::loadTableMetaData);
-            GovernanceEventBus.getInstance().post(
-                    new MetaDataPersistEvent(logicSQL.getSchema().getName(), logicSQL.getSchema().getMetaData().getRuleSchemaMetaData()));
+            refreshStrategy.get().refreshMetaData(
+                    schema.getMetaData(), ProxyContext.getInstance().getSchemaContexts().getDatabaseType(), schema.getDataSources(), sqlStatement, this::loadTableMetaData);
+            GovernanceEventBus.getInstance().post(new MetaDataPersistEvent(schema.getName(), schema.getMetaData().getRuleSchemaMetaData()));
         }
     }
     
     private Optional<TableMetaData> loadTableMetaData(final String tableName) throws SQLException {
-        RuleSchemaMetaDataLoader loader = new RuleSchemaMetaDataLoader(logicSQL.getSchema().getRules());
-        return loader.load(ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
-                logicSQL.getSchema().getDataSources(), tableName, ProxyContext.getInstance().getSchemaContexts().getProps());
+        RuleSchemaMetaDataLoader loader = new RuleSchemaMetaDataLoader(schema.getRules());
+        return loader.load(ProxyContext.getInstance().getSchemaContexts().getDatabaseType(), schema.getDataSources(), tableName, ProxyContext.getInstance().getSchemaContexts().getProps());
     }
     
     private BackendResponse merge(final SQLStatementContext<?> sqlStatementContext) throws SQLException {
@@ -125,14 +126,14 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
     
     private boolean isNeedAccumulate(final SQLStatementContext<?> sqlStatementContext) {
         Optional<DataNodeRoutedRule> dataNodeRoutedRule =
-                logicSQL.getSchema().getRules().stream().filter(each -> each instanceof DataNodeRoutedRule).findFirst().map(rule -> (DataNodeRoutedRule) rule);
+                schema.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(),
-                logicSQL.getSchema().getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData(), 
-                ProxyContext.getInstance().getSchemaContexts().getProps(), logicSQL.getSchema().getRules());
+                schema.getMetaData().getRuleSchemaMetaData().getConfiguredSchemaMetaData(), 
+                ProxyContext.getInstance().getSchemaContexts().getProps(), schema.getRules());
         return mergeEngine.merge(queryResults, sqlStatementContext);
     }
     
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 80110e8..d37cce6 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
@@ -74,7 +74,7 @@ public final class ShardingCTLExplainBackendHandler implements TextProtocolBacke
             throw new RuleNotExistsException();
         }
         LogicSQL logicSQL = createLogicSQL(schema, explainStatement.get());
-        executionUnits = kernelProcessor.generateExecutionContext(logicSQL, ProxyContext.getInstance().getSchemaContexts().getProps()).getExecutionUnits().iterator();
+        executionUnits = kernelProcessor.generateExecutionContext(logicSQL, schema, 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));
@@ -85,7 +85,7 @@ public final class ShardingCTLExplainBackendHandler implements TextProtocolBacke
         SchemaMetaData schemaMetaData = schema.getMetaData().getRuleSchemaMetaData().getSchemaMetaData();
         SQLStatement sqlStatement = ProxyContext.getInstance().getSchemaContexts().getSqlParserEngine().parse(explainStatement.getSql(), false);
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schemaMetaData, Collections.emptyList(), sqlStatement);
-        return new LogicSQL(schema, sqlStatementContext, explainStatement.getSql(), Collections.emptyList());
+        return new LogicSQL(sqlStatementContext, explainStatement.getSql(), Collections.emptyList());
     }
     
     @Override