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 2021/05/04 03:25:06 UTC

[shardingsphere] branch master updated: fix bad rewritten result for foreign key SQL (#10193)

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 16aef56  fix bad rewritten result for foreign key SQL (#10193)
16aef56 is described below

commit 16aef56ae33fa71ebd47ef112dbb9e83bf91b47f
Author: Zhengqiang Duan <st...@gmail.com>
AuthorDate: Tue May 4 11:24:31 2021 +0800

    fix bad rewritten result for foreign key SQL (#10193)
    
    * fix bad rewritten result for foreign key SQL when user config single data node sharding config
    
    * refactor ShardingTableBroadcastRoutingEngine to support multiple tables route
    
    * add different type test cases and fix some bug
    
    * fix ci exception
---
 .../shardingsphere/sharding/rule/ShardingRule.java |   2 +-
 .../sharding/rewrite/token/pojo/TableToken.java    |   6 +-
 .../ShardingSQLRewriterParameterizedTest.java      |   8 ++
 .../src/test/resources/sharding/alter.xml          |  57 +++++++++-
 .../resources/yaml/sharding/sharding-rule.yaml     |  11 +-
 .../engine/type/ShardingRouteEngineFactory.java    |   7 +-
 .../ShardingTableBroadcastRoutingEngine.java       |  49 ++++++++-
 .../complex/ShardingCartesianRoutingEngine.java    |   2 +-
 .../ShardingStatementValidatorFactory.java         |   5 +
 .../impl/ShardingAlterTableStatementValidator.java |  69 ++++++++++++
 .../ShardingTableBroadcastRoutingEngineTest.java   |   7 +-
 .../ShardingAlterTableStatementValidatorTest.java  | 121 +++++++++++++++++++++
 .../infra/route/context/RouteContext.java          |   2 +-
 13 files changed, 331 insertions(+), 15 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
index 3a5414a..825ea4f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
@@ -312,7 +312,7 @@ public final class ShardingRule implements FeatureRule, SchemaRule, DataNodeCont
         if (!singleTableRules.keySet().containsAll(logicTableNames)) {
             return false;
         }
-        return 1 == singleTableRules.values().stream().filter(each -> logicTableNames.contains(each.getTableName())).collect(Collectors.toSet()).size();
+        return 1 == singleTableRules.values().stream().filter(each -> logicTableNames.contains(each.getTableName())).map(SingleTableRule::getDataSourceName).collect(Collectors.toSet()).size();
     }
     
     /**
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/TableToken.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/TableToken.java
index 0ec8d45..f742589 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/TableToken.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/TableToken.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.sharding.rewrite.token.pojo;
 import com.google.common.base.Joiner;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.RouteUnitAware;
 import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.SQLToken;
 import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.Substitutable;
@@ -33,7 +32,6 @@ import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.Identifi
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.stream.Collectors;
 
 /**
  * Table token.
@@ -72,9 +70,7 @@ public final class TableToken extends SQLToken implements Substitutable, RouteUn
     }
     
     private Map<String, String> getLogicAndActualTables(final RouteUnit routeUnit) {
-        Collection<String> tableNames = sqlStatementContext instanceof TableAvailable
-                ? ((TableAvailable) sqlStatementContext).getAllTables().stream().map(each -> each.getTableName().getIdentifier().getValue()).collect(Collectors.toList())
-                : sqlStatementContext.getTablesContext().getTableNames();
+        Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
         Map<String, String> result = new HashMap<>(tableNames.size(), 1);
         for (RouteMapper each : routeUnit.getTableMappers()) {
             result.put(each.getLogicName().toLowerCase(), each.getActualName());
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
index fc8748e..5f3892c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/ShardingSQLRewriterParameterizedTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
@@ -30,6 +31,7 @@ import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.refresher.event.CreateTableEvent;
 import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
@@ -89,6 +91,7 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(
                 schema, getTestParameters().getInputParameters(), sqlStatementParserEngine.parse(getTestParameters().getInputSQL(), false));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
+        mockShardingSphereRuleSingleTable();
         ShardingSphereMetaData metaData = new ShardingSphereMetaData("sharding_db", mock(ShardingSphereResource.class), new ShardingSphereRuleMetaData(Collections.emptyList(), rules), schema);
         RouteContext routeContext = new SQLRouteEngine(rules, props).route(logicSQL, metaData);
         SQLRewriteResult sqlRewriteResult = new SQLRewriteEntry(
@@ -96,6 +99,11 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
         return sqlRewriteResult instanceof GenericSQLRewriteResult
                 ? Collections.singletonList(((GenericSQLRewriteResult) sqlRewriteResult).getSqlRewriteUnit()) : (((RouteSQLRewriteResult) sqlRewriteResult).getSqlRewriteUnits()).values();
     }
+        
+    private void mockShardingSphereRuleSingleTable() {
+        ShardingSphereEventBus.getInstance().post(new CreateTableEvent("db", "t_single", mock(TableMetaData.class)));
+        ShardingSphereEventBus.getInstance().post(new CreateTableEvent("db", "t_single_extend", mock(TableMetaData.class)));
+    }
     
     private YamlRootRuleConfigurations createYamlRootRuleConfigurations() throws IOException {
         URL url = ShardingSQLRewriterParameterizedTest.class.getClassLoader().getResource(getTestParameters().getRuleFile());
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/alter.xml b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/alter.xml
index d5e7dab..0dd0b26 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/alter.xml
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/alter.xml
@@ -17,9 +17,64 @@
   -->
 
 <rewrite-assertions yaml-rule="yaml/sharding/sharding-rule.yaml">
-    <rewrite-assertion id="alter_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+    <rewrite-assertion id="alter_table_with_single_data_node_binding_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_order add constraint t_order_fk foreign key (order_id) references t_order_item (order_id)" />
+        <output sql="ALTER TABLE t_order_0 add constraint t_order_fk foreign key (order_id) references t_order_item_0 (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_single_data_node_unbinding_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_order add constraint t_order_fk foreign key (order_id) references t_order_extend (order_id)" />
+        <output sql="ALTER TABLE t_order_0 add constraint t_order_fk foreign key (order_id) references t_order_extend_0 (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_multi_data_node_and_single_data_node_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_account add constraint t_account_fk foreign key (account_id) references t_order (account_id)" />
+        <output sql="ALTER TABLE t_account_0 add constraint t_account_fk foreign key (account_id) references t_order_0 (account_id)" />
+        <output sql="ALTER TABLE t_account_1 add constraint t_account_fk foreign key (account_id) references t_order_0 (account_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_multi_data_node_binding_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
         <input sql="ALTER TABLE t_account add constraint t_account_fk foreign key (account_id) references t_account_detail (account_id)" />
         <output sql="ALTER TABLE t_account_0 add constraint t_account_fk foreign key (account_id) references t_account_detail_0 (account_id)" />
         <output sql="ALTER TABLE t_account_1 add constraint t_account_fk foreign key (account_id) references t_account_detail_1 (account_id)" />
     </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_single_data_node_and_broadcast_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_order add constraint t_order_fk foreign key (order_id) references t_config (order_id)" />
+        <output sql="ALTER TABLE t_order_0 add constraint t_order_fk foreign key (order_id) references t_config (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_multi_data_node_and_broadcast_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_account add constraint t_account_fk foreign key (account_id) references t_config (account_id)" />
+        <output sql="ALTER TABLE t_account_0 add constraint t_account_fk foreign key (account_id) references t_config (account_id)" />
+        <output sql="ALTER TABLE t_account_1 add constraint t_account_fk foreign key (account_id) references t_config (account_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_single_data_node_and_single_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_order add constraint t_order_fk foreign key (order_id) references t_single (order_id)" />
+        <output sql="ALTER TABLE t_order_0 add constraint t_order_fk foreign key (order_id) references t_single (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_multi_data_node_and_single_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_account add constraint t_account_fk foreign key (account_id) references t_single (account_id)" />
+        <output sql="ALTER TABLE t_account_0 add constraint t_account_fk foreign key (account_id) references t_single (account_id)" />
+        <output sql="ALTER TABLE t_account_1 add constraint t_account_fk foreign key (account_id) references t_single (account_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_broadcast_and_single_data_node_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_config add constraint t_config_fk foreign key (order_id) references t_order (order_id)" />
+        <output sql="ALTER TABLE t_config add constraint t_config_fk foreign key (order_id) references t_order_0 (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_broadcast_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_config add constraint t_config_fk foreign key (order_id) references t_order_type (order_id)" />
+        <output sql="ALTER TABLE t_config add constraint t_config_fk foreign key (order_id) references t_order_type (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_broadcast_and_single_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_config add constraint t_config_fk foreign key (order_id) references t_single (order_id)" />
+        <output sql="ALTER TABLE t_config add constraint t_config_fk foreign key (order_id) references t_single (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_single_and_single_data_node_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_single add constraint t_single_fk foreign key (order_id) references t_order (order_id)" />
+        <output sql="ALTER TABLE t_single add constraint t_single_fk foreign key (order_id) references t_order_0 (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_single_and_broadcast_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_single add constraint t_single_fk foreign key (order_id) references t_config (order_id)" />
+        <output sql="ALTER TABLE t_single add constraint t_single_fk foreign key (order_id) references t_config (order_id)" />
+    </rewrite-assertion>
+    <rewrite-assertion id="alter_table_with_single_table_with_add_foreign_constraint_for_postgresql" db-type="PostgreSQL">
+        <input sql="ALTER TABLE t_single add constraint t_single_fk foreign key (order_id) references t_single_extend (order_id)" />
+        <output sql="ALTER TABLE t_single add constraint t_single_fk foreign key (order_id) references t_single_extend (order_id)" />
+    </rewrite-assertion>
 </rewrite-assertions>
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/yaml/sharding/sharding-rule.yaml b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/yaml/sharding/sharding-rule.yaml
index b1dd488..beb2be7 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/yaml/sharding/sharding-rule.yaml
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/yaml/sharding/sharding-rule.yaml
@@ -41,9 +41,18 @@ rules:
         standard:
           shardingColumn: account_id
           shardingAlgorithmName: t_account_detail_inline
+    t_order:
+      actualDataNodes: db.t_order_0
+    t_order_item:
+      actualDataNodes: db.t_order_item_0 
+    t_order_extend:
+      actualDataNodes: db.t_order_extend_0 
   bindingTables:
     - t_account, t_account_detail
-  
+    - t_order, t_order_item
+  broadcastTables:
+    - t_config
+    - t_order_type
   shardingAlgorithms:
     t_account_inline:
       type: INLINE
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
index 5ee91db..c9ff9bf 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
@@ -30,8 +30,8 @@ import org.apache.shardingsphere.sharding.route.engine.type.broadcast.ShardingDa
 import org.apache.shardingsphere.sharding.route.engine.type.broadcast.ShardingInstanceBroadcastRoutingEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.broadcast.ShardingTableBroadcastRoutingEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.complex.ShardingComplexRoutingEngine;
-import org.apache.shardingsphere.sharding.route.engine.type.ignore.ShardingIgnoreRoutingEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.federated.ShardingFederatedRoutingEngine;
+import org.apache.shardingsphere.sharding.route.engine.type.ignore.ShardingIgnoreRoutingEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.single.SingleTablesRoutingEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.standard.ShardingStandardRoutingEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.unicast.ShardingUnicastRoutingEngine;
@@ -55,6 +55,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dal.PostgreSQLResetParameterStatement;
 
 import java.util.Collection;
+import java.util.stream.Collectors;
 
 /**
  * Sharding routing engine factory.
@@ -98,7 +99,9 @@ public final class ShardingRouteEngineFactory {
         if (functionStatement || procedureStatement) {
             return new ShardingDatabaseBroadcastRoutingEngine();
         }
-        Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
+        Collection<String> tableNames = sqlStatementContext instanceof TableAvailable
+                ? ((TableAvailable) sqlStatementContext).getAllTables().stream().map(each -> each.getTableName().getIdentifier().getValue()).collect(Collectors.toList())
+                : sqlStatementContext.getTablesContext().getTableNames();
         if (!tableNames.isEmpty() && !shardingRule.tableRuleExists(tableNames)) {
             return new SingleTablesRoutingEngine(tableNames, sqlStatement);
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngine.java
index f24735a..76fbb98 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngine.java
@@ -19,12 +19,14 @@ package org.apache.shardingsphere.sharding.route.engine.type.broadcast;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.sharding.route.engine.type.ShardingRouteEngine;
+import org.apache.shardingsphere.sharding.route.engine.type.complex.ShardingCartesianRoutingEngine;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
@@ -34,6 +36,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.Optional;
+import java.util.stream.Collectors;
 
 /**
  * Sharding broadcast routing engine for tables.
@@ -52,17 +55,61 @@ public final class ShardingTableBroadcastRoutingEngine implements ShardingRouteE
             routeContext.getRouteUnits().addAll(getBroadcastTableRouteUnits(shardingRule, ""));
             return;
         }
+        Collection<String> shardingLogicTableNames = shardingRule.getShardingLogicTableNames(logicTableNames);
+        if (shardingLogicTableNames.size() > 1 && shardingRule.isAllBindingTables(shardingLogicTableNames)) {
+            routeContext.getRouteUnits().addAll(getBindingTableRouteUnits(shardingRule, shardingLogicTableNames));
+        } else {
+            Collection<RouteContext> routeContexts = getRouteContexts(shardingRule, logicTableNames);
+            RouteContext newRouteContext = new RouteContext();
+            new ShardingCartesianRoutingEngine(routeContexts).route(newRouteContext, shardingRule);
+            routeContext.getOriginalDataNodes().addAll(newRouteContext.getOriginalDataNodes());
+            routeContext.getRouteUnits().addAll(newRouteContext.getRouteUnits());
+        }
+    }
+    
+    private Collection<RouteContext> getRouteContexts(final ShardingRule shardingRule, final Collection<String> logicTableNames) {
+        Collection<RouteContext> result = new LinkedList<>();
         for (String each : logicTableNames) {
+            RouteContext routeContext = new RouteContext();
             if (shardingRule.getBroadcastTables().contains(each)) {
                 routeContext.getRouteUnits().addAll(getBroadcastTableRouteUnits(shardingRule, each));
+            } else if (shardingRule.getSingleTableRules().containsKey(each)) {
+                routeContext.getRouteUnits().addAll(getSingleTableRouteUnits(shardingRule, each));
             } else {
                 routeContext.getRouteUnits().addAll(getAllRouteUnits(shardingRule, each));
             }
+            result.add(routeContext);
         }
+        return result;
+    }
+    
+    private Collection<RouteUnit> getSingleTableRouteUnits(final ShardingRule shardingRule, final String tableName) {
+        String dataSourceName = shardingRule.getSingleTableRules().get(tableName).getDataSourceName();
+        return Collections.singletonList(new RouteUnit(new RouteMapper(dataSourceName, dataSourceName), Collections.singletonList(new RouteMapper(tableName, tableName))));
+    }
+    
+    private Collection<RouteUnit> getBindingTableRouteUnits(final ShardingRule shardingRule, final Collection<String> tableNames) {
+        String primaryTableName = tableNames.iterator().next();
+        Collection<RouteUnit> result = new LinkedList<>();
+        TableRule tableRule = shardingRule.getTableRule(primaryTableName);
+        for (DataNode each : tableRule.getActualDataNodes()) {
+            result.add(new RouteUnit(new RouteMapper(each.getDataSourceName(), each.getDataSourceName()), getBindingTableMappers(shardingRule, each, primaryTableName, tableNames)));
+        }
+        return result;
+    }
+    
+    private Collection<RouteMapper> getBindingTableMappers(final ShardingRule shardingRule, final DataNode dataNode, final String primaryTableName, final Collection<String> tableNames) {
+        Collection<RouteMapper> result = new LinkedList<>();
+        result.add(new RouteMapper(primaryTableName, dataNode.getTableName()));
+        result.addAll(shardingRule.getLogicAndActualTablesFromBindingTable(dataNode.getDataSourceName(), primaryTableName, dataNode.getTableName(), tableNames)
+                .entrySet().stream().map(each -> new RouteMapper(each.getKey(), each.getValue())).collect(Collectors.toList()));
+        return result;
     }
     
     private Collection<String> getLogicTableNames() {
-        Collection<String> tableNamesInSQL = sqlStatementContext.getTablesContext().getTableNames();
+        Collection<String> tableNamesInSQL = sqlStatementContext instanceof TableAvailable 
+                ? ((TableAvailable) sqlStatementContext).getAllTables().stream().map(each -> each.getTableName().getIdentifier().getValue()).collect(Collectors.toList()) 
+                : sqlStatementContext.getTablesContext().getTableNames();
         if (!tableNamesInSQL.isEmpty()) {
             return tableNamesInSQL;
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/complex/ShardingCartesianRoutingEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/complex/ShardingCartesianRoutingEngine.java
index 95ae5da..6c8d723 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/complex/ShardingCartesianRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/complex/ShardingCartesianRoutingEngine.java
@@ -91,7 +91,7 @@ public final class ShardingCartesianRoutingEngine implements ShardingRouteEngine
     private List<Set<RouteMapper>> toRoutingTableGroups(final String dataSource, final List<Set<String>> actualTableGroups) {
         List<Set<RouteMapper>> result = new ArrayList<>(actualTableGroups.size());
         for (Set<String> each : actualTableGroups) {
-            result.add(new HashSet<>(new ArrayList<>(each).stream().map(input -> findRoutingTable(dataSource, input)).collect(Collectors.toList())));
+            result.add(new LinkedHashSet<>(new ArrayList<>(each).stream().map(input -> findRoutingTable(dataSource, input)).collect(Collectors.toList())));
         }
         return result;
     }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
index 19b1bb1..9b22847 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.sharding.route.engine.validator;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingAlterTableStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingAlterViewStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateFunctionStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingCreateProcedureStatementValidator;
@@ -30,6 +31,7 @@ import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.Shardi
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingSelectStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingUpdateStatementValidator;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateProcedureStatement;
@@ -80,6 +82,9 @@ public final class ShardingStatementValidatorFactory {
         if (sqlStatement instanceof CreateViewStatement) {
             return Optional.of(new ShardingCreateViewStatementValidator());
         }
+        if (sqlStatement instanceof AlterTableStatement) {
+            return Optional.of(new ShardingAlterTableStatementValidator());
+        }
         if (sqlStatement instanceof AlterViewStatement) {
             return Optional.of(new ShardingAlterViewStatementValidator());
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java
new file mode 100644
index 0000000..c211ba4
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/impl/ShardingAlterTableStatementValidator.java
@@ -0,0 +1,69 @@
+/*
+ * 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.sharding.route.engine.validator.ddl.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.context.RouteMapper;
+import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.ShardingDDLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Sharding alter table statement validator.
+ */
+public final class ShardingAlterTableStatementValidator extends ShardingDDLStatementValidator<AlterTableStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<AlterTableStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereSchema schema) {
+        Collection<String> tableNames = sqlStatementContext instanceof TableAvailable
+                ? ((TableAvailable) sqlStatementContext).getAllTables().stream().map(each -> each.getTableName().getIdentifier().getValue()).collect(Collectors.toList())
+                : sqlStatementContext.getTablesContext().getTableNames();
+        if (!shardingRule.tableRuleExists(tableNames) && !shardingRule.isSingleTablesInSameDataSource(tableNames)) {
+            throw new ShardingSphereException("Single tables must be in the same datasource.");
+        }
+    }
+    
+    @Override
+    public void postValidate(final AlterTableStatement sqlStatement, final RouteContext routeContext) {
+        if (routeContext.getRouteUnits().isEmpty()) {
+            throw new ShardingSphereException("Can not get route result, please check your sharding table config.");
+        }
+        String primaryTableName = sqlStatement.getTable().getTableName().getIdentifier().getValue();
+        for (String each : routeContext.getActualDataSourceNames()) {
+            if (containsSameDataNodeResult(primaryTableName, each, routeContext.getRouteUnits())) { 
+                throw new ShardingSphereException("ALTER TABLE ... statement can not support unbinding sharding tables route to multiple same data nodes.");
+            }
+        }
+    }
+    
+    private boolean containsSameDataNodeResult(final String primaryTableName, final String actualDataSourceName, final Collection<RouteUnit> routeUnits) {
+        Collection<RouteMapper> tableMappers = routeUnits.stream().filter(routeUnit -> routeUnit.getDataSourceMapper()
+                .getActualName().equals(actualDataSourceName)).flatMap(routeUnit -> routeUnit.getTableMappers().stream()).collect(Collectors.toList());
+        return tableMappers.stream().filter(routeMapper -> routeMapper.getLogicName().equals(primaryTableName))
+                .collect(Collectors.groupingBy(RouteMapper::getActualName)).entrySet().stream().anyMatch(each -> each.getValue().size() > 1);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngineTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngineTest.java
index b32fc0d..85b5d83 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngineTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngineTest.java
@@ -21,6 +21,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.DropIndexStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
@@ -106,9 +107,10 @@ public final class ShardingTableBroadcastRoutingEngineTest extends AbstractRouti
         when(segment.getIdentifier().getValue()).thenReturn("t_order");
         DropIndexStatement dropIndexStatement = mock(DropIndexStatement.class, RETURNS_DEEP_STUBS);
         when(dropIndexStatement.getIndexes()).thenReturn(Collections.singletonList(segment));
-        SQLStatementContext<?> sqlStatementContext = mock(DropIndexStatementContext.class, RETURNS_DEEP_STUBS);
+        SQLStatementContext<DropIndexStatement> sqlStatementContext = mock(DropIndexStatementContext.class, RETURNS_DEEP_STUBS);
         when(sqlStatementContext.getSqlStatement()).thenReturn(dropIndexStatement);
         when(sqlStatementContext.getTablesContext().getTableNames()).thenReturn(Collections.emptyList());
+        when(((TableAvailable) sqlStatementContext).getAllTables()).thenReturn(Collections.emptyList());
 
         ShardingTableBroadcastRoutingEngine shardingTableBroadcastRoutingEngine = new ShardingTableBroadcastRoutingEngine(schema, sqlStatementContext);
         
@@ -133,9 +135,10 @@ public final class ShardingTableBroadcastRoutingEngineTest extends AbstractRouti
         when(segment.getIdentifier().getValue()).thenReturn("t_order");
         DropIndexStatement dropIndexStatement = mock(DropIndexStatement.class, RETURNS_DEEP_STUBS);
         when(dropIndexStatement.getIndexes()).thenReturn(Collections.singletonList(segment));
-        SQLStatementContext<?> sqlStatementContext = mock(DropIndexStatementContext.class, RETURNS_DEEP_STUBS);
+        SQLStatementContext<DropIndexStatement> sqlStatementContext = mock(DropIndexStatementContext.class, RETURNS_DEEP_STUBS);
         when(sqlStatementContext.getSqlStatement()).thenReturn(dropIndexStatement);
         when(sqlStatementContext.getTablesContext().getTableNames()).thenReturn(Collections.emptyList());
+        when(((TableAvailable) sqlStatementContext).getAllTables()).thenReturn(Collections.emptyList());
 
         ShardingTableBroadcastRoutingEngine shardingTableBroadcastRoutingEngine = new ShardingTableBroadcastRoutingEngine(schema, sqlStatementContext);
 
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterTableStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterTableStatementValidatorTest.java
new file mode 100644
index 0000000..fe78030
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingAlterTableStatementValidatorTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.sharding.route.engine.validator.ddl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.AlterTableStatementContext;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.context.RouteMapper;
+import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingAlterTableStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.constraint.ConstraintDefinitionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterTableStatement;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class ShardingAlterTableStatementValidatorTest {
+    
+    @Mock
+    private ShardingRule shardingRule;
+    
+    @Mock
+    private ShardingSphereSchema schema;
+    
+    @Mock
+    private RouteContext routeContext;
+    
+    @Test
+    public void assertPreValidateAlterTableWithSameDatasourceSingleTablesForPostgreSQL() {
+        PostgreSQLAlterTableStatement sqlStatement = new PostgreSQLAlterTableStatement();
+        sqlStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        ConstraintDefinitionSegment definitionSegment = new ConstraintDefinitionSegment(0, 0);
+        definitionSegment.setReferencedTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order_item")));
+        sqlStatement.getAddConstraintDefinitions().add(definitionSegment);
+        SQLStatementContext<AlterTableStatement> sqlStatementContext = new AlterTableStatementContext(sqlStatement);
+        when(shardingRule.tableRuleExists(Arrays.asList("t_order", "t_order_item"))).thenReturn(false);
+        when(shardingRule.isSingleTablesInSameDataSource(Arrays.asList("t_order", "t_order_item"))).thenReturn(true);
+        new ShardingAlterTableStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), schema);
+    }
+    
+    @Test(expected = ShardingSphereException.class)
+    public void assertPreValidateAlterTableWithDifferentDatasourceSingleTablesForPostgreSQL() {
+        PostgreSQLAlterTableStatement sqlStatement = new PostgreSQLAlterTableStatement();
+        sqlStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        ConstraintDefinitionSegment definitionSegment = new ConstraintDefinitionSegment(0, 0);
+        definitionSegment.setReferencedTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order_item")));
+        sqlStatement.getAddConstraintDefinitions().add(definitionSegment);
+        SQLStatementContext<AlterTableStatement> sqlStatementContext = new AlterTableStatementContext(sqlStatement);
+        when(shardingRule.tableRuleExists(Arrays.asList("t_order", "t_order_item"))).thenReturn(false);
+        when(shardingRule.isSingleTablesInSameDataSource(Arrays.asList("t_order", "t_order_item"))).thenReturn(false);
+        new ShardingAlterTableStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), schema);
+    }
+    
+    @Test(expected = ShardingSphereException.class)
+    public void assertPostValidateAlterTableWithEmptyRouteResultForPostgreSQL() {
+        PostgreSQLAlterTableStatement sqlStatement = new PostgreSQLAlterTableStatement();
+        sqlStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        when(routeContext.getRouteUnits()).thenReturn(Collections.emptyList());
+        new ShardingAlterTableStatementValidator().postValidate(sqlStatement, routeContext);
+    }
+    
+    @Test
+    public void assertPostValidateAlterTableWithSingleDataNodeForPostgreSQL() {
+        PostgreSQLAlterTableStatement sqlStatement = new PostgreSQLAlterTableStatement();
+        sqlStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        when(routeContext.getActualDataSourceNames()).thenReturn(Collections.singletonList("ds_0"));
+        Collection<RouteUnit> routeUnits = new LinkedList<>();
+        routeUnits.add(new RouteUnit(new RouteMapper("ds_0", "ds_0"), 
+                Arrays.asList(new RouteMapper("t_order", "t_order_0"), new RouteMapper("t_order_item", "t_order_item_0"))));
+        when(routeContext.getRouteUnits()).thenReturn(routeUnits);
+        new ShardingAlterTableStatementValidator().postValidate(sqlStatement, routeContext);
+    }
+    
+    @Test(expected = ShardingSphereException.class)
+    public void assertPostValidateAlterTableWithMultiDataNodeForPostgreSQL() {
+        PostgreSQLAlterTableStatement sqlStatement = new PostgreSQLAlterTableStatement();
+        sqlStatement.setTable(new SimpleTableSegment(0, 0, new IdentifierValue("t_order")));
+        when(routeContext.getActualDataSourceNames()).thenReturn(Collections.singletonList("ds_0"));
+        Collection<RouteUnit> routeUnits = new LinkedList<>();
+        routeUnits.add(new RouteUnit(new RouteMapper("ds_0", "ds_0"),
+                Arrays.asList(new RouteMapper("t_order", "t_order_0"), new RouteMapper("t_order_item", "t_order_item_0"))));
+        routeUnits.add(new RouteUnit(new RouteMapper("ds_0", "ds_0"),
+                Arrays.asList(new RouteMapper("t_order", "t_order_0"), new RouteMapper("t_order_item", "t_order_item_1"))));
+        routeUnits.add(new RouteUnit(new RouteMapper("ds_0", "ds_0"),
+                Arrays.asList(new RouteMapper("t_order", "t_order_1"), new RouteMapper("t_order_item", "t_order_item_0"))));
+        routeUnits.add(new RouteUnit(new RouteMapper("ds_0", "ds_0"),
+                Arrays.asList(new RouteMapper("t_order", "t_order_1"), new RouteMapper("t_order_item", "t_order_item_1"))));
+        when(routeContext.getRouteUnits()).thenReturn(routeUnits);
+        new ShardingAlterTableStatementValidator().postValidate(sqlStatement, routeContext);
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/RouteContext.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/RouteContext.java
index 0ed5ff7..6a5c4ec 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/RouteContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/RouteContext.java
@@ -83,7 +83,7 @@ public final class RouteContext {
     }
     
     private Set<String> getActualTableNames(final String actualDataSourceName, final String logicTableName) {
-        Set<String> result = new HashSet<>();
+        Set<String> result = new LinkedHashSet<>();
         for (RouteUnit each : routeUnits) {
             if (actualDataSourceName.equalsIgnoreCase(each.getDataSourceMapper().getActualName())) {
                 result.addAll(each.getActualTableNames(logicTableName));