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

[shardingsphere] branch master updated: Refactor ShardingDeleteStatementValidator to simplify the parameter list (#7033)

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

zhangyonglun 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 48ddf2b  Refactor ShardingDeleteStatementValidator to simplify the parameter list (#7033)
48ddf2b is described below

commit 48ddf2b9ececa05214351e23087689d47a847abd
Author: Juan Pan(Trista) <pa...@apache.org>
AuthorDate: Mon Aug 24 18:18:30 2020 +0800

    Refactor ShardingDeleteStatementValidator to simplify the parameter list (#7033)
---
 .../route/engine/ShardingRouteDecorator.java       |  2 +-
 .../validator/ShardingStatementValidator.java      |  9 +++------
 .../impl/ShardingDeleteStatementValidator.java     |  6 +++---
 .../impl/ShardingInsertStatementValidator.java     |  7 ++++---
 .../impl/ShardingUpdateStatementValidator.java     | 10 ++++++----
 .../impl/ShardingDeleteStatementValidatorTest.java |  5 ++++-
 .../impl/ShardingInsertStatementValidatorTest.java | 23 +++++++++++++++-------
 .../impl/ShardingUpdateStatementValidatorTest.java | 20 +++++++++++++------
 .../infra/metadata/schema/RuleSchemaMetaData.java  | 15 ++++++++++++++
 9 files changed, 66 insertions(+), 31 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
index c3d83f8..6eddbd3 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
@@ -62,7 +62,7 @@ public final class ShardingRouteDecorator implements RouteDecorator<ShardingRule
         List<Object> parameters = routeContext.getParameters();
         SQLStatement sqlStatement = sqlStatementContext.getSqlStatement();
         Optional<ShardingStatementValidator> shardingStatementValidator = ShardingStatementValidatorFactory.newInstance(sqlStatement);
-        shardingStatementValidator.ifPresent(validator -> validator.preValidate(shardingRule, sqlStatementContext, parameters));
+        shardingStatementValidator.ifPresent(validator -> validator.preValidate(shardingRule, routeContext));
         ShardingConditions shardingConditions = getShardingConditions(parameters, sqlStatementContext, metaData.getSchema().getConfiguredSchemaMetaData(), shardingRule);
         boolean needMergeShardingValues = isNeedMergeShardingValues(sqlStatementContext, shardingRule);
         if (sqlStatement instanceof DMLStatement && needMergeShardingValues) {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidator.java
index 9b70dfa..ffe162b 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidator.java
@@ -17,13 +17,11 @@
 
 package org.apache.shardingsphere.sharding.route.engine.validator;
 
+import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.sql.parser.sql.statement.SQLStatement;
 
-import java.util.List;
-
 /**
  * Sharding statement validator.
  *
@@ -35,10 +33,9 @@ public interface ShardingStatementValidator<T extends SQLStatement> {
      * Validate whether sharding operation is supported before route.
      * 
      * @param shardingRule sharding rule
-     * @param sqlStatementContext SQL statement context
-     * @param parameters SQL parameters
+     * @param routeContext route context
      */
-    void preValidate(ShardingRule shardingRule, SQLStatementContext<T> sqlStatementContext, List<Object> parameters);
+    void preValidate(ShardingRule shardingRule, RouteContext routeContext);
 
     /**
      * Validate whether sharding operation is supported after route.
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
index 064f6ff..56daf45 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidator.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.sharding.route.engine.validator.impl;
 
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -26,15 +27,14 @@ import org.apache.shardingsphere.sql.parser.binder.type.TableAvailable;
 import org.apache.shardingsphere.sql.parser.sql.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.DeleteStatement;
 
-import java.util.List;
-
 /**
  * Sharding delete statement validator.
  */
 public final class ShardingDeleteStatementValidator implements ShardingStatementValidator<DeleteStatement> {
 
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<DeleteStatement> sqlStatementContext, final List<Object> parameters) {
+    public void preValidate(final ShardingRule shardingRule, final RouteContext routeContext) {
+        SQLStatementContext sqlStatementContext = routeContext.getSqlStatementContext();
         if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
             throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidator.java
index dd0b00d..aed38a2 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidator.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.sharding.route.engine.validator.impl;
 
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -33,7 +34,6 @@ import org.apache.shardingsphere.sql.parser.sql.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.statement.dml.InsertStatement;
 
 import java.util.Collection;
-import java.util.List;
 import java.util.Optional;
 
 /**
@@ -42,11 +42,12 @@ import java.util.Optional;
 public final class ShardingInsertStatementValidator implements ShardingStatementValidator<InsertStatement> {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<InsertStatement> sqlStatementContext, final List<Object> parameters) {
+    public void preValidate(final ShardingRule shardingRule, final RouteContext routeContext) {
+        SQLStatementContext sqlStatementContext = routeContext.getSqlStatementContext();
         if (null == ((InsertStatementContext) sqlStatementContext).getInsertSelectContext() && 1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
             throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
         }
-        InsertStatement sqlStatement = sqlStatementContext.getSqlStatement();
+        InsertStatement sqlStatement = (InsertStatement) sqlStatementContext.getSqlStatement();
         Optional<OnDuplicateKeyColumnsSegment> onDuplicateKeyColumnsSegment = sqlStatement.getOnDuplicateKeyColumns();
         String tableName = sqlStatement.getTable().getTableName().getIdentifier().getValue();
         if (onDuplicateKeyColumnsSegment.isPresent() && isUpdateShardingKey(shardingRule, onDuplicateKeyColumnsSegment.get(), tableName)) {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidator.java
index 8d2ef8d..4125d96 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidator.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.sharding.route.engine.validator.impl;
 
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.sharding.route.engine.validator.ShardingStatementValidator;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
@@ -46,20 +47,21 @@ import java.util.Optional;
 public final class ShardingUpdateStatementValidator implements ShardingStatementValidator<UpdateStatement> {
     
     @Override
-    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<UpdateStatement> sqlStatementContext, final List<Object> parameters) {
+    public void preValidate(final ShardingRule shardingRule, final RouteContext routeContext) {
+        SQLStatementContext sqlStatementContext = routeContext.getSqlStatementContext();
         if (1 != ((TableAvailable) sqlStatementContext).getAllTables().size()) {
             throw new ShardingSphereException("Cannot support Multiple-Table for '%s'.", sqlStatementContext.getSqlStatement());
         }
-        UpdateStatement sqlStatement = sqlStatementContext.getSqlStatement();
+        UpdateStatement sqlStatement = (UpdateStatement) sqlStatementContext.getSqlStatement();
         String tableName = sqlStatement.getTables().iterator().next().getTableName().getIdentifier().getValue();
         for (AssignmentSegment each : sqlStatement.getSetAssignment().getAssignments()) {
             String shardingColumn = each.getColumn().getIdentifier().getValue();
             if (shardingRule.isShardingColumn(shardingColumn, tableName)) {
-                Optional<Object> shardingColumnSetAssignmentValue = getShardingColumnSetAssignmentValue(each, parameters);
+                Optional<Object> shardingColumnSetAssignmentValue = getShardingColumnSetAssignmentValue(each, routeContext.getParameters());
                 Optional<Object> shardingValue = Optional.empty();
                 Optional<WhereSegment> whereSegmentOptional = sqlStatement.getWhere();
                 if (whereSegmentOptional.isPresent()) {
-                    shardingValue = getShardingValue(whereSegmentOptional.get(), parameters, shardingColumn);
+                    shardingValue = getShardingValue(whereSegmentOptional.get(), routeContext.getParameters(), shardingColumn);
                 }
                 if (shardingColumnSetAssignmentValue.isPresent() && shardingValue.isPresent() && shardingColumnSetAssignmentValue.get().equals(shardingValue.get())) {
                     continue;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidatorTest.java
index 191ddf2..cc37806 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingDeleteStatementValidatorTest.java
@@ -18,6 +18,8 @@
 package org.apache.shardingsphere.sharding.route.engine.validator.impl;
 
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
@@ -42,7 +44,8 @@ public final class ShardingDeleteStatementValidatorTest {
         DeleteStatement sqlStatement = new DeleteStatement();
         sqlStatement.getTables().addAll(createMultiTablesContext().getTables());
         SQLStatementContext<DeleteStatement> sqlStatementContext = new DeleteStatementContext(sqlStatement);
-        new ShardingDeleteStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingDeleteStatementValidator().preValidate(shardingRule, routeContext);
     }
 
     private TablesContext createMultiTablesContext() {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidatorTest.java
index e30fa72..c8cc5fd 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingInsertStatementValidatorTest.java
@@ -18,6 +18,8 @@
 package org.apache.shardingsphere.sharding.route.engine.validator.impl;
 
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.binder.metadata.schema.SchemaMetaData;
 import org.apache.shardingsphere.sql.parser.binder.segment.table.TablesContext;
@@ -57,21 +59,24 @@ public final class ShardingInsertStatementValidatorTest {
     public void assertValidateInsertModifyMultiTables() {
         SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement());
         sqlStatementContext.getTablesContext().getTables().addAll(createMultiTablesContext().getTables());
-        new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingInsertStatementValidator().preValidate(shardingRule, routeContext);
     }
 
     @Test
     public void assertValidateOnDuplicateKeyWithoutShardingKey() {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(false);
         SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement());
-        new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingInsertStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test(expected = ShardingSphereException.class)
     public void assertValidateOnDuplicateKeyWithShardingKey() {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(true);
         SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertStatement());
-        new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingInsertStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test(expected = ShardingSphereException.class)
@@ -80,7 +85,8 @@ public final class ShardingInsertStatementValidatorTest {
         when(shardingRule.isGenerateKeyColumn("id", "user")).thenReturn(false);
         SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertSelectStatement());
         sqlStatementContext.getTablesContext().getTables().addAll(createSingleTablesContext().getTables());
-        new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingInsertStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test
@@ -89,7 +95,8 @@ public final class ShardingInsertStatementValidatorTest {
         when(shardingRule.isGenerateKeyColumn("id", "user")).thenReturn(true);
         SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertSelectStatement());
         sqlStatementContext.getTablesContext().getTables().addAll(createSingleTablesContext().getTables());
-        new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingInsertStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test(expected = ShardingSphereException.class)
@@ -100,7 +107,8 @@ public final class ShardingInsertStatementValidatorTest {
         when(shardingRule.isAllBindingTables(multiTablesContext.getTableNames())).thenReturn(false);
         SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertSelectStatement());
         sqlStatementContext.getTablesContext().getTables().addAll(multiTablesContext.getTables());
-        new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingInsertStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test
@@ -111,7 +119,8 @@ public final class ShardingInsertStatementValidatorTest {
         when(shardingRule.isAllBindingTables(multiTablesContext.getTableNames())).thenReturn(true);
         SQLStatementContext<InsertStatement> sqlStatementContext = new InsertStatementContext(new SchemaMetaData(Collections.emptyMap()), Collections.singletonList(1), createInsertSelectStatement());
         sqlStatementContext.getTablesContext().getTables().addAll(multiTablesContext.getTables());
-        new ShardingInsertStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingInsertStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     private InsertStatement createInsertStatement() {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidatorTest.java
index 47f45fb..08e7468 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/impl/ShardingUpdateStatementValidatorTest.java
@@ -18,6 +18,8 @@
 package org.apache.shardingsphere.sharding.route.engine.validator.impl;
 
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sql.parser.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
@@ -57,26 +59,30 @@ public final class ShardingUpdateStatementValidatorTest {
     public void assertValidateUpdateModifyMultiTables() {
         SQLStatementContext<UpdateStatement> sqlStatementContext = new UpdateStatementContext(createUpdateStatement());
         sqlStatementContext.getTablesContext().getTables().addAll(createMultiTablesContext().getTables());
-        new ShardingUpdateStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList());
+        RouteContext routeContext = new RouteContext(sqlStatementContext, Collections.emptyList(), new RouteResult());
+        new ShardingUpdateStatementValidator().preValidate(shardingRule, routeContext);
     }
 
     @Test
     public void assertValidateUpdateWithoutShardingKey() {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(false);
-        new ShardingUpdateStatementValidator().preValidate(shardingRule, new UpdateStatementContext(createUpdateStatement()), Collections.emptyList());
+        RouteContext routeContext = new RouteContext(new UpdateStatementContext(createUpdateStatement()), Collections.emptyList(), new RouteResult());
+        new ShardingUpdateStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test(expected = ShardingSphereException.class)
     public void assertValidateUpdateWithShardingKey() {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(true);
-        new ShardingUpdateStatementValidator().preValidate(shardingRule, new UpdateStatementContext(createUpdateStatement()), Collections.emptyList());
+        RouteContext routeContext = new RouteContext(new UpdateStatementContext(createUpdateStatement()), Collections.emptyList(), new RouteResult());
+        new ShardingUpdateStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test
     public void assertValidateUpdateWithoutShardingKeyAndParameters() {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(false);
         List<Object> parameters = Arrays.asList(1, 1);
-        new ShardingUpdateStatementValidator().preValidate(shardingRule, new UpdateStatementContext(createUpdateStatement()), parameters);
+        RouteContext routeContext = new RouteContext(new UpdateStatementContext(createUpdateStatement()), parameters, new RouteResult());
+        new ShardingUpdateStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test
@@ -84,7 +90,8 @@ public final class ShardingUpdateStatementValidatorTest {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(true);
         List<Object> parameters = Arrays.asList(1, 1);
         SQLStatementContext<UpdateStatement> updateStatementContext = new UpdateStatementContext(createUpdateStatementAndParameters(1));
-        new ShardingUpdateStatementValidator().preValidate(shardingRule, updateStatementContext, parameters);
+        RouteContext routeContext = new RouteContext(updateStatementContext, parameters, new RouteResult());
+        new ShardingUpdateStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     @Test(expected = ShardingSphereException.class)
@@ -92,7 +99,8 @@ public final class ShardingUpdateStatementValidatorTest {
         when(shardingRule.isShardingColumn("id", "user")).thenReturn(true);
         List<Object> parameters = Arrays.asList(1, 1);
         SQLStatementContext<UpdateStatement> updateStatementContext = new UpdateStatementContext(createUpdateStatementAndParameters(2));
-        new ShardingUpdateStatementValidator().preValidate(shardingRule, updateStatementContext, parameters);
+        RouteContext routeContext = new RouteContext(updateStatementContext, parameters, new RouteResult());
+        new ShardingUpdateStatementValidator().preValidate(shardingRule, routeContext);
     }
     
     private UpdateStatement createUpdateStatement() {
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/RuleSchemaMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/RuleSchemaMetaData.java
index 0bdb645..00c2f68 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/RuleSchemaMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/RuleSchemaMetaData.java
@@ -21,6 +21,8 @@ import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.sql.parser.binder.metadata.schema.SchemaMetaData;
 
+import java.util.Collection;
+import java.util.LinkedList;
 import java.util.Map;
 
 /**
@@ -45,4 +47,17 @@ public final class RuleSchemaMetaData {
         result.merge(configuredSchemaMetaData);
         return result;
     }
+    
+    /**
+     * Get all table names.
+     *
+     * @return all table names
+     */
+    public Collection<String> getAllTableNames() {
+        Collection<String> result = new LinkedList<>(configuredSchemaMetaData.getAllTableNames());
+        for (SchemaMetaData each : unconfiguredSchemaMetaDataMap.values()) {
+            result.addAll(each.getAllTableNames());
+        }
+        return result;
+    }
 }