You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by me...@apache.org on 2020/11/08 23:16:29 UTC

[shardingsphere] branch master updated: Move ShardingSphereSchema.tableAddressingMetaData to PhysicalTableMetaData (#8080)

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

menghaoran 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 cd192e5  Move ShardingSphereSchema.tableAddressingMetaData to PhysicalTableMetaData (#8080)
cd192e5 is described below

commit cd192e5416649e50bb5cc1cdeb4023e89f83277e
Author: Liang Zhang <te...@163.com>
AuthorDate: Mon Nov 9 07:16:02 2020 +0800

    Move ShardingSphereSchema.tableAddressingMetaData to PhysicalTableMetaData (#8080)
    
    * Add PhysicalTableMetaData.addressingDataSources
    
    * Refactor AllSQLRouteExecutor
    
    * Refactor ShardingDDLStatementValidator
    
    * Refactor ShardingDDLStatementValidator
    
    * Refactor SingleTableRoutingEngine
    
    * Refactor MetaDataRefreshStrategy
    
    * Remove ShardingSphereSchema.tableAddressingMetaData
---
 .../EncryptSQLRewriterParameterizedTest.java       |  7 +++---
 .../MixSQLRewriterParameterizedTest.java           | 13 ++++++-----
 .../ShardingSQLRewriterParameterizedTest.java      | 13 ++++++-----
 .../engine/type/ShardingRouteEngineFactory.java    |  8 +++----
 .../type/single/SingleTableRoutingEngine.java      |  8 +++----
 .../ddl/ShardingDDLStatementValidator.java         |  6 +++---
 .../type/single/SingleTableRoutingEngineTest.java  | 25 +++++++++++++++-------
 .../engine/type/standard/AbstractSQLRouteTest.java | 13 ++++++-----
 ...ardingCreateFunctionStatementValidatorTest.java |  4 ++--
 ...rdingCreateProcedureStatementValidatorTest.java |  4 ++--
 .../ShardingCreateTableStatementValidatorTest.java |  4 ++--
 .../context/schema/GovernanceSchemaContexts.java   |  2 +-
 .../schema/loader/SchemaMetaDataLoader.java        | 21 ++++++++++++++++++
 .../schema/model/ShardingSphereSchema.java         |  3 ---
 .../model/physical/PhysicalTableMetaData.java      |  3 +++
 ...reateTableStatementMetaDataRefreshStrategy.java | 17 +--------------
 ...CreateViewStatementMetaDataRefreshStrategy.java | 17 +--------------
 .../DropTableStatementMetaDataRefreshStrategy.java |  5 ++---
 .../DropViewStatementMetaDataRefreshStrategy.java  |  1 -
 .../AbstractMetaDataRefreshStrategyTest.java       |  5 +----
 .../context/schema/SchemaContextsBuilder.java      | 15 ++++++-------
 .../sql/context/ExecutionContextBuilderTest.java   | 19 ++++++++--------
 .../route/engine/impl/AllSQLRouteExecutor.java     | 14 ++----------
 .../hint/ShardingCTLHintBackendHandlerTest.java    |  3 +--
 24 files changed, 103 insertions(+), 127 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
index 9329ac6..bdc1307 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/test/java/org/apache/shardingsphere/encrypt/rewrite/parameterized/EncryptSQLRewriterParameterizedTest.java
@@ -25,6 +25,8 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
 import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
@@ -38,9 +40,6 @@ import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRulesBuilder;
-import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -98,6 +97,6 @@ public final class EncryptSQLRewriterParameterizedTest extends AbstractSQLRewrit
         PhysicalSchemaMetaData schemaMetaData = mock(PhysicalSchemaMetaData.class);
         when(schemaMetaData.getAllColumnNames("t_account")).thenReturn(Arrays.asList("account_id", "certificate_number", "password", "amount", "status"));
         when(schemaMetaData.getAllColumnNames("t_account_bak")).thenReturn(Arrays.asList("account_id", "certificate_number", "password", "amount", "status"));
-        return new ShardingSphereSchema(mock(TableAddressingMetaData.class), schemaMetaData);
+        return new ShardingSphereSchema(schemaMetaData);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
index e060bd5..c58dd1e 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/MixSQLRewriterParameterizedTest.java
@@ -25,6 +25,11 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
@@ -38,12 +43,6 @@ import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRulesBuilder;
-import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -118,7 +117,7 @@ public final class MixSQLRewriterParameterizedTest extends AbstractSQLRewriterPa
         when(schemaMetaData.get("t_account_detail")).thenReturn(mock(PhysicalTableMetaData.class));
         when(schemaMetaData.getAllColumnNames("t_account")).thenReturn(Arrays.asList("account_id", "password", "amount", "status"));
         when(schemaMetaData.getAllColumnNames("t_account_bak")).thenReturn(Arrays.asList("account_id", "password", "amount", "status"));
-        return new ShardingSphereSchema(mock(TableAddressingMetaData.class), schemaMetaData);
+        return new ShardingSphereSchema(schemaMetaData);
     }
     
     private Map<String, PhysicalColumnMetaData> createColumnMetaDataMap() {
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 18a5153..55bbc53 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
@@ -25,6 +25,11 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
 import org.apache.shardingsphere.infra.rewrite.SQLRewriteEntry;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
@@ -38,12 +43,6 @@ import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRulesBuilder;
-import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.yaml.config.YamlRootRuleConfigurations;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -113,7 +112,7 @@ public final class ShardingSQLRewriterParameterizedTest extends AbstractSQLRewri
         when(schemaMetaData.get("t_account")).thenReturn(accountTableMetaData);
         when(schemaMetaData.get("t_account_detail")).thenReturn(mock(PhysicalTableMetaData.class));
         when(schemaMetaData.getAllColumnNames("t_account")).thenReturn(Arrays.asList("account_id", "amount", "status"));
-        return new ShardingSphereSchema(mock(TableAddressingMetaData.class), schemaMetaData);
+        return new ShardingSphereSchema(schemaMetaData);
     }
     
     private Map<String, PhysicalColumnMetaData> createColumnMetaDataMap() {
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 6956c99..1dd17ca 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
@@ -93,7 +93,7 @@ public final class ShardingRouteEngineFactory {
             return new ShardingUnicastRoutingEngine(tableNames);
         }
         if (!shardingRule.tableRuleExists(tableNames)) {
-            return new SingleTableRoutingEngine(tableNames, metaData.getSchema().getTableAddressingMetaData(), sqlStatement);
+            return new SingleTableRoutingEngine(tableNames, metaData.getSchema(), sqlStatement);
         }
         return getShardingRoutingEngine(shardingRule, shardingConditions, tableNames, props);
     }
@@ -107,7 +107,7 @@ public final class ShardingRouteEngineFactory {
         }
         Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
         if (!tableNames.isEmpty() && !shardingRule.tableRuleExists(tableNames)) {
-            return new SingleTableRoutingEngine(tableNames, metaData.getSchema().getTableAddressingMetaData(), sqlStatement);
+            return new SingleTableRoutingEngine(tableNames, metaData.getSchema(), sqlStatement);
         }
         return new ShardingTableBroadcastRoutingEngine(metaData.getSchema().getSchemaMetaData(), sqlStatementContext);
     }
@@ -121,7 +121,7 @@ public final class ShardingRouteEngineFactory {
             return new ShardingDatabaseBroadcastRoutingEngine();
         }
         if (!tableNames.isEmpty() && !shardingRule.tableRuleExists(tableNames)) {
-            return new SingleTableRoutingEngine(tableNames, metaData.getSchema().getTableAddressingMetaData(), sqlStatement);
+            return new SingleTableRoutingEngine(tableNames, metaData.getSchema(), sqlStatement);
         }
         if (!tableNames.isEmpty()) {
             return new ShardingUnicastRoutingEngine(tableNames);
@@ -134,7 +134,7 @@ public final class ShardingRouteEngineFactory {
             Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
             return shardingRule.tableRuleExists(tableNames)
                     ? new ShardingTableBroadcastRoutingEngine(metaData.getSchema().getSchemaMetaData(), sqlStatementContext)
-                    : new SingleTableRoutingEngine(tableNames, metaData.getSchema().getTableAddressingMetaData(), sqlStatementContext.getSqlStatement());
+                    : new SingleTableRoutingEngine(tableNames, metaData.getSchema(), sqlStatementContext.getSqlStatement());
         } else {
             return new ShardingInstanceBroadcastRoutingEngine(metaData.getResource().getDataSourcesMetaData());
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngine.java
index 8da615d..c6f1a93 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngine.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sharding.route.engine.type.single;
 import com.google.common.collect.Lists;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.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;
@@ -43,7 +43,7 @@ public final class SingleTableRoutingEngine implements ShardingRouteEngine {
     
     private final Collection<String> logicTables;
     
-    private final TableAddressingMetaData tableAddressingMetaData;
+    private final ShardingSphereSchema schema;
     
     private final SQLStatement sqlStatement;
     
@@ -60,8 +60,8 @@ public final class SingleTableRoutingEngine implements ShardingRouteEngine {
     // TODO maybe enhance here, only return one data source for multiple tables for now
     private Optional<String> findDataSourceName() {
         for (String each : logicTables) {
-            if (tableAddressingMetaData.getTableDataSourceNamesMapper().containsKey(each)) {
-                return Optional.of(tableAddressingMetaData.getTableDataSourceNamesMapper().get(each).iterator().next());
+            if (schema.getSchemaMetaData().containsTable(each)) {
+                return Optional.of(schema.getSchemaMetaData().get(each).getAddressingDataSources().iterator().next());
             }
         }
         return Optional.empty();
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDDLStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDDLStatementValidator.java
index a1a70b9..b472b76 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDDLStatementValidator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingDDLStatementValidator.java
@@ -56,8 +56,8 @@ public abstract class ShardingDDLStatementValidator<T extends DDLStatement> impl
     protected void validateTableExist(final ShardingSphereSchema schema, final Collection<SimpleTableSegment> tables) {
         for (SimpleTableSegment each : tables) {
             String tableName = each.getTableName().getIdentifier().getValue();
-            if (!schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().containsKey(tableName)) {
-                String dataSourceName = schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().get(tableName).iterator().next();
+            if (!schema.getSchemaMetaData().containsTable(tableName)) {
+                String dataSourceName = schema.getSchemaMetaData().get(tableName).getAddressingDataSources().iterator().next();
                 throw new NoSuchTableException(dataSourceName, tableName);
             }
         }
@@ -72,7 +72,7 @@ public abstract class ShardingDDLStatementValidator<T extends DDLStatement> impl
     protected void validateTableNotExist(final ShardingSphereSchema schema, final Collection<SimpleTableSegment> tables) {
         for (SimpleTableSegment each : tables) {
             String tableName = each.getTableName().getIdentifier().getValue();
-            if (schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().containsKey(tableName)) {
+            if (schema.getSchemaMetaData().containsTable(tableName)) {
                 throw new TableExistsException(tableName);
             }
         }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngineTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngineTest.java
index e028699..f2be7e3 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngineTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/single/SingleTableRoutingEngineTest.java
@@ -17,7 +17,9 @@
 
 package org.apache.shardingsphere.sharding.route.engine.type.single;
 
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
@@ -28,9 +30,10 @@ import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -39,7 +42,7 @@ public final class SingleTableRoutingEngineTest {
     
     @Test
     public void assertRoute() {
-        SingleTableRoutingEngine singleTableRoutingEngine = new SingleTableRoutingEngine(Arrays.asList("t_order", "t_order_item"), createTableAddressingMetaData(), null);
+        SingleTableRoutingEngine singleTableRoutingEngine = new SingleTableRoutingEngine(Arrays.asList("t_order", "t_order_item"), createShardingSphereSchema(), null);
         ShardingRuleConfiguration shardingRuleConfig = new ShardingRuleConfiguration();
         ShardingRule shardingRule = new ShardingRule(shardingRuleConfig, Arrays.asList("ds_0", "ds_1"));
         RouteContext routeContext = new RouteContext();
@@ -59,7 +62,7 @@ public final class SingleTableRoutingEngineTest {
     
     @Test
     public void assertRouteWithoutShardingRule() {
-        SingleTableRoutingEngine singleTableRoutingEngine = new SingleTableRoutingEngine(Arrays.asList("t_order", "t_order_item"), createTableAddressingMetaData(), new MySQLCreateTableStatement());
+        SingleTableRoutingEngine singleTableRoutingEngine = new SingleTableRoutingEngine(Arrays.asList("t_order", "t_order_item"), createShardingSphereSchema(), new MySQLCreateTableStatement());
         ShardingRuleConfiguration shardingRuleConfig = new ShardingRuleConfiguration();
         ShardingRule shardingRule = new ShardingRule(shardingRuleConfig, Arrays.asList("ds_0", "ds_1"));
         RouteContext routeContext = new RouteContext();
@@ -76,10 +79,16 @@ public final class SingleTableRoutingEngineTest {
         assertThat(tableMapper1.getLogicName(), is("t_order_item"));
     }
     
-    private TableAddressingMetaData createTableAddressingMetaData() {
-        TableAddressingMetaData result = new TableAddressingMetaData();
-        result.getTableDataSourceNamesMapper().put("t_order", Collections.singletonList("ds_0"));
-        result.getTableDataSourceNamesMapper().put("t_order_item", Collections.singletonList("ds_0"));
+    private ShardingSphereSchema createShardingSphereSchema() {
+        Map<String, PhysicalTableMetaData> tables = new HashMap<>(2, 1);
+        tables.put("t_order", createTableMetaData());
+        tables.put("t_order_item", createTableMetaData());
+        return new ShardingSphereSchema(new PhysicalSchemaMetaData(tables));
+    }
+    
+    private PhysicalTableMetaData createTableMetaData() {
+        PhysicalTableMetaData result = new PhysicalTableMetaData();
+        result.getAddressingDataSources().add("ds_0");
         return result;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
index a778ac3..33d56ff 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/type/standard/AbstractSQLRouteTest.java
@@ -24,14 +24,13 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
-import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+import org.apache.shardingsphere.infra.parser.sql.SQLStatementParserEngine;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.infra.route.engine.SQLRouteEngine;
 import org.apache.shardingsphere.sharding.route.engine.fixture.AbstractRoutingEngineTest;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 
@@ -52,9 +51,7 @@ public abstract class AbstractSQLRouteTest extends AbstractRoutingEngineTest {
     
     protected final RouteContext assertRoute(final String sql, final List<Object> parameters) {
         ShardingRule shardingRule = createAllShardingRule();
-        TableAddressingMetaData tableAddressingMetaData = new TableAddressingMetaData();
-        tableAddressingMetaData.getTableDataSourceNamesMapper().put("t_category", Collections.singletonList("single_db"));
-        ShardingSphereSchema schema = new ShardingSphereSchema(tableAddressingMetaData, buildPhysicalSchemaMetaData());
+        ShardingSphereSchema schema = new ShardingSphereSchema(buildPhysicalSchemaMetaData());
         ConfigurationProperties props = new ConfigurationProperties(new Properties());
         SQLStatementParserEngine sqlStatementParserEngine = new SQLStatementParserEngine("MySQL");
         SQLStatementContext<?> sqlStatementContext = SQLStatementContextFactory.newInstance(schema.getSchemaMetaData(), parameters, sqlStatementParserEngine.parse(sql, false));
@@ -77,6 +74,8 @@ public abstract class AbstractSQLRouteTest extends AbstractRoutingEngineTest {
                 new PhysicalColumnMetaData("status", Types.VARCHAR, "varchar", false, false, false),
                 new PhysicalColumnMetaData("c_date", Types.TIMESTAMP, "timestamp", false, false, false)), Collections.emptySet()));
         tableMetaDataMap.put("t_other", new PhysicalTableMetaData(Collections.singletonList(new PhysicalColumnMetaData("order_id", Types.INTEGER, "int", true, false, false)), Collections.emptySet()));
+        tableMetaDataMap.put("t_category", new PhysicalTableMetaData());
+        tableMetaDataMap.get("t_category").getAddressingDataSources().add("single_db");
         return new PhysicalSchemaMetaData(tableMetaDataMap);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateFunctionStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateFunctionStatementValidatorTest.java
index ece7da7..cde6d99 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateFunctionStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateFunctionStatementValidatorTest.java
@@ -64,7 +64,7 @@ public final class ShardingCreateFunctionStatementValidatorTest {
         sqlStatement.setRoutineBody(routineBody);
         SQLStatementContext<CreateFunctionStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS);
-        when(schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().containsKey("t_order_item")).thenReturn(true);
+        when(schema.getSchemaMetaData().containsTable("t_order_item")).thenReturn(true);
         new ShardingCreateFunctionStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), schema);
     }
     
@@ -110,7 +110,7 @@ public final class ShardingCreateFunctionStatementValidatorTest {
         sqlStatement.setRoutineBody(routineBody);
         SQLStatementContext<CreateFunctionStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS);
-        when(schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().containsKey("t_order")).thenReturn(true);
+        when(schema.getSchemaMetaData().containsTable("t_order")).thenReturn(true);
         new ShardingCreateFunctionStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), schema);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateProcedureStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateProcedureStatementValidatorTest.java
index 16c1729..ef5e476 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateProcedureStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateProcedureStatementValidatorTest.java
@@ -64,7 +64,7 @@ public final class ShardingCreateProcedureStatementValidatorTest {
         sqlStatement.setRoutineBody(routineBody);
         SQLStatementContext<CreateProcedureStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS);
-        when(schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().containsKey("t_order_item")).thenReturn(true);
+        when(schema.getSchemaMetaData().containsTable("t_order_item")).thenReturn(true);
         new ShardingCreateProcedureStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), schema);
     }
     
@@ -110,7 +110,7 @@ public final class ShardingCreateProcedureStatementValidatorTest {
         sqlStatement.setRoutineBody(routineBody);
         SQLStatementContext<CreateProcedureStatement> sqlStatementContext = new CommonSQLStatementContext<>(sqlStatement);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS);
-        when(schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().containsKey("t_order")).thenReturn(true);
+        when(schema.getSchemaMetaData().containsTable("t_order")).thenReturn(true);
         new ShardingCreateProcedureStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), schema);
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateTableStatementValidatorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateTableStatementValidatorTest.java
index f7aebcf..694220c 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateTableStatementValidatorTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/test/java/org/apache/shardingsphere/sharding/route/engine/validator/ddl/ShardingCreateTableStatementValidatorTest.java
@@ -85,7 +85,7 @@ public final class ShardingCreateTableStatementValidatorTest {
     private void assertValidateCreateTable(final CreateTableStatement sqlStatement) {
         SQLStatementContext<CreateTableStatement> sqlStatementContext = new CreateTableStatementContext(sqlStatement);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS);
-        when(schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().containsKey("t_order")).thenReturn(true);
+        when(schema.getSchemaMetaData().containsTable("t_order")).thenReturn(true);
         new ShardingCreateTableStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), schema);
     }
     
@@ -108,7 +108,7 @@ public final class ShardingCreateTableStatementValidatorTest {
     private void assertValidateCreateTableIfNotExists(final CreateTableStatement sqlStatement) {
         SQLStatementContext<CreateTableStatement> sqlStatementContext = new CreateTableStatementContext(sqlStatement);
         ShardingSphereSchema schema = mock(ShardingSphereSchema.class, RETURNS_DEEP_STUBS);
-        when(schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().containsKey("t_order")).thenReturn(true);
+        when(schema.getSchemaMetaData().containsTable("t_order")).thenReturn(true);
         new ShardingCreateTableStatementValidator().preValidate(shardingRule, sqlStatementContext, Collections.emptyList(), schema);
     }
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/main/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContexts.java b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/main/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContexts.java
index 11c3d14..1ef2b18 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/main/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContexts.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/shardingsphere-governance-core-context/src/main/java/org/apache/shardingsphere/governance/context/schema/GovernanceSchemaContexts.java
@@ -300,7 +300,7 @@ public final class GovernanceSchemaContexts implements SchemaContexts {
     
     private ShardingSphereMetaData getChangedMetaData(final ShardingSphereMetaData oldMetaData, final PhysicalSchemaMetaData newSchemaMetaData, final String schemaName) {
         // TODO refresh tableAddressingMetaData
-        ShardingSphereSchema schema = new ShardingSphereSchema(oldMetaData.getSchema().getTableAddressingMetaData(), newSchemaMetaData);
+        ShardingSphereSchema schema = new ShardingSphereSchema(newSchemaMetaData);
         return new ShardingSphereMetaData(schemaName, oldMetaData.getResource(), oldMetaData.getRuleMetaData(), schema);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoader.java
index d199763..ebdcdb1 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoader.java
@@ -21,7 +21,9 @@ import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.metadata.schema.loader.addressing.TableAddressingMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 
@@ -29,6 +31,7 @@ import javax.sql.DataSource;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Map;
+import java.util.Map.Entry;
 
 /**
  * Schema meta data loader.
@@ -48,6 +51,13 @@ public final class SchemaMetaDataLoader {
      */
     public static PhysicalSchemaMetaData load(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, 
                                               final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
+        PhysicalSchemaMetaData result = loadSchemaMetaData(databaseType, dataSourceMap, rules, props);
+        setAddressingDataSources(databaseType, dataSourceMap, rules, result);
+        return result;
+    }
+    
+    private static PhysicalSchemaMetaData loadSchemaMetaData(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, 
+                                                             final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
         PhysicalSchemaMetaData result = new PhysicalSchemaMetaData();
         for (ShardingSphereRule rule : rules) {
             if (rule instanceof TableContainedRule) {
@@ -60,4 +70,15 @@ public final class SchemaMetaDataLoader {
         }
         return result;
     }
+    
+    private static void setAddressingDataSources(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, 
+                                                 final Collection<ShardingSphereRule> rules, final PhysicalSchemaMetaData schemaMetaData) throws SQLException {
+        for (Entry<String, Collection<String>> entry : TableAddressingMetaDataLoader.load(databaseType, dataSourceMap, rules).getTableDataSourceNamesMapper().entrySet()) {
+            String tableName = entry.getKey();
+            if (!schemaMetaData.containsTable(tableName)) {
+                schemaMetaData.put(tableName, new PhysicalTableMetaData());
+            }
+            schemaMetaData.get(tableName).getAddressingDataSources().addAll(entry.getValue());
+        }
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/ShardingSphereSchema.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/ShardingSphereSchema.java
index c035579..9dc6eb1 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/ShardingSphereSchema.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/ShardingSphereSchema.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.infra.metadata.schema.model;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
 
 /**
@@ -29,7 +28,5 @@ import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSc
 @Getter
 public final class ShardingSphereSchema {
     
-    private final TableAddressingMetaData tableAddressingMetaData;
-    
     private final PhysicalSchemaMetaData schemaMetaData;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalTableMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalTableMetaData.java
index a5ae3e5..7cd441e 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalTableMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalTableMetaData.java
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 
@@ -46,6 +47,8 @@ public final class PhysicalTableMetaData {
     
     private final List<String> primaryKeyColumns = new ArrayList<>();
     
+    private final Collection<String> addressingDataSources = new LinkedHashSet<>();
+    
     public PhysicalTableMetaData() {
         this(Collections.emptyList(), Collections.emptyList());
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategy.java
index 99543fc..4695983 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateTableStatementMetaDataRefreshStrategy.java
@@ -17,10 +17,8 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
-import com.google.common.collect.Lists;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.TableMetaDataLoaderCallback;
@@ -45,19 +43,6 @@ public final class CreateTableStatementMetaDataRefreshStrategy implements MetaDa
         } else {
             schema.getSchemaMetaData().put(tableName, new PhysicalTableMetaData());
         }
-        refreshTableAddressingMetaData(schema.getTableAddressingMetaData(), tableName, routeDataSourceNames);
-    }
-    
-    private void refreshTableAddressingMetaData(final TableAddressingMetaData tableAddressingMetaData, final String tableName, final Collection<String> routeDataSourceNames) {
-        for (String each : routeDataSourceNames) {
-            refreshTableAddressingMetaData(tableAddressingMetaData, tableName, each);
-        }
-    }
-    
-    private void refreshTableAddressingMetaData(final TableAddressingMetaData tableAddressingMetaData, final String tableName, final String dataSourceName) {
-        Collection<String> previousDataSourceNames = tableAddressingMetaData.getTableDataSourceNamesMapper().putIfAbsent(tableName, Lists.newArrayList(dataSourceName));
-        if (null != previousDataSourceNames) {
-            previousDataSourceNames.add(dataSourceName);
-        }
+        schema.getSchemaMetaData().get(tableName).getAddressingDataSources().addAll(routeDataSourceNames);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateViewStatementMetaDataRefreshStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateViewStatementMetaDataRefreshStrategy.java
index 8679278..ad4e036 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateViewStatementMetaDataRefreshStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/CreateViewStatementMetaDataRefreshStrategy.java
@@ -17,10 +17,8 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.refresh.impl;
 
-import com.google.common.collect.Lists;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.MetaDataRefreshStrategy;
 import org.apache.shardingsphere.infra.metadata.schema.refresh.TableMetaDataLoaderCallback;
@@ -38,19 +36,6 @@ public final class CreateViewStatementMetaDataRefreshStrategy implements MetaDat
                                 final CreateViewStatement sqlStatement, final TableMetaDataLoaderCallback callback) {
         String viewName = sqlStatement.getView().getTableName().getIdentifier().getValue();
         schema.getSchemaMetaData().put(viewName, new PhysicalTableMetaData());
-        refreshTableAddressingMetaData(schema.getTableAddressingMetaData(), viewName, routeDataSourceNames);
-    }
-    
-    private void refreshTableAddressingMetaData(final TableAddressingMetaData tableAddressingMetaData, final String tableName, final Collection<String> routeDataSourceNames) {
-        for (String each : routeDataSourceNames) {
-            refreshTableAddressingMetaData(tableAddressingMetaData, tableName, each);
-        }
-    }
-    
-    private void refreshTableAddressingMetaData(final TableAddressingMetaData tableAddressingMetaData, final String tableName, final String dataSourceName) {
-        Collection<String> previousDataSourceNames = tableAddressingMetaData.getTableDataSourceNamesMapper().putIfAbsent(tableName, Lists.newArrayList(dataSourceName));
-        if (null != previousDataSourceNames) {
-            previousDataSourceNames.add(dataSourceName);
-        }
+        schema.getSchemaMetaData().get(viewName).getAddressingDataSources().addAll(routeDataSourceNames);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropTableStatementMetaDataRefreshStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropTableStatementMetaDataRefreshStrategy.java
index 521e3d8..7a25797 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropTableStatementMetaDataRefreshStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropTableStatementMetaDataRefreshStrategy.java
@@ -33,11 +33,10 @@ public final class DropTableStatementMetaDataRefreshStrategy implements MetaData
     @Override
     public void refreshMetaData(final ShardingSphereSchema schema, final DatabaseType databaseType, final Collection<String> routeDataSourceNames,
                                 final DropTableStatement sqlStatement, final TableMetaDataLoaderCallback callback) {
-        sqlStatement.getTables().forEach(each -> removeMetaData(schema, each.getTableName().getIdentifier().getValue(), routeDataSourceNames));
+        sqlStatement.getTables().forEach(each -> removeMetaData(schema, each.getTableName().getIdentifier().getValue()));
     }
     
-    private void removeMetaData(final ShardingSphereSchema schema, final String tableName, final Collection<String> routeDataSourceNames) {
+    private void removeMetaData(final ShardingSphereSchema schema, final String tableName) {
         schema.getSchemaMetaData().remove(tableName);
-        schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().remove(tableName);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropViewStatementMetaDataRefreshStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropViewStatementMetaDataRefreshStrategy.java
index eab2bb2..bb4c3d1 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropViewStatementMetaDataRefreshStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/refresh/impl/DropViewStatementMetaDataRefreshStrategy.java
@@ -38,6 +38,5 @@ public final class DropViewStatementMetaDataRefreshStrategy implements MetaDataR
     
     private void removeMetaData(final ShardingSphereSchema schema, final String viewName) {
         schema.getSchemaMetaData().remove(viewName);
-        schema.getTableAddressingMetaData().getTableDataSourceNamesMapper().remove(viewName);
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/AbstractMetaDataRefreshStrategyTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/AbstractMetaDataRefreshStrategyTest.java
index bd980b9..1a00000 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/AbstractMetaDataRefreshStrategyTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/refresh/AbstractMetaDataRefreshStrategyTest.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.infra.metadata.schema.refresh;
 import com.google.common.collect.ImmutableMap;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalIndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
@@ -29,8 +28,6 @@ import org.junit.Before;
 
 import java.util.Collections;
 
-import static org.mockito.Mockito.mock;
-
 @Getter
 public abstract class AbstractMetaDataRefreshStrategyTest {
     
@@ -44,7 +41,7 @@ public abstract class AbstractMetaDataRefreshStrategyTest {
     private ShardingSphereSchema buildSchema() {
         PhysicalSchemaMetaData schemaMetaData = new PhysicalSchemaMetaData(ImmutableMap.of("t_order", new PhysicalTableMetaData(
                 Collections.singletonList(new PhysicalColumnMetaData("order_id", 1, "String", false, false, false)), Collections.singletonList(new PhysicalIndexMetaData("index")))));
-        return new ShardingSphereSchema(mock(TableAddressingMetaData.class), schemaMetaData);
+        return new ShardingSphereSchema(schemaMetaData);
     }
 }
 
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContextsBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContextsBuilder.java
index b447df3..debae05 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContextsBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/schema/SchemaContextsBuilder.java
@@ -27,17 +27,15 @@ import org.apache.shardingsphere.infra.context.schema.impl.StandardSchemaContext
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
-import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRulesBuilder;
-import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.loader.addressing.TableAddressingMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.resource.CachedDatabaseMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.DataSourcesMetaData;
+import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
+import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.loader.SchemaMetaDataLoader;
+import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRulesBuilder;
 
 import javax.sql.DataSource;
 import java.sql.Connection;
@@ -134,9 +132,8 @@ public final class SchemaContextsBuilder {
     
     private ShardingSphereSchema buildSchema(final String schemaName, final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereRule> rules) throws SQLException {
         long start = System.currentTimeMillis();
-        TableAddressingMetaData tableAddressingMetaData = TableAddressingMetaDataLoader.load(databaseType, dataSourceMap, rules);
         PhysicalSchemaMetaData physicalSchemaMetaData = SchemaMetaDataLoader.load(databaseType, dataSourceMap, rules, props);
-        ShardingSphereSchema result = new ShardingSphereSchema(tableAddressingMetaData, physicalSchemaMetaData);
+        ShardingSphereSchema result = new ShardingSphereSchema(physicalSchemaMetaData);
         log.info("Load meta data for schema {} finished, cost {} milliseconds.", schemaName, System.currentTimeMillis() - start);
         return result;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
index c69834e..5384bc2 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/test/java/org/apache/shardingsphere/infra/executor/sql/context/ExecutionContextBuilderTest.java
@@ -19,20 +19,19 @@ package org.apache.shardingsphere.infra.executor.sql.context;
 
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.resource.CachedDatabaseMetaData;
+import org.apache.shardingsphere.infra.metadata.resource.DataSourcesMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.rewrite.engine.result.GenericSQLRewriteResult;
 import org.apache.shardingsphere.infra.rewrite.engine.result.RouteSQLRewriteResult;
 import org.apache.shardingsphere.infra.rewrite.engine.result.SQLRewriteUnit;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
-import org.apache.shardingsphere.infra.metadata.resource.CachedDatabaseMetaData;
-import org.apache.shardingsphere.infra.metadata.resource.DataSourcesMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.junit.Test;
 
 import java.sql.Types;
@@ -61,7 +60,7 @@ public final class ExecutionContextBuilderTest {
         when(dataSourcesMetaData.getAllInstanceDataSourceNames()).thenReturn(Arrays.asList(firstDataSourceName, "lastDataSourceName"));
         ShardingSphereResource resource = new ShardingSphereResource(Collections.emptyMap(), dataSourcesMetaData, mock(CachedDatabaseMetaData.class));
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList());
-        ShardingSphereSchema schema = new ShardingSphereSchema(mock(TableAddressingMetaData.class), buildPhysicalSchemaMetaData());
+        ShardingSphereSchema schema = new ShardingSphereSchema(buildPhysicalSchemaMetaData());
         ShardingSphereMetaData metaData = new ShardingSphereMetaData("name", resource, ruleMetaData, schema);
         Collection<ExecutionUnit> actual = ExecutionContextBuilder.build(metaData, genericSQLRewriteResult, mock(SQLStatementContext.class));
         Collection<ExecutionUnit> expected = Collections.singletonList(new ExecutionUnit(firstDataSourceName, new SQLUnit(sql, parameters)));
@@ -79,7 +78,7 @@ public final class ExecutionContextBuilderTest {
         sqlRewriteUnits.put(routeUnit2, sqlRewriteUnit2);
         ShardingSphereResource resource = new ShardingSphereResource(Collections.emptyMap(), mock(DataSourcesMetaData.class), mock(CachedDatabaseMetaData.class));
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList());
-        ShardingSphereSchema schema = new ShardingSphereSchema(mock(TableAddressingMetaData.class), buildPhysicalSchemaMetaData());
+        ShardingSphereSchema schema = new ShardingSphereSchema(buildPhysicalSchemaMetaData());
         ShardingSphereMetaData metaData = new ShardingSphereMetaData("name", resource, ruleMetaData, schema);
         Collection<ExecutionUnit> actual = ExecutionContextBuilder.build(metaData, new RouteSQLRewriteResult(sqlRewriteUnits), mock(SQLStatementContext.class));
         ExecutionUnit expectedUnit1 = new ExecutionUnit("actualName1", new SQLUnit("sql1", Collections.singletonList("parameter1")));
@@ -99,7 +98,7 @@ public final class ExecutionContextBuilderTest {
         sqlRewriteUnits.put(routeUnit2, sqlRewriteUnit2);
         ShardingSphereResource resource = new ShardingSphereResource(Collections.emptyMap(), mock(DataSourcesMetaData.class), mock(CachedDatabaseMetaData.class));
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.emptyList());
-        ShardingSphereSchema schema = new ShardingSphereSchema(mock(TableAddressingMetaData.class), buildPhysicalSchemaMetaDataWithoutPrimaryKey());
+        ShardingSphereSchema schema = new ShardingSphereSchema(buildPhysicalSchemaMetaDataWithoutPrimaryKey());
         ShardingSphereMetaData metaData = new ShardingSphereMetaData("name", resource, ruleMetaData, schema);
         Collection<ExecutionUnit> actual = ExecutionContextBuilder.build(metaData, new RouteSQLRewriteResult(sqlRewriteUnits), mock(SQLStatementContext.class));
         ExecutionUnit expectedUnit2 = new ExecutionUnit("actualName2", new SQLUnit("sql2", Collections.singletonList("parameter2")));
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/AllSQLRouteExecutor.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/AllSQLRouteExecutor.java
index 954b6bd..4220815 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/AllSQLRouteExecutor.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/AllSQLRouteExecutor.java
@@ -18,15 +18,13 @@
 package org.apache.shardingsphere.infra.route.engine.impl;
 
 import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteExecutor;
-import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 
-import java.util.Collection;
 import java.util.Collections;
-import java.util.LinkedHashSet;
 
 /**
  * All SQL route executor.
@@ -36,17 +34,9 @@ public final class AllSQLRouteExecutor implements SQLRouteExecutor {
     @Override
     public RouteContext route(final LogicSQL logicSQL, final ShardingSphereMetaData metaData) {
         RouteContext result = new RouteContext();
-        for (String each : getAllDataSourceNames(metaData)) {
+        for (String each : metaData.getResource().getDataSources().keySet()) {
             result.getRouteUnits().add(new RouteUnit(new RouteMapper(each, each), Collections.emptyList()));
         }
         return result;
     }
-    
-    private Collection<String> getAllDataSourceNames(final ShardingSphereMetaData metaData) {
-        Collection<String> result = new LinkedHashSet<>();
-        for (Collection<String> each : metaData.getSchema().getTableAddressingMetaData().getTableDataSourceNamesMapper().values()) {
-            result.addAll(each);
-        }
-        return result;
-    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
index 85fed09..16f585e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/text/sctl/hint/ShardingCTLHintBackendHandlerTest.java
@@ -28,7 +28,6 @@ import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
 import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.model.addressing.TableAddressingMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -204,7 +203,7 @@ public final class ShardingCTLHintBackendHandlerTest {
     
     private Map<String, ShardingSphereMetaData> getMetaDataMap() {
         ShardingSphereMetaData metaData = mock(ShardingSphereMetaData.class);
-        when(metaData.getSchema()).thenReturn(new ShardingSphereSchema(mock(TableAddressingMetaData.class), new PhysicalSchemaMetaData(ImmutableMap.of("user", mock(PhysicalTableMetaData.class)))));
+        when(metaData.getSchema()).thenReturn(new ShardingSphereSchema(new PhysicalSchemaMetaData(ImmutableMap.of("user", mock(PhysicalTableMetaData.class)))));
         when(metaData.isComplete()).thenReturn(true);
         return Collections.singletonMap("schema", metaData);
     }