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 2022/01/27 14:34:05 UTC

[shardingsphere] branch master updated: Remove SchemaAvailable interface and add schemaName param for encrypt (#15120)

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 f0fa941  Remove SchemaAvailable interface and add schemaName param for encrypt (#15120)
f0fa941 is described below

commit f0fa941e502f04256e054a726ab0581880d89f9d
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Thu Jan 27 22:33:01 2022 +0800

    Remove SchemaAvailable interface and add schemaName param for encrypt (#15120)
---
 .../merge/EncryptResultDecoratorEngine.java        |  4 +-
 .../merge/dql/EncryptAlgorithmMetaData.java        |  3 +-
 .../encrypt/rewrite/aware/SchemaNameAware.java     | 12 ++---
 .../rewrite/condition/EncryptConditionEngine.java  | 16 +++---
 .../context/EncryptSQLRewriteContextDecorator.java | 10 ++--
 .../parameter/EncryptParameterRewriterBuilder.java |  6 +++
 .../EncryptAssignmentParameterRewriter.java        |  7 +--
 ...OnDuplicateKeyUpdateValueParameterRewriter.java |  6 ++-
 .../EncryptInsertValueParameterRewriter.java       |  6 ++-
 .../EncryptPredicateParameterRewriter.java         |  7 +--
 .../rewrite/token/EncryptTokenGenerateBuilder.java |  6 +++
 .../generator/EncryptAssignmentTokenGenerator.java |  7 +--
 .../EncryptInsertOnUpdateTokenGenerator.java       |  6 ++-
 .../EncryptInsertValuesTokenGenerator.java         |  7 ++-
 .../EncryptPredicateRightValueTokenGenerator.java  |  7 +--
 .../generator/EncryptProjectionTokenGenerator.java | 10 ----
 .../merge/EncryptResultDecoratorEngineTest.java    |  7 +--
 .../merge/dql/EncryptAlgorithmMetaDataTest.java    | 13 ++---
 .../EncryptParameterRewriterBuilderTest.java       |  7 ++-
 .../token/EncryptTokenGenerateBuilderTest.java     |  4 +-
 .../dml/ShardingDeleteStatementValidatorTest.java  |  3 +-
 .../dml/ShardingUpdateStatementValidatorTest.java  | 11 ++---
 .../infra/binder/SQLStatementContextFactory.java   |  8 +--
 .../binder/statement/dml/CallStatementContext.java | 10 +---
 .../binder/statement/dml/CopyStatementContext.java | 10 +---
 .../statement/dml/DeleteStatementContext.java      |  8 +--
 .../statement/dml/InsertStatementContext.java      |  9 ++--
 .../statement/dml/SelectStatementContext.java      | 12 ++---
 .../statement/dml/UpdateStatementContext.java      |  8 +--
 .../dml/util/DMLStatementContextHelper.java        | 57 ----------------------
 .../statement/dml/CallStatementContextTest.java    |  3 +-
 .../statement/dml/DeleteStatementContextTest.java  |  3 +-
 .../statement/dml/UpdateStatementContextTest.java  |  8 +--
 .../infra/context/kernel/KernelProcessor.java      |  4 +-
 .../shardingsphere/infra/merge/MergeEngine.java    |  4 +-
 .../engine/decorator/ResultDecoratorEngine.java    |  3 +-
 .../decorator/ResultDecoratorEngineFixture.java    |  4 +-
 .../infra/rewrite/SQLRewriteEntry.java             |  7 ++-
 .../infra/rewrite/context/SQLRewriteContext.java   |  5 +-
 .../infra/rewrite/SQLRewriteEntryTest.java         |  5 +-
 .../rewrite/context/SQLRewriteContextTest.java     | 13 +++--
 .../engine/GenericSQLRewriteEngineTest.java        |  8 +--
 .../rewrite/engine/RouteSQLRewriteEngineTest.java  | 19 +++++---
 .../AbstractSQLRewriterParameterizedTest.java      |  4 +-
 44 files changed, 166 insertions(+), 211 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/merge/EncryptResultDecoratorEngine.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/merge/EncryptResultDecoratorEngine.java
index 004e036..e1697b9 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/merge/EncryptResultDecoratorEngine.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/merge/EncryptResultDecoratorEngine.java
@@ -38,10 +38,10 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatemen
 public final class EncryptResultDecoratorEngine implements ResultDecoratorEngine<EncryptRule> {
     
     @Override
-    public ResultDecorator newInstance(final DatabaseType databaseType, final ShardingSphereSchema schema,
+    public ResultDecorator newInstance(final DatabaseType databaseType, final String schemaName, final ShardingSphereSchema schema,
                                        final EncryptRule encryptRule, final ConfigurationProperties props, final SQLStatementContext sqlStatementContext) {
         if (sqlStatementContext instanceof SelectStatementContext) {
-            EncryptAlgorithmMetaData metaData = new EncryptAlgorithmMetaData(schema, encryptRule, (SelectStatementContext) sqlStatementContext);
+            EncryptAlgorithmMetaData metaData = new EncryptAlgorithmMetaData(schemaName, schema, encryptRule, (SelectStatementContext) sqlStatementContext);
             return new EncryptDQLResultDecorator(metaData);
         } 
         if (sqlStatementContext.getSqlStatement() instanceof DALStatement) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaData.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaData.java
index c24ecc0..0837126 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaData.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaData.java
@@ -36,6 +36,8 @@ import java.util.Optional;
 @RequiredArgsConstructor
 public final class EncryptAlgorithmMetaData {
     
+    private final String schemaName;
+    
     private final ShardingSphereSchema schema;
     
     private final EncryptRule encryptRule;
@@ -55,7 +57,6 @@ public final class EncryptAlgorithmMetaData {
         }
         Map<String, String> columnTableNames = selectStatementContext.getTablesContext().findTableName(Collections.singletonList(columnProjection.get()), schema);
         Optional<String> tableName = Optional.ofNullable(columnTableNames.get(columnProjection.get().getExpression()));
-        String schemaName = selectStatementContext.getSchemaName();
         String columnName = columnProjection.get().getName();
         return tableName.isPresent() ? findEncryptor(schemaName, tableName.get(), columnName) : findEncryptor(schemaName, columnName);
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/type/SchemaAvailable.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/aware/SchemaNameAware.java
similarity index 79%
rename from shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/type/SchemaAvailable.java
rename to shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/aware/SchemaNameAware.java
index 00111bd..fa9a4f5 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/type/SchemaAvailable.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/aware/SchemaNameAware.java
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.type;
+package org.apache.shardingsphere.encrypt.rewrite.aware;
 
 /**
- * Schema available.
+ * Schema name aware.
  */
-public interface SchemaAvailable {
+public interface SchemaNameAware {
     
     /**
-     * Get schema name.
+     * Set schema name.
      * 
-     * @return current schema name.
+     * @param schemaName schema name
      */
-    String getSchemaName();
+    void setSchemaName(String schemaName);
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/condition/EncryptConditionEngine.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/condition/EncryptConditionEngine.java
index d3d3c65..1a576c8 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/condition/EncryptConditionEngine.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/condition/EncryptConditionEngine.java
@@ -23,7 +23,6 @@ import org.apache.shardingsphere.encrypt.rewrite.condition.impl.EncryptInConditi
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ColumnProjection;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.util.DMLStatementContextHelper;
 import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
@@ -60,6 +59,8 @@ public final class EncryptConditionEngine {
     
     private final EncryptRule encryptRule;
     
+    private final String schemaName;
+    
     private final ShardingSphereSchema schema;
     
     static {
@@ -82,26 +83,25 @@ public final class EncryptConditionEngine {
         for (WhereSegment each : whereSegments) {
             Collection<AndPredicate> andPredicates = ExpressionExtractUtil.getAndPredicates(each.getExpr());
             Map<String, String> columnTableNames = getColumnTableNames(sqlStatementContext, andPredicates);
-            String schemaName = DMLStatementContextHelper.getSchemaName(sqlStatementContext);
             for (AndPredicate predicate : andPredicates) {
-                result.addAll(createEncryptConditions(schemaName, predicate.getPredicates(), columnTableNames));
+                result.addAll(createEncryptConditions(predicate.getPredicates(), columnTableNames));
             }
         }
         return result;
     }
     
-    private Collection<EncryptCondition> createEncryptConditions(final String schemaName, final Collection<ExpressionSegment> predicates, final Map<String, String> columnTableNames) {
+    private Collection<EncryptCondition> createEncryptConditions(final Collection<ExpressionSegment> predicates, final Map<String, String> columnTableNames) {
         Collection<EncryptCondition> result = new LinkedList<>();
         Collection<Integer> stopIndexes = new HashSet<>();
         for (ExpressionSegment each : predicates) {
             if (stopIndexes.add(each.getStopIndex())) {
-                result.addAll(createEncryptConditions(schemaName, each, columnTableNames));
+                result.addAll(createEncryptConditions(each, columnTableNames));
             }
         }
         return result;
     }
     
-    private Collection<EncryptCondition> createEncryptConditions(final String schemaName, final ExpressionSegment expression, final Map<String, String> columnTableNames) {
+    private Collection<EncryptCondition> createEncryptConditions(final ExpressionSegment expression, final Map<String, String> columnTableNames) {
         Collection<EncryptCondition> result = new LinkedList<>();
         for (ColumnSegment each : ColumnExtractor.extract(expression)) {
             ColumnProjection projection = buildColumnProjection(each);
@@ -115,7 +115,7 @@ public final class EncryptConditionEngine {
     
     private Optional<EncryptCondition> createEncryptCondition(final ExpressionSegment expression, final String tableName) {
         if (expression instanceof BinaryOperationExpression) {
-            return createEncryptCondition((BinaryOperationExpression) expression, tableName);
+            return createBinaryEncryptCondition((BinaryOperationExpression) expression, tableName);
         }
         if (expression instanceof InExpression) {
             return createInEncryptCondition(tableName, (InExpression) expression, ((InExpression) expression).getRight());
@@ -126,7 +126,7 @@ public final class EncryptConditionEngine {
         return Optional.empty();
     }
     
-    private Optional<EncryptCondition> createEncryptCondition(final BinaryOperationExpression expression, final String tableName) {
+    private Optional<EncryptCondition> createBinaryEncryptCondition(final BinaryOperationExpression expression, final String tableName) {
         String operator = expression.getOperator();
         if (!LOGICAL_OPERATOR.contains(operator)) {
             if (isSupportedOperator(operator)) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/context/EncryptSQLRewriteContextDecorator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/context/EncryptSQLRewriteContextDecorator.java
index 75aaebb..0bdbc15 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/context/EncryptSQLRewriteContextDecorator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/context/EncryptSQLRewriteContextDecorator.java
@@ -47,19 +47,21 @@ public final class EncryptSQLRewriteContextDecorator implements SQLRewriteContex
         SQLStatementContext<?> sqlStatementContext = sqlRewriteContext.getSqlStatementContext();
         boolean containsEncryptTable = containsEncryptTable(encryptRule, sqlStatementContext);
         if (!sqlRewriteContext.getParameters().isEmpty()) {
-            Collection<ParameterRewriter> parameterRewriters = new EncryptParameterRewriterBuilder(encryptRule, 
-                    sqlRewriteContext.getSchema(), sqlStatementContext, encryptConditions, containsEncryptTable).getParameterRewriters();
+            Collection<ParameterRewriter> parameterRewriters = new EncryptParameterRewriterBuilder(encryptRule,
+                    sqlRewriteContext.getSchemaName(), sqlRewriteContext.getSchema(), sqlStatementContext, encryptConditions, containsEncryptTable).getParameterRewriters();
             rewriteParameters(sqlRewriteContext, parameterRewriters);
         }
         // TODO Optimize logic of statement context init for encryptors
         encryptRule.setUpEncryptorSchema(sqlRewriteContext.getSchema());
-        sqlRewriteContext.addSQLTokenGenerators(new EncryptTokenGenerateBuilder(encryptRule, sqlStatementContext, encryptConditions, containsEncryptTable).getSQLTokenGenerators());
+        sqlRewriteContext.addSQLTokenGenerators(new EncryptTokenGenerateBuilder(
+                encryptRule, sqlStatementContext, encryptConditions, containsEncryptTable, sqlRewriteContext.getSchemaName()).getSQLTokenGenerators());
     }
     
     private Collection<EncryptCondition> getEncryptConditions(final EncryptRule encryptRule, final SQLRewriteContext sqlRewriteContext) {
         SQLStatementContext<?> sqlStatementContext = sqlRewriteContext.getSqlStatementContext();
         Collection<WhereSegment> whereSegments = sqlStatementContext instanceof WhereAvailable ? ((WhereAvailable) sqlStatementContext).getWhereSegments() : Collections.emptyList();
-        return whereSegments.isEmpty() ? Collections.emptyList() : new EncryptConditionEngine(encryptRule, sqlRewriteContext.getSchema()).createEncryptConditions(sqlStatementContext);
+        return whereSegments.isEmpty() ? Collections.emptyList() 
+                : new EncryptConditionEngine(encryptRule, sqlRewriteContext.getSchemaName(), sqlRewriteContext.getSchema()).createEncryptConditions(sqlStatementContext);
     }
     
     @SuppressWarnings("rawtypes")
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/EncryptParameterRewriterBuilder.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/EncryptParameterRewriterBuilder.java
index 38b1ebc..67b3a74 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/EncryptParameterRewriterBuilder.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/EncryptParameterRewriterBuilder.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.encrypt.rewrite.parameter;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.encrypt.rewrite.aware.EncryptConditionsAware;
 import org.apache.shardingsphere.encrypt.rewrite.aware.QueryWithCipherColumnAware;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rewrite.condition.EncryptCondition;
 import org.apache.shardingsphere.encrypt.rewrite.parameter.rewriter.EncryptAssignmentParameterRewriter;
 import org.apache.shardingsphere.encrypt.rewrite.parameter.rewriter.EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter;
@@ -45,6 +46,8 @@ public final class EncryptParameterRewriterBuilder implements ParameterRewriterB
     
     private final EncryptRule encryptRule;
     
+    private final String schemaName;
+    
     private final ShardingSphereSchema schema;
     
     private final SQLStatementContext<?> sqlStatementContext;
@@ -81,6 +84,9 @@ public final class EncryptParameterRewriterBuilder implements ParameterRewriterB
         if (toBeAddedParameterRewriter instanceof EncryptConditionsAware) {
             ((EncryptConditionsAware) toBeAddedParameterRewriter).setEncryptConditions(encryptConditions);
         }
+        if (toBeAddedParameterRewriter instanceof SchemaNameAware) {
+            ((SchemaNameAware) toBeAddedParameterRewriter).setSchemaName(schemaName);
+        }
         if (toBeAddedParameterRewriter.isNeedRewrite(sqlStatementContext)) {
             parameterRewriters.add(toBeAddedParameterRewriter);
         }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptAssignmentParameterRewriter.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptAssignmentParameterRewriter.java
index 1df3df7..d870005 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptAssignmentParameterRewriter.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptAssignmentParameterRewriter.java
@@ -19,12 +19,12 @@ package org.apache.shardingsphere.encrypt.rewrite.parameter.rewriter;
 
 import com.google.common.base.Preconditions;
 import lombok.Setter;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 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.UpdateStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.util.DMLStatementContextHelper;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.rewrite.parameter.builder.ParameterBuilder;
 import org.apache.shardingsphere.infra.rewrite.parameter.builder.impl.GroupedParameterBuilder;
@@ -48,10 +48,12 @@ import java.util.Optional;
  * Assignment parameter rewriter for encrypt.
  */
 @Setter
-public final class EncryptAssignmentParameterRewriter implements ParameterRewriter<SQLStatementContext>, EncryptRuleAware {
+public final class EncryptAssignmentParameterRewriter implements ParameterRewriter<SQLStatementContext>, EncryptRuleAware, SchemaNameAware {
     
     private EncryptRule encryptRule;
     
+    private String schemaName;
+    
     @Override
     public boolean isNeedRewrite(final SQLStatementContext sqlStatementContext) {
         if (sqlStatementContext instanceof UpdateStatementContext) {
@@ -66,7 +68,6 @@ public final class EncryptAssignmentParameterRewriter implements ParameterRewrit
     @Override
     public void rewrite(final ParameterBuilder parameterBuilder, final SQLStatementContext sqlStatementContext, final List<Object> parameters) {
         String tableName = ((TableAvailable) sqlStatementContext).getAllTables().iterator().next().getTableName().getIdentifier().getValue();
-        String schemaName = DMLStatementContextHelper.getSchemaName(sqlStatementContext);
         for (AssignmentSegment each : getSetAssignmentSegment(sqlStatementContext.getSqlStatement()).getAssignments()) {
             if (each.getValue() instanceof ParameterMarkerExpressionSegment && encryptRule.findEncryptor(schemaName, tableName, each.getColumns().get(0).getIdentifier().getValue()).isPresent()) {
                 StandardParameterBuilder standardParameterBuilder = parameterBuilder instanceof StandardParameterBuilder
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter.java
index 86a773b..8d33164 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.encrypt.rewrite.parameter.rewriter;
 
 import com.google.common.base.Preconditions;
 import lombok.Setter;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
@@ -41,10 +42,12 @@ import java.util.Optional;
  * Insert on duplicate key update parameter rewriter for encrypt.
  */
 @Setter
-public final class EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter implements ParameterRewriter<InsertStatementContext>, EncryptRuleAware {
+public final class EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter implements ParameterRewriter<InsertStatementContext>, EncryptRuleAware, SchemaNameAware {
     
     private EncryptRule encryptRule;
     
+    private String schemaName;
+    
     @Override
     public boolean isNeedRewrite(final SQLStatementContext sqlStatementContext) {
         return sqlStatementContext instanceof InsertStatementContext
@@ -57,7 +60,6 @@ public final class EncryptInsertOnDuplicateKeyUpdateValueParameterRewriter imple
         String tableName = insertStatementContext.getSqlStatement().getTable().getTableName().getIdentifier().getValue();
         GroupedParameterBuilder groupedParameterBuilder = (GroupedParameterBuilder) parameterBuilder;
         OnDuplicateUpdateContext onDuplicateKeyUpdateValueContext = insertStatementContext.getOnDuplicateKeyUpdateValueContext();
-        String schemaName = insertStatementContext.getSchemaName();
         for (int index = 0; index < onDuplicateKeyUpdateValueContext.getValueExpressions().size(); index++) {
             int columnIndex = index;
             String encryptLogicColumnName = onDuplicateKeyUpdateValueContext.getColumn(columnIndex).getIdentifier().getValue();
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertValueParameterRewriter.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertValueParameterRewriter.java
index 0b266ba..e727a86 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertValueParameterRewriter.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptInsertValueParameterRewriter.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.encrypt.rewrite.parameter.rewriter;
 
 import com.google.common.base.Preconditions;
 import lombok.Setter;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
@@ -44,10 +45,12 @@ import java.util.Optional;
  * Insert value parameter rewriter for encrypt.
  */
 @Setter
-public final class EncryptInsertValueParameterRewriter implements ParameterRewriter<InsertStatementContext>, EncryptRuleAware {
+public final class EncryptInsertValueParameterRewriter implements ParameterRewriter<InsertStatementContext>, EncryptRuleAware, SchemaNameAware {
     
     private EncryptRule encryptRule;
     
+    private String schemaName;
+    
     @Override
     public boolean isNeedRewrite(final SQLStatementContext sqlStatementContext) {
         return sqlStatementContext instanceof InsertStatementContext && !InsertStatementHandler.getSetAssignmentSegment(((InsertStatementContext) sqlStatementContext).getSqlStatement()).isPresent()
@@ -58,7 +61,6 @@ public final class EncryptInsertValueParameterRewriter implements ParameterRewri
     public void rewrite(final ParameterBuilder parameterBuilder, final InsertStatementContext insertStatementContext, final List<Object> parameters) {
         String tableName = insertStatementContext.getSqlStatement().getTable().getTableName().getIdentifier().getValue();
         Iterator<String> descendingColumnNames = insertStatementContext.getDescendingColumnNames();
-        String schemaName = insertStatementContext.getSchemaName();
         while (descendingColumnNames.hasNext()) {
             String columnName = descendingColumnNames.next();
             encryptRule.findEncryptor(schemaName, tableName, columnName).ifPresent(
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptPredicateParameterRewriter.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptPredicateParameterRewriter.java
index 6ae8c93..d9fe1c4 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptPredicateParameterRewriter.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/parameter/rewriter/EncryptPredicateParameterRewriter.java
@@ -19,11 +19,11 @@ package org.apache.shardingsphere.encrypt.rewrite.parameter.rewriter;
 
 import lombok.Setter;
 import org.apache.shardingsphere.encrypt.rewrite.aware.EncryptConditionsAware;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rewrite.condition.EncryptCondition;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.util.DMLStatementContextHelper;
 import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.rewrite.parameter.builder.ParameterBuilder;
@@ -39,12 +39,14 @@ import java.util.Map.Entry;
  * Predicate parameter rewriter for encrypt.
  */
 @Setter
-public final class EncryptPredicateParameterRewriter implements ParameterRewriter<SQLStatementContext>, EncryptRuleAware, EncryptConditionsAware {
+public final class EncryptPredicateParameterRewriter implements ParameterRewriter<SQLStatementContext>, EncryptRuleAware, EncryptConditionsAware, SchemaNameAware {
     
     private EncryptRule encryptRule;
     
     private Collection<EncryptCondition> encryptConditions;
     
+    private String schemaName;
+    
     @Override
     public boolean isNeedRewrite(final SQLStatementContext sqlStatementContext) {
         return sqlStatementContext instanceof WhereAvailable && !((WhereAvailable) sqlStatementContext).getWhereSegments().isEmpty();
@@ -52,7 +54,6 @@ public final class EncryptPredicateParameterRewriter implements ParameterRewrite
     
     @Override
     public void rewrite(final ParameterBuilder parameterBuilder, final SQLStatementContext sqlStatementContext, final List<Object> parameters) {
-        String schemaName = DMLStatementContextHelper.getSchemaName(sqlStatementContext);
         for (EncryptCondition each : encryptConditions) {
             boolean queryWithCipherColumn = encryptRule.isQueryWithCipherColumn(each.getTableName());
             if (queryWithCipherColumn) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/EncryptTokenGenerateBuilder.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/EncryptTokenGenerateBuilder.java
index a1c5d51..acb55df 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/EncryptTokenGenerateBuilder.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/EncryptTokenGenerateBuilder.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.encrypt.rewrite.token;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.encrypt.rewrite.aware.EncryptConditionsAware;
 import org.apache.shardingsphere.encrypt.rewrite.aware.QueryWithCipherColumnAware;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rewrite.condition.EncryptCondition;
 import org.apache.shardingsphere.encrypt.rewrite.token.generator.AssistQueryAndPlainInsertColumnsTokenGenerator;
 import org.apache.shardingsphere.encrypt.rewrite.token.generator.EncryptAlterTableTokenGenerator;
@@ -57,6 +58,8 @@ public final class EncryptTokenGenerateBuilder implements SQLTokenGeneratorBuild
     
     private final boolean containsEncryptTable;
     
+    private final String schemaName;
+    
     @Override
     public Collection<SQLTokenGenerator> getSQLTokenGenerators() {
         if (!containsEncryptTable) {
@@ -88,6 +91,9 @@ public final class EncryptTokenGenerateBuilder implements SQLTokenGeneratorBuild
         if (toBeAddedSQLTokenGenerator instanceof EncryptConditionsAware) {
             ((EncryptConditionsAware) toBeAddedSQLTokenGenerator).setEncryptConditions(encryptConditions);
         }
+        if (toBeAddedSQLTokenGenerator instanceof SchemaNameAware) {
+            ((SchemaNameAware) toBeAddedSQLTokenGenerator).setSchemaName(schemaName);
+        }
         if (toBeAddedSQLTokenGenerator.isGenerateSQLToken(sqlStatementContext)) {
             sqlTokenGenerators.add(toBeAddedSQLTokenGenerator);
         }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptAssignmentTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptAssignmentTokenGenerator.java
index fa177b6..0530d3a 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptAssignmentTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptAssignmentTokenGenerator.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.encrypt.rewrite.token.generator;
 
 import com.google.common.base.Preconditions;
 import lombok.Setter;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptAssignmentToken;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptLiteralAssignmentToken;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptParameterAssignmentToken;
@@ -27,7 +28,6 @@ import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 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.UpdateStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.util.DMLStatementContextHelper;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.CollectionSQLTokenGenerator;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
@@ -48,10 +48,12 @@ import java.util.Optional;
  * Assignment generator for encrypt.
  */
 @Setter
-public final class EncryptAssignmentTokenGenerator implements CollectionSQLTokenGenerator, EncryptRuleAware {
+public final class EncryptAssignmentTokenGenerator implements CollectionSQLTokenGenerator, EncryptRuleAware, SchemaNameAware {
     
     private EncryptRule encryptRule;
     
+    private String schemaName;
+    
     @Override
     public boolean isGenerateSQLToken(final SQLStatementContext sqlStatementContext) {
         return sqlStatementContext instanceof UpdateStatementContext || (sqlStatementContext instanceof InsertStatementContext
@@ -62,7 +64,6 @@ public final class EncryptAssignmentTokenGenerator implements CollectionSQLToken
     public Collection<EncryptAssignmentToken> generateSQLTokens(final SQLStatementContext sqlStatementContext) {
         Collection<EncryptAssignmentToken> result = new LinkedList<>();
         String tableName = ((TableAvailable) sqlStatementContext).getAllTables().iterator().next().getTableName().getIdentifier().getValue();
-        String schemaName = DMLStatementContextHelper.getSchemaName(sqlStatementContext);
         for (AssignmentSegment each : getSetAssignmentSegment(sqlStatementContext.getSqlStatement()).getAssignments()) {
             if (encryptRule.findEncryptor(schemaName, tableName, each.getColumns().get(0).getIdentifier().getValue()).isPresent()) {
                 generateSQLToken(schemaName, tableName, each).ifPresent(result::add);
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertOnUpdateTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertOnUpdateTokenGenerator.java
index f9a3d4b..1cdf9e2 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertOnUpdateTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertOnUpdateTokenGenerator.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.encrypt.rewrite.token.generator;
 
 import com.google.common.base.Preconditions;
 import lombok.Setter;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptAssignmentToken;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptFunctionAssignmentToken;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptLiteralAssignmentToken;
@@ -47,10 +48,12 @@ import java.util.Optional;
  * Insert on update values token generator for encrypt.
  */
 @Setter
-public final class EncryptInsertOnUpdateTokenGenerator implements CollectionSQLTokenGenerator<InsertStatementContext>, EncryptRuleAware {
+public final class EncryptInsertOnUpdateTokenGenerator implements CollectionSQLTokenGenerator<InsertStatementContext>, EncryptRuleAware, SchemaNameAware {
     
     private EncryptRule encryptRule;
     
+    private String schemaName;
+    
     @Override
     public boolean isGenerateSQLToken(final SQLStatementContext sqlStatementContext) {
         return sqlStatementContext instanceof InsertStatementContext
@@ -68,7 +71,6 @@ public final class EncryptInsertOnUpdateTokenGenerator implements CollectionSQLT
         if (onDuplicateKeyColumnsSegments.isEmpty()) {
             return result;
         }
-        String schemaName = insertStatementContext.getSchemaName();
         for (AssignmentSegment each : onDuplicateKeyColumnsSegments) {
             boolean leftEncryptorPresent = encryptRule.findEncryptor(schemaName, tableName, each.getColumns().get(0).getIdentifier().getValue()).isPresent();
             if (each.getValue() instanceof FunctionSegment && "VALUES".equalsIgnoreCase(((FunctionSegment) each.getValue()).getFunctionName())) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertValuesTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertValuesTokenGenerator.java
index b2a24a6..64443dc 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertValuesTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptInsertValuesTokenGenerator.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.encrypt.rewrite.token.generator;
 
 import lombok.Setter;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptInsertValuesToken;
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
@@ -49,12 +50,14 @@ import java.util.Optional;
  * Insert values token generator for encrypt.
  */
 @Setter
-public final class EncryptInsertValuesTokenGenerator implements OptionalSQLTokenGenerator<InsertStatementContext>, PreviousSQLTokensAware, EncryptRuleAware {
+public final class EncryptInsertValuesTokenGenerator implements OptionalSQLTokenGenerator<InsertStatementContext>, PreviousSQLTokensAware, EncryptRuleAware, SchemaNameAware {
     
     private List<SQLToken> previousSQLTokens;
     
     private EncryptRule encryptRule;
     
+    private String schemaName;
+    
     @Override
     public boolean isGenerateSQLToken(final SQLStatementContext sqlStatementContext) {
         return sqlStatementContext instanceof InsertStatementContext && !(((InsertStatementContext) sqlStatementContext).getSqlStatement()).getValues().isEmpty();
@@ -121,7 +124,7 @@ public final class EncryptInsertValuesTokenGenerator implements OptionalSQLToken
         Iterator<String> descendingColumnNames = insertStatementContext.getDescendingColumnNames();
         while (descendingColumnNames.hasNext()) {
             String columnName = descendingColumnNames.next();
-            Optional<EncryptAlgorithm> encryptor = encryptRule.findEncryptor(insertStatementContext.getSchemaName(), tableName, columnName);
+            Optional<EncryptAlgorithm> encryptor = encryptRule.findEncryptor(schemaName, tableName, columnName);
             if (encryptor.isPresent()) {
                 int columnIndex = useDefaultInsertColumnsToken.map(sqlToken -> ((UseDefaultInsertColumnsToken) sqlToken).getColumns().indexOf(columnName))
                         .orElseGet(() -> insertStatementContext.getColumnNames().indexOf(columnName));
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptPredicateRightValueTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptPredicateRightValueTokenGenerator.java
index 435391e..a7849a6 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptPredicateRightValueTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptPredicateRightValueTokenGenerator.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.encrypt.rewrite.token.generator;
 
 import lombok.Setter;
 import org.apache.shardingsphere.encrypt.rewrite.aware.EncryptConditionsAware;
+import org.apache.shardingsphere.encrypt.rewrite.aware.SchemaNameAware;
 import org.apache.shardingsphere.encrypt.rewrite.condition.EncryptCondition;
 import org.apache.shardingsphere.encrypt.rewrite.condition.impl.EncryptInCondition;
 import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptPredicateEqualRightValueToken;
@@ -26,7 +27,6 @@ import org.apache.shardingsphere.encrypt.rewrite.token.pojo.EncryptPredicateInRi
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.util.DMLStatementContextHelper;
 import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.CollectionSQLTokenGenerator;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.aware.ParametersAware;
@@ -43,7 +43,7 @@ import java.util.Optional;
  * Predicate right value token generator for encrypt.
  */
 @Setter
-public final class EncryptPredicateRightValueTokenGenerator implements CollectionSQLTokenGenerator, ParametersAware, EncryptConditionsAware, EncryptRuleAware {
+public final class EncryptPredicateRightValueTokenGenerator implements CollectionSQLTokenGenerator, ParametersAware, EncryptConditionsAware, EncryptRuleAware, SchemaNameAware {
     
     private List<Object> parameters;
     
@@ -51,6 +51,8 @@ public final class EncryptPredicateRightValueTokenGenerator implements Collectio
     
     private EncryptRule encryptRule;
     
+    private String schemaName;
+    
     @Override
     public boolean isGenerateSQLToken(final SQLStatementContext sqlStatementContext) {
         return sqlStatementContext instanceof WhereAvailable && !((WhereAvailable) sqlStatementContext).getWhereSegments().isEmpty();
@@ -58,7 +60,6 @@ public final class EncryptPredicateRightValueTokenGenerator implements Collectio
     
     @Override
     public Collection<SQLToken> generateSQLTokens(final SQLStatementContext sqlStatementContext) {
-        String schemaName = DMLStatementContextHelper.getSchemaName(sqlStatementContext);
         Collection<SQLToken> result = new LinkedHashSet<>();
         for (EncryptCondition each : encryptConditions) {
             result.add(generateSQLToken(schemaName, each));
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptProjectionTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptProjectionTokenGenerator.java
index d7c2fcc..617925a 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptProjectionTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/EncryptProjectionTokenGenerator.java
@@ -230,14 +230,4 @@ public final class EncryptProjectionTokenGenerator implements CollectionSQLToken
         }
         throw new IllegalStateException(String.format("Can not find shorthand projection segment, owner is: `%s`", owner.orElse(null)));
     }
-    
-    @Override
-    public void setPreviousSQLTokens(final List<SQLToken> previousSQLTokens) {
-        this.previousSQLTokens = previousSQLTokens;
-    }
-    
-    @Override
-    public void setSchema(final ShardingSphereSchema schema) {
-        this.schema = schema;
-    }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/merge/EncryptResultDecoratorEngineTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/merge/EncryptResultDecoratorEngineTest.java
index 6c9dbfc..5e37b01 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/merge/EncryptResultDecoratorEngineTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/merge/EncryptResultDecoratorEngineTest.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.infra.binder.statement.dal.ExplainStatementCont
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.merge.engine.ResultProcessEngine;
 import org.apache.shardingsphere.infra.merge.engine.decorator.ResultDecorator;
@@ -66,7 +67,7 @@ public final class EncryptResultDecoratorEngineTest {
     @Test
     public void assertNewInstanceWithSelectStatement() {
         EncryptResultDecoratorEngine engine = (EncryptResultDecoratorEngine) OrderedSPIRegistry.getRegisteredServices(ResultProcessEngine.class, Collections.singleton(rule)).get(rule);
-        ResultDecorator actual = engine.newInstance(databaseType, schema, rule, mock(ConfigurationProperties.class), mock(SelectStatementContext.class, RETURNS_DEEP_STUBS));
+        ResultDecorator actual = engine.newInstance(databaseType, DefaultSchema.LOGIC_NAME, schema, rule, mock(ConfigurationProperties.class), mock(SelectStatementContext.class, RETURNS_DEEP_STUBS));
         assertThat(actual, instanceOf(EncryptDQLResultDecorator.class));
     }
     
@@ -75,14 +76,14 @@ public final class EncryptResultDecoratorEngineTest {
         SQLStatementContext<ExplainStatement> sqlStatementContext = mock(ExplainStatementContext.class);
         when(sqlStatementContext.getSqlStatement()).thenReturn(mock(MySQLExplainStatement.class));
         EncryptResultDecoratorEngine engine = (EncryptResultDecoratorEngine) OrderedSPIRegistry.getRegisteredServices(ResultProcessEngine.class, Collections.singleton(rule)).get(rule);
-        ResultDecorator actual = engine.newInstance(databaseType, schema, rule, mock(ConfigurationProperties.class), sqlStatementContext);
+        ResultDecorator actual = engine.newInstance(databaseType, DefaultSchema.LOGIC_NAME, schema, rule, mock(ConfigurationProperties.class), sqlStatementContext);
         assertThat(actual, instanceOf(EncryptDALResultDecorator.class));
     }
     
     @Test
     public void assertNewInstanceWithOtherStatement() {
         EncryptResultDecoratorEngine engine = (EncryptResultDecoratorEngine) OrderedSPIRegistry.getRegisteredServices(ResultProcessEngine.class, Collections.singleton(rule)).get(rule);
-        ResultDecorator actual = engine.newInstance(databaseType, schema, rule, mock(ConfigurationProperties.class), mock(InsertStatementContext.class));
+        ResultDecorator actual = engine.newInstance(databaseType, DefaultSchema.LOGIC_NAME, schema, rule, mock(ConfigurationProperties.class), mock(InsertStatementContext.class));
         assertThat(actual, instanceOf(TransparentResultDecorator.class));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaDataTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaDataTest.java
index bd84fc2..0c3d39b 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaDataTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/merge/dql/EncryptAlgorithmMetaDataTest.java
@@ -27,6 +27,7 @@ import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.junit.Before;
 import org.junit.Test;
@@ -87,8 +88,8 @@ public final class EncryptAlgorithmMetaDataTest {
         Map<String, String> columnTableNames = new HashMap<>();
         columnTableNames.put(columnProjection.getExpression(), "t_order");
         when(tablesContext.findTableName(Collections.singletonList(columnProjection), schema)).thenReturn(columnTableNames);
-        when(encryptRule.findEncryptor(null, "t_order", "id")).thenReturn(Optional.of(encryptAlgorithm));
-        EncryptAlgorithmMetaData encryptAlgorithmMetaData = new EncryptAlgorithmMetaData(schema, encryptRule, selectStatementContext);
+        when(encryptRule.findEncryptor(DefaultSchema.LOGIC_NAME, "t_order", "id")).thenReturn(Optional.of(encryptAlgorithm));
+        EncryptAlgorithmMetaData encryptAlgorithmMetaData = new EncryptAlgorithmMetaData(DefaultSchema.LOGIC_NAME, schema, encryptRule, selectStatementContext);
         Optional<EncryptAlgorithm> actualEncryptor = encryptAlgorithmMetaData.findEncryptor(1);
         assertTrue(actualEncryptor.isPresent());
         assertThat(actualEncryptor.get().getType(), is("MD5"));
@@ -98,8 +99,8 @@ public final class EncryptAlgorithmMetaDataTest {
     public void assertFindEncryptorByColumnName() {
         when(tablesContext.findTableName(Collections.singletonList(columnProjection), schema)).thenReturn(Collections.emptyMap());
         when(tablesContext.getTableNames()).thenReturn(Arrays.asList("t_user", "t_user_item", "t_order_item"));
-        when(encryptRule.findEncryptor(null, "t_order_item", "id")).thenReturn(Optional.of(encryptAlgorithm));
-        EncryptAlgorithmMetaData encryptAlgorithmMetaData = new EncryptAlgorithmMetaData(schema, encryptRule, selectStatementContext);
+        when(encryptRule.findEncryptor(DefaultSchema.LOGIC_NAME, "t_order_item", "id")).thenReturn(Optional.of(encryptAlgorithm));
+        EncryptAlgorithmMetaData encryptAlgorithmMetaData = new EncryptAlgorithmMetaData(DefaultSchema.LOGIC_NAME, schema, encryptRule, selectStatementContext);
         Optional<EncryptAlgorithm> actualEncryptor = encryptAlgorithmMetaData.findEncryptor(1);
         assertTrue(actualEncryptor.isPresent());
         assertThat(actualEncryptor.get().getType(), is("MD5"));
@@ -108,7 +109,7 @@ public final class EncryptAlgorithmMetaDataTest {
     @Test
     public void assertFindEncryptorWhenColumnProjectionIsNotExist() {
         when(projectionsContext.getExpandProjections()).thenReturn(Collections.singletonList(mock(DerivedProjection.class)));
-        EncryptAlgorithmMetaData encryptAlgorithmMetaData = new EncryptAlgorithmMetaData(schema, encryptRule, selectStatementContext);
+        EncryptAlgorithmMetaData encryptAlgorithmMetaData = new EncryptAlgorithmMetaData(DefaultSchema.LOGIC_NAME, schema, encryptRule, selectStatementContext);
         Optional<EncryptAlgorithm> actualEncryptor = encryptAlgorithmMetaData.findEncryptor(1);
         assertFalse(actualEncryptor.isPresent());
     }
@@ -119,7 +120,7 @@ public final class EncryptAlgorithmMetaDataTest {
         columnTableNames.put(columnProjection.getExpression(), "t_order");
         when(tablesContext.findTableName(Collections.singletonList(columnProjection), schema)).thenReturn(columnTableNames);
         when(encryptRule.isQueryWithCipherColumn("t_order")).thenReturn(true);
-        EncryptAlgorithmMetaData encryptAlgorithmMetaData = new EncryptAlgorithmMetaData(schema, encryptRule, selectStatementContext);
+        EncryptAlgorithmMetaData encryptAlgorithmMetaData = new EncryptAlgorithmMetaData(DefaultSchema.LOGIC_NAME, schema, encryptRule, selectStatementContext);
         assertTrue(encryptAlgorithmMetaData.isQueryWithCipherColumn(1));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameter/EncryptParameterRewriterBuilderTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameter/EncryptParameterRewriterBuilderTest.java
index a29f806..624192f 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameter/EncryptParameterRewriterBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameter/EncryptParameterRewriterBuilderTest.java
@@ -21,6 +21,7 @@ import org.apache.shardingsphere.encrypt.rewrite.parameter.rewriter.EncryptPredi
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.parameter.rewriter.ParameterRewriter;
 import org.junit.Test;
@@ -46,7 +47,8 @@ public final class EncryptParameterRewriterBuilderTest {
         ShardingSphereSchema shardingSphereSchema = mock(ShardingSphereSchema.class);
         SQLStatementContext<?> sqlStatementContext = mock(SelectStatementContext.class, RETURNS_DEEP_STUBS);
         when(sqlStatementContext.getTablesContext().getTableNames()).thenReturn(Collections.singletonList("t_order"));
-        Collection<ParameterRewriter> actual = new EncryptParameterRewriterBuilder(encryptRule, shardingSphereSchema, sqlStatementContext, Collections.emptyList(), true).getParameterRewriters();
+        Collection<ParameterRewriter> actual = new EncryptParameterRewriterBuilder(
+                encryptRule, DefaultSchema.LOGIC_NAME, shardingSphereSchema, sqlStatementContext, Collections.emptyList(), true).getParameterRewriters();
         assertThat(actual.size(), is(1));
         ParameterRewriter parameterRewriter = actual.iterator().next();
         assertThat(parameterRewriter, instanceOf(EncryptPredicateParameterRewriter.class));
@@ -61,7 +63,8 @@ public final class EncryptParameterRewriterBuilderTest {
         SelectStatementContext sqlStatementContext = mock(SelectStatementContext.class, RETURNS_DEEP_STUBS);
         when(sqlStatementContext.getTablesContext().getTableNames()).thenReturn(Collections.singletonList("t_order"));
         when(sqlStatementContext.getWhereSegments()).thenReturn(Collections.emptyList());
-        Collection<ParameterRewriter> actual = new EncryptParameterRewriterBuilder(encryptRule, shardingSphereSchema, sqlStatementContext, Collections.emptyList(), true).getParameterRewriters();
+        Collection<ParameterRewriter> actual = new EncryptParameterRewriterBuilder(
+                encryptRule, DefaultSchema.LOGIC_NAME, shardingSphereSchema, sqlStatementContext, Collections.emptyList(), true).getParameterRewriters();
         assertThat(actual.size(), is(0));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rewrite/token/EncryptTokenGenerateBuilderTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rewrite/token/EncryptTokenGenerateBuilderTest.java
index c50335a..aa5a700 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rewrite/token/EncryptTokenGenerateBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-core/src/test/java/org/apache/shardingsphere/encrypt/rewrite/token/EncryptTokenGenerateBuilderTest.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.aware.EncryptRuleAware;
 import org.apache.shardingsphere.infra.binder.segment.select.orderby.OrderByItem;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.SQLTokenGenerator;
 import org.junit.Before;
 import org.junit.Test;
@@ -67,7 +68,8 @@ public final class EncryptTokenGenerateBuilderTest {
         when(selectStatementContext.getGroupByContext().getItems()).thenReturn(Collections.emptyList());
         when(selectStatementContext.isContainsJoinQuery()).thenReturn(true);
         when(selectStatementContext.getWhereSegments()).thenReturn(Collections.emptyList());
-        EncryptTokenGenerateBuilder encryptTokenGenerateBuilder = new EncryptTokenGenerateBuilder(encryptRule, selectStatementContext, Collections.emptyList(), true);
+        EncryptTokenGenerateBuilder encryptTokenGenerateBuilder = new EncryptTokenGenerateBuilder(
+                encryptRule, selectStatementContext, Collections.emptyList(), true, DefaultSchema.LOGIC_NAME);
         Collection<SQLTokenGenerator> sqlTokenGenerators = encryptTokenGenerateBuilder.getSQLTokenGenerators();
         assertThat(sqlTokenGenerators.size(), is(3));
         Iterator<SQLTokenGenerator> iterator = sqlTokenGenerators.iterator();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
index 6f1b629..1b6bade 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingDeleteStatementValidatorTest.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.sharding.route.engine.validator.dml;
 
 import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
-import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingDeleteStatementValidator;
@@ -83,7 +82,7 @@ public final class ShardingDeleteStatementValidatorTest {
         tableSegment.getActualDeleteTables().add(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("order"))));
         tableSegment.getActualDeleteTables().add(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("order_item"))));
         sqlStatement.setTableSegment(tableSegment);
-        DeleteStatementContext sqlStatementContext = new DeleteStatementContext(sqlStatement, DefaultSchema.LOGIC_NAME);
+        DeleteStatementContext sqlStatementContext = new DeleteStatementContext(sqlStatement);
         Collection<String> tableNames = new HashSet<>(Arrays.asList("user", "order", "order_item"));
         when(shardingRule.isAllShardingTables(tableNames)).thenReturn(false);
         when(shardingRule.tableRuleExists(tableNames)).thenReturn(true);
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
index 95f957c..ff167fc 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/ShardingUpdateStatementValidatorTest.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.sharding.route.engine.validator.dml;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
@@ -71,7 +70,7 @@ public final class ShardingUpdateStatementValidatorTest {
     public void assertPreValidateWhenUpdateSingleTable() {
         UpdateStatement updateStatement = createUpdateStatement();
         updateStatement.setTableSegment(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("user"))));
-        SQLStatementContext<UpdateStatement> sqlStatementContext = new UpdateStatementContext(updateStatement, DefaultSchema.LOGIC_NAME);
+        SQLStatementContext<UpdateStatement> sqlStatementContext = new UpdateStatementContext(updateStatement);
         Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
         when(shardingRule.isAllShardingTables(tableNames)).thenReturn(true);
         when(shardingRule.tableRuleExists(tableNames)).thenReturn(true);
@@ -85,7 +84,7 @@ public final class ShardingUpdateStatementValidatorTest {
         joinTableSegment.setLeft(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("user"))));
         joinTableSegment.setRight(new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("order"))));
         updateStatement.setTableSegment(joinTableSegment);
-        SQLStatementContext<UpdateStatement> sqlStatementContext = new UpdateStatementContext(updateStatement, DefaultSchema.LOGIC_NAME);
+        SQLStatementContext<UpdateStatement> sqlStatementContext = new UpdateStatementContext(updateStatement);
         Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
         when(shardingRule.isAllShardingTables(tableNames)).thenReturn(false);
         when(shardingRule.tableRuleExists(tableNames)).thenReturn(true);
@@ -94,7 +93,7 @@ public final class ShardingUpdateStatementValidatorTest {
     
     @Test
     public void assertPostValidateWhenNotUpdateShardingColumn() {
-        UpdateStatementContext sqlStatementContext = new UpdateStatementContext(createUpdateStatement(), DefaultSchema.LOGIC_NAME);
+        UpdateStatementContext sqlStatementContext = new UpdateStatementContext(createUpdateStatement());
         new ShardingUpdateStatementValidator().postValidate(shardingRule, sqlStatementContext, Collections.emptyList(), 
                 mock(ShardingSphereSchema.class), mock(ConfigurationProperties.class), mock(RouteContext.class));
     }
@@ -102,14 +101,14 @@ public final class ShardingUpdateStatementValidatorTest {
     @Test
     public void assertPostValidateWhenUpdateShardingColumnWithSameRouteContext() {
         mockShardingRuleForUpdateShardingColumn();
-        new ShardingUpdateStatementValidator().postValidate(shardingRule, new UpdateStatementContext(createUpdateStatement(), DefaultSchema.LOGIC_NAME),
+        new ShardingUpdateStatementValidator().postValidate(shardingRule, new UpdateStatementContext(createUpdateStatement()),
                 Collections.emptyList(), mock(ShardingSphereSchema.class), mock(ConfigurationProperties.class), createSingleRouteContext());
     }
     
     @Test(expected = ShardingSphereException.class)
     public void assertPostValidateWhenUpdateShardingColumnWithDifferentRouteContext() {
         mockShardingRuleForUpdateShardingColumn();
-        new ShardingUpdateStatementValidator().postValidate(shardingRule, new UpdateStatementContext(createUpdateStatement(), DefaultSchema.LOGIC_NAME), 
+        new ShardingUpdateStatementValidator().postValidate(shardingRule, new UpdateStatementContext(createUpdateStatement()), 
                 Collections.emptyList(), mock(ShardingSphereSchema.class), mock(ConfigurationProperties.class), createFullRouteContext());
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
index 3abeb3e..ff3358e 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
@@ -133,19 +133,19 @@ public final class SQLStatementContextFactory {
             return new SelectStatementContext(metaDataMap, parameters, (SelectStatement) sqlStatement, defaultSchemaName);
         }
         if (sqlStatement instanceof UpdateStatement) {
-            return new UpdateStatementContext((UpdateStatement) sqlStatement, defaultSchemaName);
+            return new UpdateStatementContext((UpdateStatement) sqlStatement);
         }
         if (sqlStatement instanceof DeleteStatement) {
-            return new DeleteStatementContext((DeleteStatement) sqlStatement, defaultSchemaName);
+            return new DeleteStatementContext((DeleteStatement) sqlStatement);
         }
         if (sqlStatement instanceof InsertStatement) {
             return new InsertStatementContext(metaDataMap, parameters, (InsertStatement) sqlStatement, defaultSchemaName);
         }
         if (sqlStatement instanceof CallStatement) {
-            return new CallStatementContext((CallStatement) sqlStatement, defaultSchemaName);
+            return new CallStatementContext((CallStatement) sqlStatement);
         }
         if (sqlStatement instanceof CopyStatement) {
-            return new CopyStatementContext((CopyStatement) sqlStatement, defaultSchemaName);
+            return new CopyStatementContext((CopyStatement) sqlStatement);
         }
         throw new UnsupportedOperationException(String.format("Unsupported SQL statement `%s`", sqlStatement.getClass().getSimpleName()));
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CallStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CallStatementContext.java
index a8177d9..d8fbe8c 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CallStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CallStatementContext.java
@@ -17,21 +17,15 @@
 
 package org.apache.shardingsphere.infra.binder.statement.dml;
 
-import lombok.Getter;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.SchemaAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CallStatement;
 
 /**
  * Call statement context.
  */
-@Getter
-public final class CallStatementContext extends CommonSQLStatementContext<CallStatement> implements SchemaAvailable {
+public final class CallStatementContext extends CommonSQLStatementContext<CallStatement> {
     
-    private final String schemaName;
-    
-    public CallStatementContext(final CallStatement sqlStatement, final String schemaName) {
+    public CallStatementContext(final CallStatement sqlStatement) {
         super(sqlStatement);
-        this.schemaName = schemaName;
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CopyStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CopyStatementContext.java
index 40a1039..dbddaa9 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CopyStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CopyStatementContext.java
@@ -17,21 +17,15 @@
 
 package org.apache.shardingsphere.infra.binder.statement.dml;
 
-import lombok.Getter;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.SchemaAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CopyStatement;
 
 /**
  * Copy statement context.
  */
-@Getter
-public final class CopyStatementContext extends CommonSQLStatementContext<CopyStatement> implements SchemaAvailable {
+public final class CopyStatementContext extends CommonSQLStatementContext<CopyStatement> {
     
-    private final String schemaName;
-    
-    public CopyStatementContext(final CopyStatement sqlStatement, final String schemaName) {
+    public CopyStatementContext(final CopyStatement sqlStatement) {
         super(sqlStatement);
-        this.schemaName = schemaName;
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/DeleteStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/DeleteStatementContext.java
index 0a0d4e6..19e95ee 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/DeleteStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/DeleteStatementContext.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.infra.binder.statement.dml;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.SchemaAvailable;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
@@ -38,18 +37,15 @@ import java.util.stream.Collectors;
  * Delete statement context.
  */
 @Getter
-public final class DeleteStatementContext extends CommonSQLStatementContext<DeleteStatement> implements TableAvailable, WhereAvailable, SchemaAvailable {
+public final class DeleteStatementContext extends CommonSQLStatementContext<DeleteStatement> implements TableAvailable, WhereAvailable {
     
     private final TablesContext tablesContext;
     
-    private final String schemaName;
-    
     private final Collection<WhereSegment> whereSegments = new LinkedList<>();
     
-    public DeleteStatementContext(final DeleteStatement sqlStatement, final String schemaName) {
+    public DeleteStatementContext(final DeleteStatement sqlStatement) {
         super(sqlStatement);
         tablesContext = new TablesContext(getAllSimpleTableSegments());
-        this.schemaName = schemaName;
         getSqlStatement().getWhere().ifPresent(whereSegments::add);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/InsertStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/InsertStatementContext.java
index 282fef0..381c2d1 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/InsertStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/InsertStatementContext.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.infra.binder.segment.insert.values.InsertValueC
 import org.apache.shardingsphere.infra.binder.segment.insert.values.OnDuplicateUpdateContext;
 import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.SchemaAvailable;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.exception.SchemaNotExistedException;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
@@ -56,7 +55,7 @@ import java.util.concurrent.atomic.AtomicInteger;
  * Insert SQL statement context.
  */
 @Getter
-public final class InsertStatementContext extends CommonSQLStatementContext<InsertStatement> implements TableAvailable, SchemaAvailable {
+public final class InsertStatementContext extends CommonSQLStatementContext<InsertStatement> implements TableAvailable {
     
     private final TablesContext tablesContext;
     
@@ -70,9 +69,8 @@ public final class InsertStatementContext extends CommonSQLStatementContext<Inse
     
     private final GeneratedKeyContext generatedKeyContext;
     
-    private final String schemaName;
-    
-    public InsertStatementContext(final Map<String, ShardingSphereMetaData> metaDataMap, final List<Object> parameters, final InsertStatement sqlStatement, final String defaultSchemaName) {
+    public InsertStatementContext(final Map<String, ShardingSphereMetaData> metaDataMap, final List<Object> parameters, 
+                                  final InsertStatement sqlStatement, final String defaultSchemaName) {
         super(sqlStatement);
         AtomicInteger parametersOffset = new AtomicInteger(0);
         insertValueContexts = getInsertValueContexts(parameters, parametersOffset);
@@ -83,7 +81,6 @@ public final class InsertStatementContext extends CommonSQLStatementContext<Inse
         List<String> insertColumnNames = getInsertColumnNames();
         columnNames = useDefaultColumns() ? schema.getAllColumnNames(sqlStatement.getTable().getTableName().getIdentifier().getValue()) : insertColumnNames;
         generatedKeyContext = new GeneratedKeyContextEngine(sqlStatement, schema).createGenerateKeyContext(insertColumnNames, getAllValueExpressions(sqlStatement), parameters).orElse(null);
-        this.schemaName = defaultSchemaName;
     }
     
     private ShardingSphereSchema getSchema(final Map<String, ShardingSphereMetaData> metaDataMap, final String defaultSchemaName) {
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/SelectStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/SelectStatementContext.java
index fa25e70..eb3b3f2 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/SelectStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/SelectStatementContext.java
@@ -36,7 +36,6 @@ import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.Col
 import org.apache.shardingsphere.infra.binder.segment.select.projection.impl.ParameterMarkerProjection;
 import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.SchemaAvailable;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
 import org.apache.shardingsphere.infra.exception.SchemaNotExistedException;
@@ -76,7 +75,7 @@ import java.util.stream.Collectors;
  * Select SQL statement context.
  */
 @Getter
-public final class SelectStatementContext extends CommonSQLStatementContext<SelectStatement> implements TableAvailable, WhereAvailable, SchemaAvailable {
+public final class SelectStatementContext extends CommonSQLStatementContext<SelectStatement> implements TableAvailable, WhereAvailable {
     
     private final TablesContext tablesContext;
     
@@ -90,8 +89,6 @@ public final class SelectStatementContext extends CommonSQLStatementContext<Sele
     
     private final Map<Integer, SelectStatementContext> subqueryContexts;
     
-    private final String schemaName;
-    
     private final Collection<WhereSegment> whereSegments = new LinkedList<>();
     
     @Setter
@@ -100,7 +97,8 @@ public final class SelectStatementContext extends CommonSQLStatementContext<Sele
     @Setter
     private boolean needAggregateRewrite;
     
-    public SelectStatementContext(final Map<String, ShardingSphereMetaData> metaDataMap, final List<Object> parameters, final SelectStatement sqlStatement, final String defaultSchemaName) {
+    public SelectStatementContext(final Map<String, ShardingSphereMetaData> metaDataMap, final List<Object> parameters, 
+                                  final SelectStatement sqlStatement, final String defaultSchemaName) {
         super(sqlStatement);
         whereSegments.addAll(getWhereSegments(sqlStatement));
         subqueryContexts = createSubqueryContexts(metaDataMap, parameters, defaultSchemaName);
@@ -111,10 +109,10 @@ public final class SelectStatementContext extends CommonSQLStatementContext<Sele
         projectionsContext = new ProjectionsContextEngine(schema, getDatabaseType())
                 .createProjectionsContext(getSqlStatement().getFrom(), getSqlStatement().getProjections(), groupByContext, orderByContext);
         paginationContext = new PaginationContextEngine().createPaginationContext(sqlStatement, projectionsContext, parameters, whereSegments);
-        schemaName = defaultSchemaName;
     }
     
-    private Map<Integer, SelectStatementContext> createSubqueryContexts(final Map<String, ShardingSphereMetaData> metaDataMap, final List<Object> parameters, final String defaultSchemaName) {
+    private Map<Integer, SelectStatementContext> createSubqueryContexts(final Map<String, ShardingSphereMetaData> metaDataMap, 
+                                                                        final List<Object> parameters, final String defaultSchemaName) {
         Collection<SubquerySegment> subquerySegments = SubqueryExtractUtil.getSubquerySegments(getSqlStatement());
         Map<Integer, SelectStatementContext> result = new HashMap<>(subquerySegments.size(), 1);
         for (SubquerySegment each : subquerySegments) {
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/UpdateStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/UpdateStatementContext.java
index 58a9cca..6074d09 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/UpdateStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/UpdateStatementContext.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.infra.binder.statement.dml;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.SchemaAvailable;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
@@ -35,18 +34,15 @@ import java.util.LinkedList;
  * Update SQL statement context.
  */
 @Getter
-public final class UpdateStatementContext extends CommonSQLStatementContext<UpdateStatement> implements TableAvailable, WhereAvailable, SchemaAvailable {
+public final class UpdateStatementContext extends CommonSQLStatementContext<UpdateStatement> implements TableAvailable, WhereAvailable {
     
     private final TablesContext tablesContext;
     
-    private final String schemaName;
-    
     private final Collection<WhereSegment> whereSegments = new LinkedList<>();
     
-    public UpdateStatementContext(final UpdateStatement sqlStatement, final String schemaName) {
+    public UpdateStatementContext(final UpdateStatement sqlStatement) {
         super(sqlStatement);
         tablesContext = new TablesContext(getAllSimpleTableSegments());
-        this.schemaName = schemaName;
         getSqlStatement().getWhere().ifPresent(whereSegments::add);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/util/DMLStatementContextHelper.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/util/DMLStatementContextHelper.java
deleted file mode 100644
index 5479da6..0000000
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/util/DMLStatementContextHelper.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.binder.statement.dml.util;
-
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.CallStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
-import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
-import org.apache.shardingsphere.infra.database.DefaultSchema;
-
-/**
- * DML statement context helper.
- */
-public final class DMLStatementContextHelper {
-    
-    /**
-     * Get schema name from DML statement context.
-     * 
-     * @param sqlStatementContext SQLStatementContext
-     * @return schema name.
-     */
-    public static String getSchemaName(final SQLStatementContext sqlStatementContext) {
-        if (sqlStatementContext instanceof CallStatementContext) {
-            return ((CallStatementContext) sqlStatementContext).getSchemaName();
-        }
-        if (sqlStatementContext instanceof DeleteStatementContext) {
-            return ((DeleteStatementContext) sqlStatementContext).getSchemaName();
-        }
-        if (sqlStatementContext instanceof InsertStatementContext) {
-            return ((InsertStatementContext) sqlStatementContext).getSchemaName();
-        }
-        if (sqlStatementContext instanceof SelectStatementContext) {
-            return ((SelectStatementContext) sqlStatementContext).getSchemaName();
-        }
-        if (sqlStatementContext instanceof UpdateStatementContext) {
-            return ((UpdateStatementContext) sqlStatementContext).getSchemaName();
-        }
-        return DefaultSchema.LOGIC_NAME;
-    }
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/CallStatementContextTest.java b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/CallStatementContextTest.java
index ad9a693..18a23c1 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/CallStatementContextTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/CallStatementContextTest.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.infra.binder.statement.dml;
 
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CallStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLCallStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml.PostgreSQLCallStatement;
@@ -44,7 +43,7 @@ public final class CallStatementContextTest {
     }
     
     private void assertNewInstance(final CallStatement callStatement) {
-        CallStatementContext actual = new CallStatementContext(callStatement, DefaultSchema.LOGIC_NAME);
+        CallStatementContext actual = new CallStatementContext(callStatement);
         assertThat(actual, instanceOf(CommonSQLStatementContext.class));
         assertThat(actual.getSqlStatement(), is(callStatement));
         assertThat(actual.getTablesContext().getTableNames(), is(Collections.emptySet()));
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/DeleteStatementContextTest.java b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/DeleteStatementContextTest.java
index f71c63d..e9b1e9d 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/DeleteStatementContextTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/DeleteStatementContextTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.infra.binder.statement.dml;
 
-import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.JoinTableSegment;
@@ -85,7 +84,7 @@ public final class DeleteStatementContextTest {
         tableSegment.setRight(table2);
         deleteStatement.setWhere(whereSegment);
         deleteStatement.setTableSegment(tableSegment);
-        DeleteStatementContext actual = new DeleteStatementContext(deleteStatement, DefaultSchema.LOGIC_NAME);
+        DeleteStatementContext actual = new DeleteStatementContext(deleteStatement);
         assertThat(actual.getTablesContext().getTableNames(), is(new HashSet<>(Arrays.asList("tbl_1", "tbl_2"))));
         assertThat(actual.getWhereSegments(), is(Collections.singletonList(whereSegment)));
         assertThat(actual.getAllTables().stream().map(a -> a.getTableName().getIdentifier().getValue()).collect(Collectors.toList()), is(Arrays.asList("tbl_1", "tbl_2")));
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/UpdateStatementContextTest.java b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/UpdateStatementContextTest.java
index ccd77d5..9118656 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/UpdateStatementContextTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/dml/UpdateStatementContextTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.infra.binder.statement.dml;
 
-import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.SetAssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
@@ -37,8 +36,6 @@ import org.mockito.junit.MockitoJUnitRunner;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
 import java.util.Optional;
 import java.util.stream.Collectors;
 
@@ -62,9 +59,6 @@ public final class UpdateStatementContextTest {
         when(whereSegment.getExpr()).thenReturn(expression);
         SimpleTableSegment table1 = new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("tbl_1")));
         SimpleTableSegment table2 = new SimpleTableSegment(new TableNameSegment(0, 0, new IdentifierValue("tbl_2")));
-        List<SimpleTableSegment> tables = new LinkedList<>();
-        tables.add(table1);
-        tables.add(table2);
         JoinTableSegment joinTableSegment = new JoinTableSegment();
         joinTableSegment.setLeft(table1);
         joinTableSegment.setRight(table2);
@@ -72,7 +66,7 @@ public final class UpdateStatementContextTest {
         updateStatement.setWhere(whereSegment);
         updateStatement.setTableSegment(joinTableSegment);
         updateStatement.setSetAssignment(new SetAssignmentSegment(0, 0, Collections.emptyList()));
-        UpdateStatementContext actual = new UpdateStatementContext(updateStatement, DefaultSchema.LOGIC_NAME);
+        UpdateStatementContext actual = new UpdateStatementContext(updateStatement);
         assertThat(actual.getTablesContext().getTableNames(), is(new HashSet<>(Arrays.asList("tbl_1", "tbl_2"))));
         assertThat(actual.getWhereSegments(), is(Collections.singletonList(whereSegment)));
         assertThat(actual.getAllTables().stream().map(a -> a.getTableName().getIdentifier().getValue()).collect(Collectors.toList()), is(Arrays.asList("tbl_1", "tbl_2", "tbl_2")));
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 135f743..fc4db5f 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
@@ -55,8 +55,8 @@ public final class KernelProcessor {
     }
     
     private SQLRewriteResult rewrite(final LogicSQL logicSQL, final ShardingSphereMetaData metaData, final ConfigurationProperties props, final RouteContext routeContext) {
-        return new SQLRewriteEntry(
-                metaData.getSchema(), props, metaData.getRuleMetaData().getRules()).rewrite(logicSQL.getSql(), logicSQL.getParameters(), logicSQL.getSqlStatementContext(), routeContext);
+        SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(metaData.getName(), metaData.getSchema(), props, metaData.getRuleMetaData().getRules());
+        return sqlRewriteEntry.rewrite(logicSQL.getSql(), logicSQL.getParameters(), logicSQL.getSqlStatementContext(), routeContext);
     }
     
     private ExecutionContext createExecutionContext(final LogicSQL logicSQL, final ShardingSphereMetaData metaData, final RouteContext routeContext, final SQLRewriteResult rewriteResult) {
diff --git a/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/MergeEngine.java b/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/MergeEngine.java
index 2f4ccb1..1ceac90 100644
--- a/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/MergeEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/MergeEngine.java
@@ -98,7 +98,7 @@ public final class MergeEngine {
         MergedResult result = null;
         for (Entry<ShardingSphereRule, ResultProcessEngine> entry : engines.entrySet()) {
             if (entry.getValue() instanceof ResultDecoratorEngine) {
-                ResultDecorator resultDecorator = ((ResultDecoratorEngine) entry.getValue()).newInstance(databaseType, schema, entry.getKey(), props, sqlStatementContext);
+                ResultDecorator resultDecorator = ((ResultDecoratorEngine) entry.getValue()).newInstance(databaseType, schemaName, schema, entry.getKey(), props, sqlStatementContext);
                 result = null == result ? resultDecorator.decorate(mergedResult, sqlStatementContext, entry.getKey()) : resultDecorator.decorate(result, sqlStatementContext, entry.getKey());
             }
         }
@@ -110,7 +110,7 @@ public final class MergeEngine {
         MergedResult result = null;
         for (Entry<ShardingSphereRule, ResultProcessEngine> entry : engines.entrySet()) {
             if (entry.getValue() instanceof ResultDecoratorEngine) {
-                ResultDecorator resultDecorator = ((ResultDecoratorEngine) entry.getValue()).newInstance(databaseType, schema, entry.getKey(), props, sqlStatementContext);
+                ResultDecorator resultDecorator = ((ResultDecoratorEngine) entry.getValue()).newInstance(databaseType, schemaName, schema, entry.getKey(), props, sqlStatementContext);
                 result = null == result ? resultDecorator.decorate(queryResult, sqlStatementContext, entry.getKey()) : resultDecorator.decorate(result, sqlStatementContext, entry.getKey());
             }
         }
diff --git a/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/engine/decorator/ResultDecoratorEngine.java b/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/engine/decorator/ResultDecoratorEngine.java
index ba96490..0a25064 100644
--- a/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/engine/decorator/ResultDecoratorEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-merge/src/main/java/org/apache/shardingsphere/infra/merge/engine/decorator/ResultDecoratorEngine.java
@@ -35,11 +35,12 @@ public interface ResultDecoratorEngine<T extends ShardingSphereRule> extends Res
      * Create new instance of result decorator.
      * 
      * @param databaseType database type
+     * @param schemaName schema name
      * @param schema ShardingSphere schema
      * @param rule rule
      * @param props ShardingSphere properties
      * @param sqlStatementContext SQL statement context
      * @return new instance of result decorator
      */
-    ResultDecorator newInstance(DatabaseType databaseType, ShardingSphereSchema schema, T rule, ConfigurationProperties props, SQLStatementContext<?> sqlStatementContext);
+    ResultDecorator newInstance(DatabaseType databaseType, String schemaName, ShardingSphereSchema schema, T rule, ConfigurationProperties props, SQLStatementContext<?> sqlStatementContext);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/decorator/ResultDecoratorEngineFixture.java b/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/decorator/ResultDecoratorEngineFixture.java
index 5ba5d83..c890829 100644
--- a/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/decorator/ResultDecoratorEngineFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-merge/src/test/java/org/apache/shardingsphere/infra/merge/fixture/decorator/ResultDecoratorEngineFixture.java
@@ -17,18 +17,18 @@
 
 package org.apache.shardingsphere.infra.merge.fixture.decorator;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.merge.engine.decorator.ResultDecorator;
 import org.apache.shardingsphere.infra.merge.engine.decorator.ResultDecoratorEngine;
 import org.apache.shardingsphere.infra.merge.fixture.rule.DecoratorRuleFixture;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 
 public final class ResultDecoratorEngineFixture implements ResultDecoratorEngine<DecoratorRuleFixture> {
     
     @Override
-    public ResultDecorator newInstance(final DatabaseType databaseType, final ShardingSphereSchema schema, 
+    public ResultDecorator newInstance(final DatabaseType databaseType, final String schemaName, final ShardingSphereSchema schema, 
                                        final DecoratorRuleFixture rule, final ConfigurationProperties props, final SQLStatementContext<?> sqlStatementContext) {
         return new ResultDecoratorFixture();
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntry.java b/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntry.java
index 9f19f00..4af403f 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntry.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntry.java
@@ -44,6 +44,8 @@ public final class SQLRewriteEntry {
         ShardingSphereServiceLoader.register(SQLRewriteContextDecorator.class);
     }
     
+    private final String schemaName;
+    
     private final ShardingSphereSchema schema;
     
     private final ConfigurationProperties props;
@@ -51,7 +53,8 @@ public final class SQLRewriteEntry {
     @SuppressWarnings("rawtypes")
     private final Map<ShardingSphereRule, SQLRewriteContextDecorator> decorators;
     
-    public SQLRewriteEntry(final ShardingSphereSchema schema, final ConfigurationProperties props, final Collection<ShardingSphereRule> rules) {
+    public SQLRewriteEntry(final String schemaName, final ShardingSphereSchema schema, final ConfigurationProperties props, final Collection<ShardingSphereRule> rules) {
+        this.schemaName = schemaName;
         this.schema = schema;
         this.props = props;
         decorators = OrderedSPIRegistry.getRegisteredServices(SQLRewriteContextDecorator.class, rules);
@@ -73,7 +76,7 @@ public final class SQLRewriteEntry {
     }
     
     private SQLRewriteContext createSQLRewriteContext(final String sql, final List<Object> parameters, final SQLStatementContext<?> sqlStatementContext, final RouteContext routeContext) {
-        SQLRewriteContext result = new SQLRewriteContext(schema, sqlStatementContext, sql, parameters);
+        SQLRewriteContext result = new SQLRewriteContext(schemaName, schema, sqlStatementContext, sql, parameters);
         decorate(decorators, result, routeContext);
         result.generateSQLTokens();
         return result;
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/context/SQLRewriteContext.java b/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/context/SQLRewriteContext.java
index 1ef3449..bd0b0ea 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/context/SQLRewriteContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/src/main/java/org/apache/shardingsphere/infra/rewrite/context/SQLRewriteContext.java
@@ -40,6 +40,8 @@ import java.util.List;
 @Getter
 public final class SQLRewriteContext {
     
+    private final String schemaName;
+    
     private final ShardingSphereSchema schema;
     
     private final SQLStatementContext<?> sqlStatementContext;
@@ -55,7 +57,8 @@ public final class SQLRewriteContext {
     @Getter(AccessLevel.NONE)
     private final SQLTokenGenerators sqlTokenGenerators = new SQLTokenGenerators();
     
-    public SQLRewriteContext(final ShardingSphereSchema schema, final SQLStatementContext<?> sqlStatementContext, final String sql, final List<Object> parameters) {
+    public SQLRewriteContext(final String schemaName, final ShardingSphereSchema schema, final SQLStatementContext<?> sqlStatementContext, final String sql, final List<Object> parameters) {
+        this.schemaName = schemaName;
         this.schema = schema;
         this.sqlStatementContext = sqlStatementContext;
         this.sql = sql;
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntryTest.java b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntryTest.java
index 5859ef1..9a86c00 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntryTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntryTest.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.infra.rewrite;
 
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
 import org.apache.shardingsphere.infra.rewrite.engine.result.RouteSQLRewriteResult;
@@ -49,7 +50,7 @@ public final class SQLRewriteEntryTest {
     
     @Test
     public void assertRewriteForGenericSQLRewriteResult() {
-        SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(schema, props, Collections.emptyList());
+        SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(DefaultSchema.LOGIC_NAME, schema, props, Collections.emptyList());
         RouteContext routeContext = new RouteContext();
         GenericSQLRewriteResult sqlRewriteResult = (GenericSQLRewriteResult) sqlRewriteEntry.rewrite("SELECT ?", Collections.singletonList(1), mock(SQLStatementContext.class), routeContext);
         assertThat(sqlRewriteResult.getSqlRewriteUnit().getSql(), is("SELECT ?"));
@@ -58,7 +59,7 @@ public final class SQLRewriteEntryTest {
     
     @Test
     public void assertRewriteForRouteSQLRewriteResult() {
-        SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(schema, props, Collections.emptyList());
+        SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(DefaultSchema.LOGIC_NAME, schema, props, Collections.emptyList());
         RouteContext routeContext = new RouteContext();
         RouteUnit firstRouteUnit = mock(RouteUnit.class);
         when(firstRouteUnit.getDataSourceMapper()).thenReturn(new RouteMapper("ds", "ds_0"));
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/context/SQLRewriteContextTest.java b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/context/SQLRewriteContextTest.java
index e90cfaf..60b96cb 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/context/SQLRewriteContextTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/context/SQLRewriteContextTest.java
@@ -21,6 +21,7 @@ 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.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.parameter.builder.impl.GroupedParameterBuilder;
 import org.apache.shardingsphere.infra.rewrite.parameter.builder.impl.StandardParameterBuilder;
@@ -71,7 +72,8 @@ public final class SQLRewriteContextTest {
         InsertStatementContext statementContext = mock(InsertStatementContext.class, RETURNS_DEEP_STUBS);
         when(((TableAvailable) statementContext).getTablesContext().getSchemaName().isPresent()).thenReturn(false);
         when(statementContext.getInsertSelectContext()).thenReturn(null);
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
         assertThat(sqlRewriteContext.getParameterBuilder(), instanceOf(GroupedParameterBuilder.class));
     }
     
@@ -79,13 +81,15 @@ public final class SQLRewriteContextTest {
     public void assertNotInsertStatementContext() {
         SelectStatementContext statementContext = mock(SelectStatementContext.class, RETURNS_DEEP_STUBS);
         when(((TableAvailable) statementContext).getTablesContext().getSchemaName().isPresent()).thenReturn(false);
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(mock(ShardingSphereSchema.class), statementContext, "SELECT * FROM tbl WHERE id = ?", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                mock(ShardingSphereSchema.class), statementContext, "SELECT * FROM tbl WHERE id = ?", Collections.singletonList(1));
         assertThat(sqlRewriteContext.getParameterBuilder(), instanceOf(StandardParameterBuilder.class));
     }
     
     @Test
     public void assertGenerateOptionalSQLToken() {
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(schema, sqlStatementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                schema, sqlStatementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
         sqlRewriteContext.addSQLTokenGenerators(Collections.singleton(optionalSQLTokenGenerator));
         sqlRewriteContext.generateSQLTokens();
         assertFalse(sqlRewriteContext.getSqlTokens().isEmpty());
@@ -94,7 +98,8 @@ public final class SQLRewriteContextTest {
     
     @Test
     public void assertGenerateCollectionSQLToken() {
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(schema, sqlStatementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                schema, sqlStatementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
         sqlRewriteContext.addSQLTokenGenerators(Collections.singleton(collectionSQLTokenGenerator));
         sqlRewriteContext.generateSQLTokens();
         assertFalse(sqlRewriteContext.getSqlTokens().isEmpty());
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/engine/GenericSQLRewriteEngineTest.java b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/engine/GenericSQLRewriteEngineTest.java
index 6c23e84..d8d7934 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/engine/GenericSQLRewriteEngineTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/engine/GenericSQLRewriteEngineTest.java
@@ -17,10 +17,11 @@
 
 package org.apache.shardingsphere.infra.rewrite.engine;
 
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.context.SQLRewriteContext;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
-import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.junit.Test;
 
 import java.util.Collections;
@@ -34,7 +35,8 @@ public final class GenericSQLRewriteEngineTest {
     @Test
     public void assertRewrite() {
         GenericSQLRewriteResult actual = new GenericSQLRewriteEngine().rewrite(
-                new SQLRewriteContext(mock(ShardingSphereSchema.class), mock(SQLStatementContext.class), "SELECT 1", Collections.emptyList()));
+                new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                        mock(ShardingSphereSchema.class), mock(SQLStatementContext.class), "SELECT 1", Collections.emptyList()));
         assertThat(actual.getSqlRewriteUnit().getSql(), is("SELECT 1"));
         assertThat(actual.getSqlRewriteUnit().getParameters(), is(Collections.emptyList()));
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/engine/RouteSQLRewriteEngineTest.java b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/engine/RouteSQLRewriteEngineTest.java
index 97eaea8..7d07d45 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/engine/RouteSQLRewriteEngineTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/src/test/java/org/apache/shardingsphere/infra/rewrite/engine/RouteSQLRewriteEngineTest.java
@@ -21,6 +21,7 @@ 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.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.database.DefaultSchema;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.context.SQLRewriteContext;
@@ -44,7 +45,8 @@ public final class RouteSQLRewriteEngineTest {
     
     @Test
     public void assertRewriteWithStandardParameterBuilder() {
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(mock(ShardingSphereSchema.class), mock(SQLStatementContext.class), "SELECT ?", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                mock(ShardingSphereSchema.class), mock(SQLStatementContext.class), "SELECT ?", Collections.singletonList(1));
         RouteUnit routeUnit = new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.singletonList(new RouteMapper("tbl", "tbl_0")));
         RouteContext routeContext = new RouteContext();
         routeContext.getRouteUnits().add(routeUnit);
@@ -59,7 +61,8 @@ public final class RouteSQLRewriteEngineTest {
         SelectStatementContext statementContext = mock(SelectStatementContext.class, RETURNS_DEEP_STUBS);
         when(statementContext.getOrderByContext().getItems()).thenReturn(Collections.emptyList());
         when(statementContext.getPaginationContext().isHasPagination()).thenReturn(false);
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(mock(ShardingSphereSchema.class), statementContext, "SELECT ?", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                mock(ShardingSphereSchema.class), statementContext, "SELECT ?", Collections.singletonList(1));
         RouteContext routeContext = new RouteContext();
         RouteUnit firstRouteUnit = new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.singletonList(new RouteMapper("tbl", "tbl_0")));
         RouteUnit secondRouteUnit = new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.singletonList(new RouteMapper("tbl", "tbl_1")));
@@ -76,7 +79,8 @@ public final class RouteSQLRewriteEngineTest {
         InsertStatementContext statementContext = mock(InsertStatementContext.class, RETURNS_DEEP_STUBS);
         when(((TableAvailable) statementContext).getTablesContext().getSchemaName().isPresent()).thenReturn(false);
         when(statementContext.getGroupedParameters()).thenReturn(Collections.singletonList(Collections.singletonList(1)));
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
         RouteUnit routeUnit = new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.singletonList(new RouteMapper("tbl", "tbl_0")));
         RouteContext routeContext = new RouteContext();
         routeContext.getRouteUnits().add(routeUnit);
@@ -91,7 +95,8 @@ public final class RouteSQLRewriteEngineTest {
         InsertStatementContext statementContext = mock(InsertStatementContext.class, RETURNS_DEEP_STUBS);
         when(((TableAvailable) statementContext).getTablesContext().getSchemaName().isPresent()).thenReturn(false);
         when(statementContext.getGroupedParameters()).thenReturn(Collections.singletonList(Collections.singletonList(1)));
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
         RouteUnit routeUnit = new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.singletonList(new RouteMapper("tbl", "tbl_0")));
         RouteContext routeContext = new RouteContext();
         routeContext.getRouteUnits().add(routeUnit);
@@ -108,7 +113,8 @@ public final class RouteSQLRewriteEngineTest {
         InsertStatementContext statementContext = mock(InsertStatementContext.class, RETURNS_DEEP_STUBS);
         when(((TableAvailable) statementContext).getTablesContext().getSchemaName().isPresent()).thenReturn(false);
         when(statementContext.getGroupedParameters()).thenReturn(Collections.singletonList(Collections.singletonList(1)));
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
         RouteUnit routeUnit = new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.singletonList(new RouteMapper("tbl", "tbl_0")));
         RouteContext routeContext = new RouteContext();
         routeContext.getRouteUnits().add(routeUnit);
@@ -126,7 +132,8 @@ public final class RouteSQLRewriteEngineTest {
         when(statementContext.getInsertSelectContext()).thenReturn(null);
         when(statementContext.getGroupedParameters()).thenReturn(Collections.singletonList(Collections.singletonList(1)));
         when(statementContext.getOnDuplicateKeyUpdateParameters()).thenReturn(Collections.emptyList());
-        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
+        SQLRewriteContext sqlRewriteContext = new SQLRewriteContext(DefaultSchema.LOGIC_NAME, 
+                mock(ShardingSphereSchema.class), statementContext, "INSERT INTO tbl VALUES (?)", Collections.singletonList(1));
         RouteUnit routeUnit = new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.singletonList(new RouteMapper("tbl", "tbl_0")));
         RouteContext routeContext = new RouteContext();
         routeContext.getRouteUnits().add(routeUnit);
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
index b89982d..2f315ff 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
@@ -103,8 +103,8 @@ public abstract class AbstractSQLRewriterParameterizedTest {
                 sqlStatementParserEngine.parse(getTestParameters().getInputSQL(), false), DefaultSchema.LOGIC_NAME);
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
         RouteContext routeContext = new SQLRouteEngine(rules, props).route(logicSQL, metaData);
-        SQLRewriteResult sqlRewriteResult = new SQLRewriteEntry(
-                schema, props, rules).rewrite(getTestParameters().getInputSQL(), getTestParameters().getInputParameters(), sqlStatementContext, routeContext);
+        SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(DefaultSchema.LOGIC_NAME, schema, props, rules);
+        SQLRewriteResult sqlRewriteResult = sqlRewriteEntry.rewrite(getTestParameters().getInputSQL(), getTestParameters().getInputParameters(), sqlStatementContext, routeContext);
         return sqlRewriteResult instanceof GenericSQLRewriteResult
                 ? Collections.singletonList(((GenericSQLRewriteResult) sqlRewriteResult).getSqlRewriteUnit()) : (((RouteSQLRewriteResult) sqlRewriteResult).getSqlRewriteUnits()).values();
     }