You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by do...@apache.org on 2020/11/07 15:05:53 UTC

[shardingsphere] branch master updated: Add TableContainedRule to simplify ShardingSphereMetaDataLoader (#8074)

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

dongzonglei 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 9826a5f  Add TableContainedRule to simplify ShardingSphereMetaDataLoader (#8074)
9826a5f is described below

commit 9826a5f6beecd61b4f8b3ea25eb9dbb96703bbef
Author: Liang Zhang <te...@163.com>
AuthorDate: Sat Nov 7 23:05:33 2020 +0800

    Add TableContainedRule to simplify ShardingSphereMetaDataLoader (#8074)
    
    * Add TableContainedRule
    
    * Remove useless implements
    
    * Refactor generic type of ShardingSphereMetaDataLoader
    
    * Refactor generic type of ShardingSphereMetaDataLoader
    
    * Refactor SchemaMetaDataLoader
    
    * Rename DataNodeContainedRule's variable
    
    * Remove useless ShardingSphereMetaDataLoader.load
    
    * Refactor SchemaMetaDataLoader
    
    * Merge ShardingSphereMetaDataDecorator into ShardingSphereMetaDataLoader
---
 .../encrypt/metadata/EncryptMetaDataDecorator.java | 72 ------------------
 .../encrypt/metadata/EncryptMetaDataLoader.java    | 47 ++++++++----
 .../shardingsphere/encrypt/rule/EncryptRule.java   |  9 ++-
 ...hema.loader.spi.ShardingSphereMetaDataDecorator | 18 -----
 .../metadata/EncryptMetaDataDecoratorTest.java     | 67 -----------------
 .../metadata/EncryptMetaDataLoaderTest.java        | 37 +++++----
 .../encrypt/rule/EncryptRuleTest.java              |  5 ++
 .../metadata/ShardingMetaDataDecorator.java        | 87 ----------------------
 .../sharding/metadata/ShardingMetaDataLoader.java  | 64 +++++++++++-----
 .../shardingsphere/sharding/rule/ShardingRule.java |  9 ++-
 ...hema.loader.spi.ShardingSphereMetaDataDecorator | 18 -----
 .../sharding/rule/ShardingRuleTest.java            |  5 ++
 .../shardingsphere/infra/datanode/DataNodes.java   |  6 +-
 .../schema/loader/SchemaMetaDataLoader.java        | 50 +++++++++----
 .../schema/loader/TableMetaDataLoader.java         | 19 +++--
 .../loader/spi/ShardingSphereMetaDataLoader.java   | 31 +++-----
 .../model/physical/PhysicalSchemaMetaData.java     |  3 +
 .../type/TableContainedRule.java}                  | 19 ++---
 ...ataNodeRoutedLogicMetaDataDecoratorFixture.java | 44 -----------
 .../loader/CommonFixtureLogicMetaDataLoader.java   | 18 ++---
 ...taNodeContainedFixtureLogicMetaDataLoader.java} | 21 ++----
 .../schema/fixture/rule/CommonFixtureRule.java     | 12 ++-
 .../fixture/rule/DataNodeContainedFixtureRule.java |  8 +-
 .../schema/loader/SchemaMetaDataLoaderTest.java    |  4 +-
 ...hema.loader.spi.ShardingSphereMetaDataDecorator | 18 -----
 ....schema.loader.spi.ShardingSphereMetaDataLoader |  2 +-
 .../metadata/ShardingSphereDatabaseMetaData.java   | 11 ++-
 .../core/resultset/DatabaseMetaDataResultSet.java  |  6 +-
 .../jdbc/JDBCDatabaseCommunicationEngine.java      |  4 +-
 .../backend/response/query/QueryHeaderBuilder.java |  6 +-
 30 files changed, 241 insertions(+), 479 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataDecorator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataDecorator.java
deleted file mode 100644
index 8195b86..0000000
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataDecorator.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.encrypt.metadata;
-
-import org.apache.shardingsphere.encrypt.constant.EncryptOrder;
-import org.apache.shardingsphere.encrypt.rule.EncryptRule;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator;
-
-import java.util.Collection;
-import java.util.LinkedList;
-
-/**
- * Table meta data decorator for encrypt.
- */
-public final class EncryptMetaDataDecorator implements ShardingSphereMetaDataDecorator<EncryptRule> {
-    
-    @Override
-    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final EncryptRule encryptRule) {
-        return new PhysicalTableMetaData(getEncryptColumnMetaDataList(tableName, tableMetaData.getColumns().values(), encryptRule), tableMetaData.getIndexes().values());
-    }
-    
-    private Collection<PhysicalColumnMetaData> getEncryptColumnMetaDataList(final String tableName, 
-                                                                            final Collection<PhysicalColumnMetaData> originalColumnMetaDataList, final EncryptRule encryptRule) {
-        Collection<PhysicalColumnMetaData> result = new LinkedList<>();
-        Collection<String> derivedColumns = encryptRule.getAssistedQueryAndPlainColumns(tableName);
-        for (PhysicalColumnMetaData each : originalColumnMetaDataList) {
-            if (!derivedColumns.contains(each.getName())) {
-                result.add(getEncryptColumnMetaData(tableName, each, encryptRule));
-            }
-        }
-        return result;
-    }
-    
-    private PhysicalColumnMetaData getEncryptColumnMetaData(final String tableName, final PhysicalColumnMetaData originalColumnMetaData, final EncryptRule encryptRule) {
-        if (!encryptRule.isCipherColumn(tableName, originalColumnMetaData.getName())) {
-            return originalColumnMetaData;
-        }
-        String logicColumnName = encryptRule.getLogicColumnOfCipher(tableName, originalColumnMetaData.getName());
-        String plainColumnName = encryptRule.findPlainColumn(tableName, logicColumnName).orElse(null);
-        String assistedQueryColumnName = encryptRule.findAssistedQueryColumn(tableName, logicColumnName).orElse(null);
-        return new EncryptColumnMetaData(
-                logicColumnName, originalColumnMetaData.getDataType(), originalColumnMetaData.getDataTypeName(), originalColumnMetaData.isPrimaryKey(), originalColumnMetaData.getName(),
-                plainColumnName, assistedQueryColumnName);
-    }
-    
-    @Override
-    public int getOrder() {
-        return EncryptOrder.ORDER;
-    }
-    
-    @Override
-    public Class<EncryptRule> getTypeClass() {
-        return EncryptRule.class;
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoader.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoader.java
index 6d82e09..292d2f4 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoader.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoader.java
@@ -22,15 +22,15 @@ import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
+import org.apache.shardingsphere.infra.metadata.schema.loader.physical.PhysicalTableMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.loader.physical.PhysicalTableMetaDataLoader;
 
 import javax.sql.DataSource;
 import java.sql.SQLException;
 import java.util.Collection;
-import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Optional;
 
@@ -40,23 +40,38 @@ import java.util.Optional;
 public final class EncryptMetaDataLoader implements ShardingSphereMetaDataLoader<EncryptRule> {
     
     @Override
-    public PhysicalSchemaMetaData load(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
-                                       final EncryptRule encryptRule, final ConfigurationProperties props, final Collection<String> excludedTableNames) throws SQLException {
-        DataSource dataSource = dataSourceMap.values().iterator().next();
-        Collection<String> encryptTableNames = encryptRule.getEncryptTableNames();
-        Map<String, PhysicalTableMetaData> result = new HashMap<>(encryptTableNames.size(), 1);
-        for (String each : encryptTableNames) {
-            if (!excludedTableNames.contains(each)) {
-                PhysicalTableMetaDataLoader.load(dataSource, each, databaseType).ifPresent(tableMetaData -> result.put(each, tableMetaData));
+    public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
+                                                final EncryptRule encryptRule, final ConfigurationProperties props) throws SQLException {
+        return encryptRule.findEncryptTable(tableName).isPresent() ? PhysicalTableMetaDataLoader.load(dataSourceMap.values().iterator().next(), tableName, databaseType) : Optional.empty();
+    }
+    
+    @Override
+    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final EncryptRule encryptRule) {
+        return new PhysicalTableMetaData(getEncryptColumnMetaDataList(tableName, tableMetaData.getColumns().values(), encryptRule), tableMetaData.getIndexes().values());
+    }
+    
+    private Collection<PhysicalColumnMetaData> getEncryptColumnMetaDataList(final String tableName,
+                                                                            final Collection<PhysicalColumnMetaData> originalColumnMetaDataList, final EncryptRule encryptRule) {
+        Collection<PhysicalColumnMetaData> result = new LinkedList<>();
+        Collection<String> derivedColumns = encryptRule.getAssistedQueryAndPlainColumns(tableName);
+        for (PhysicalColumnMetaData each : originalColumnMetaDataList) {
+            if (!derivedColumns.contains(each.getName())) {
+                result.add(getEncryptColumnMetaData(tableName, each, encryptRule));
             }
         }
-        return new PhysicalSchemaMetaData(result);
+        return result;
     }
     
-    @Override
-    public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
-                                                final EncryptRule encryptRule, final ConfigurationProperties props) throws SQLException {
-        return encryptRule.findEncryptTable(tableName).isPresent() ? PhysicalTableMetaDataLoader.load(dataSourceMap.values().iterator().next(), tableName, databaseType) : Optional.empty();
+    private PhysicalColumnMetaData getEncryptColumnMetaData(final String tableName, final PhysicalColumnMetaData originalColumnMetaData, final EncryptRule encryptRule) {
+        if (!encryptRule.isCipherColumn(tableName, originalColumnMetaData.getName())) {
+            return originalColumnMetaData;
+        }
+        String logicColumnName = encryptRule.getLogicColumnOfCipher(tableName, originalColumnMetaData.getName());
+        String plainColumnName = encryptRule.findPlainColumn(tableName, logicColumnName).orElse(null);
+        String assistedQueryColumnName = encryptRule.findAssistedQueryColumn(tableName, logicColumnName).orElse(null);
+        return new EncryptColumnMetaData(
+                logicColumnName, originalColumnMetaData.getDataType(), originalColumnMetaData.getDataTypeName(), originalColumnMetaData.isPrimaryKey(), originalColumnMetaData.getName(),
+                plainColumnName, assistedQueryColumnName);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
index d4f8e39..600c1fe 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.encrypt.api.config.rule.EncryptTableRuleConfigu
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
 import org.apache.shardingsphere.encrypt.spi.QueryAssistedEncryptAlgorithm;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 
 import java.util.Collection;
@@ -41,7 +41,7 @@ import java.util.stream.Collectors;
 /**
  * Encrypt rule.
  */
-public final class EncryptRule implements ShardingSphereRule {
+public final class EncryptRule implements TableContainedRule {
     
     static {
         ShardingSphereServiceLoader.register(EncryptAlgorithm.class);
@@ -267,4 +267,9 @@ public final class EncryptRule implements ShardingSphereRule {
         }
         return Optional.empty();
     }
+    
+    @Override
+    public Collection<String> getTables() {
+        return tables.keySet();
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator
deleted file mode 100644
index 440705f..0000000
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator
+++ /dev/null
@@ -1,18 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-org.apache.shardingsphere.encrypt.metadata.EncryptMetaDataDecorator
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataDecoratorTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataDecoratorTest.java
deleted file mode 100644
index 3f9d7fb..0000000
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataDecoratorTest.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.encrypt.metadata;
-
-import org.apache.shardingsphere.encrypt.rule.EncryptRule;
-import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator;
-import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public final class EncryptMetaDataDecoratorTest {
-    
-    static {
-        ShardingSphereServiceLoader.register(ShardingSphereMetaDataDecorator.class);
-    }
-    
-    @Test
-    public void assertDecorate() {
-        EncryptRule rule = createEncryptRule();
-        EncryptMetaDataDecorator decorator = (EncryptMetaDataDecorator) OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(rule), ShardingSphereMetaDataDecorator.class).get(rule);
-        PhysicalTableMetaData actual = decorator.decorate("t_encrypt", createTableMetaData(), rule);
-        assertThat(actual.getColumns().size(), is(2));
-        assertTrue(actual.getColumns().containsKey("id"));
-        assertTrue(actual.getColumns().containsKey("pwd"));
-    }
-    
-    private EncryptRule createEncryptRule() {
-        EncryptRule result = mock(EncryptRule.class);
-        when(result.getLogicColumnOfCipher("t_encrypt", "pwd_cipher")).thenReturn("pwd");
-        when(result.isCipherColumn("t_encrypt", "pwd_cipher")).thenReturn(true);
-        when(result.getAssistedQueryAndPlainColumns("t_encrypt")).thenReturn(Collections.singletonList("pwd_plain"));
-        return result;
-    }
-    
-    private PhysicalTableMetaData createTableMetaData() {
-        Collection<PhysicalColumnMetaData> columns = Arrays.asList(new PhysicalColumnMetaData("id", 1, "int", true, true, true), 
-                new PhysicalColumnMetaData("pwd_cipher", 2, "varchar", false, false, true), new PhysicalColumnMetaData("pwd_plain", 2, "varchar", false, false, true));
-        return new PhysicalTableMetaData(columns, Collections.emptyList());
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoaderTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoaderTest.java
index 99e31d3..bc4b157 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoaderTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/metadata/EncryptMetaDataLoaderTest.java
@@ -23,10 +23,10 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -37,6 +37,8 @@ import javax.sql.DataSource;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Optional;
 
@@ -93,17 +95,6 @@ public final class EncryptMetaDataLoaderTest {
     }
     
     @Test
-    public void assertLoad() throws SQLException {
-        EncryptRule rule = createEncryptRule();
-        EncryptMetaDataLoader loader = (EncryptMetaDataLoader) OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(rule), ShardingSphereMetaDataLoader.class).get(rule);
-        PhysicalSchemaMetaData actual = loader.load(
-                databaseType, Collections.singletonMap("logic_db", dataSource), new DataNodes(Collections.singletonList(rule)), rule, props, Collections.emptyList());
-        assertThat(actual.get(TABLE_NAME).getColumnMetaData(0).getName(), is("id"));
-        assertThat(actual.get(TABLE_NAME).getColumnMetaData(1).getName(), is("pwd_cipher"));
-        assertThat(actual.get(TABLE_NAME).getColumnMetaData(2).getName(), is("pwd_plain"));
-    }
-    
-    @Test
     public void assertLoadByExistedTable() throws SQLException {
         EncryptRule rule = createEncryptRule();
         EncryptMetaDataLoader loader = (EncryptMetaDataLoader) OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(rule), ShardingSphereMetaDataLoader.class).get(rule);
@@ -123,10 +114,28 @@ public final class EncryptMetaDataLoaderTest {
         assertFalse(actual.isPresent());
     }
     
+    @Test
+    public void assertDecorate() {
+        EncryptRule rule = createEncryptRule();
+        EncryptMetaDataLoader loader = (EncryptMetaDataLoader) OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(rule), ShardingSphereMetaDataLoader.class).get(rule);
+        PhysicalTableMetaData actual = loader.decorate("t_encrypt", createTableMetaData(), rule);
+        assertThat(actual.getColumns().size(), is(2));
+        assertTrue(actual.getColumns().containsKey("id"));
+        assertTrue(actual.getColumns().containsKey("pwd"));
+    }
+    
     private EncryptRule createEncryptRule() {
         EncryptRule result = mock(EncryptRule.class);
-        when(result.getEncryptTableNames()).thenReturn(Collections.singletonList(TABLE_NAME));
         when(result.findEncryptTable(TABLE_NAME)).thenReturn(Optional.of(mock(EncryptTable.class)));
+        when(result.getLogicColumnOfCipher("t_encrypt", "pwd_cipher")).thenReturn("pwd");
+        when(result.isCipherColumn("t_encrypt", "pwd_cipher")).thenReturn(true);
+        when(result.getAssistedQueryAndPlainColumns("t_encrypt")).thenReturn(Collections.singletonList("pwd_plain"));
         return result;
     }
+    
+    private PhysicalTableMetaData createTableMetaData() {
+        Collection<PhysicalColumnMetaData> columns = Arrays.asList(new PhysicalColumnMetaData("id", 1, "int", true, true, true),
+                new PhysicalColumnMetaData("pwd_cipher", 2, "varchar", false, false, true), new PhysicalColumnMetaData("pwd_plain", 2, "varchar", false, false, true));
+        return new PhysicalTableMetaData(columns, Collections.emptyList());
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
index 5ce9197..c3ce5b8 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-common/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
@@ -150,6 +150,11 @@ public final class EncryptRuleTest {
         assertFalse(new EncryptRule(createEncryptRuleConfiguration()).findPlainColumn("t_encrypt", "notExistLogicColumn").isPresent());
     }
     
+    @Test
+    public void assertGetTables() {
+        assertThat(new EncryptRule(createEncryptRuleConfiguration()).getTables(), is(Collections.singleton("t_encrypt")));
+    }
+    
     private EncryptRuleConfiguration createEncryptRuleConfiguration() {
         ShardingSphereAlgorithmConfiguration encryptAlgorithmConfig = new ShardingSphereAlgorithmConfiguration("QUERY_ASSISTED_TEST", new Properties());
         EncryptColumnRuleConfiguration pwdColumnConfig = new EncryptColumnRuleConfiguration("pwd", "pwd_cipher", "", "pwd_plain", "test_encryptor");
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataDecorator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataDecorator.java
deleted file mode 100644
index d44e320..0000000
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataDecorator.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.sharding.metadata;
-
-import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator;
-import org.apache.shardingsphere.sharding.constant.ShardingOrder;
-import org.apache.shardingsphere.sharding.rule.ShardingRule;
-import org.apache.shardingsphere.sharding.rule.TableRule;
-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.PhysicalTableMetaData;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Map.Entry;
-import java.util.Optional;
-
-/**
- * Table meta data decorator for sharding.
- */
-public final class ShardingMetaDataDecorator implements ShardingSphereMetaDataDecorator<ShardingRule> {
-    
-    @Override
-    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final ShardingRule shardingRule) {
-        return shardingRule.findTableRule(tableName).map(
-            tableRule -> new PhysicalTableMetaData(getColumnMetaDataList(tableMetaData, tableRule), getIndexMetaDataList(tableMetaData, tableRule))).orElse(tableMetaData);
-    }
-    
-    private Collection<PhysicalColumnMetaData> getColumnMetaDataList(final PhysicalTableMetaData tableMetaData, final TableRule tableRule) {
-        Optional<String> generateKeyColumn = tableRule.getGenerateKeyColumn();
-        if (!generateKeyColumn.isPresent()) {
-            return tableMetaData.getColumns().values();
-        }
-        Collection<PhysicalColumnMetaData> result = new LinkedList<>();
-        for (Entry<String, PhysicalColumnMetaData> entry : tableMetaData.getColumns().entrySet()) {
-            if (entry.getKey().equalsIgnoreCase(generateKeyColumn.get())) {
-                result.add(new PhysicalColumnMetaData(
-                    entry.getValue().getName(), entry.getValue().getDataType(), entry.getValue().getDataTypeName(), entry.getValue().isPrimaryKey(), true, entry.getValue().isCaseSensitive()));
-            } else {
-                result.add(entry.getValue());
-            }
-        }
-        return result;
-    }
-    
-    private Collection<PhysicalIndexMetaData> getIndexMetaDataList(final PhysicalTableMetaData tableMetaData, final TableRule tableRule) {
-        Collection<PhysicalIndexMetaData> result = new HashSet<>();
-        for (Entry<String, PhysicalIndexMetaData> entry : tableMetaData.getIndexes().entrySet()) {
-            for (DataNode each : tableRule.getActualDataNodes()) {
-                getLogicIndex(entry.getKey(), each.getTableName()).ifPresent(logicIndex -> result.add(new PhysicalIndexMetaData(logicIndex)));
-            }
-        }
-        return result;
-    }
-    
-    private Optional<String> getLogicIndex(final String actualIndexName, final String actualTableName) {
-        String indexNameSuffix = "_" + actualTableName;
-        return actualIndexName.endsWith(indexNameSuffix) ? Optional.of(actualIndexName.replace(indexNameSuffix, "")) : Optional.empty();
-    }
-    
-    @Override
-    public int getOrder() {
-        return ShardingOrder.ORDER;
-    }
-    
-    @Override
-    public Class<ShardingRule> getTypeClass() {
-        return ShardingRule.class;
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataLoader.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataLoader.java
index 952f097..4962258 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataLoader.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataLoader.java
@@ -25,18 +25,21 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.loader.physical.PhysicalTableMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
+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.PhysicalTableMetaData;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-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.metadata.schema.loader.physical.PhysicalTableMetaDataLoader;
 
 import javax.sql.DataSource;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -59,18 +62,6 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
     private static final int FUTURE_GET_TIME_OUT_SECOND = 5;
     
     @Override
-    public PhysicalSchemaMetaData load(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
-                                       final ShardingRule rule, final ConfigurationProperties props, final Collection<String> excludedTableNames) throws SQLException {
-        PhysicalSchemaMetaData result = new PhysicalSchemaMetaData(new HashMap<>(rule.getTableRules().size(), 1));
-        for (TableRule each : rule.getTableRules()) {
-            if (!excludedTableNames.contains(each.getLogicTable())) {
-                load(each.getLogicTable(), databaseType, dataSourceMap, dataNodes, rule, props).ifPresent(tableMetaData -> result.put(each.getLogicTable(), tableMetaData));
-            }
-        }
-        return result;
-    }
-    
-    @Override
     public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, final DataNodes dataNodes,
                                                 final ShardingRule rule, final ConfigurationProperties props) throws SQLException {
         if (!rule.findTableRule(tableName).isPresent()) {
@@ -127,10 +118,9 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
     }
     
     private void checkUniformed(final String logicTableName, final Map<String, PhysicalTableMetaData> actualTableMetaDataMap, final ShardingRule shardingRule) {
-        ShardingMetaDataDecorator decorator = new ShardingMetaDataDecorator();
-        PhysicalTableMetaData sample = decorator.decorate(logicTableName, actualTableMetaDataMap.values().iterator().next(), shardingRule);
+        PhysicalTableMetaData sample = decorate(logicTableName, actualTableMetaDataMap.values().iterator().next(), shardingRule);
         Collection<TableMetaDataViolation> violations = actualTableMetaDataMap.entrySet().stream()
-                .filter(entry -> !sample.equals(decorator.decorate(logicTableName, entry.getValue(), shardingRule)))
+                .filter(entry -> !sample.equals(decorate(logicTableName, entry.getValue(), shardingRule)))
                 .map(entry -> new TableMetaDataViolation(entry.getKey(), entry.getValue())).collect(Collectors.toList());
         throwExceptionIfNecessary(violations, logicTableName);
     }
@@ -145,6 +135,44 @@ public final class ShardingMetaDataLoader implements ShardingSphereMetaDataLoade
             throw new ShardingSphereException(errorMessage.toString(), logicTableName);
         }
     }
+
+    @Override
+    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final ShardingRule shardingRule) {
+        return shardingRule.findTableRule(tableName).map(
+            tableRule -> new PhysicalTableMetaData(getColumnMetaDataList(tableMetaData, tableRule), getIndexMetaDataList(tableMetaData, tableRule))).orElse(tableMetaData);
+    }
+    
+    private Collection<PhysicalColumnMetaData> getColumnMetaDataList(final PhysicalTableMetaData tableMetaData, final TableRule tableRule) {
+        Optional<String> generateKeyColumn = tableRule.getGenerateKeyColumn();
+        if (!generateKeyColumn.isPresent()) {
+            return tableMetaData.getColumns().values();
+        }
+        Collection<PhysicalColumnMetaData> result = new LinkedList<>();
+        for (Entry<String, PhysicalColumnMetaData> entry : tableMetaData.getColumns().entrySet()) {
+            if (entry.getKey().equalsIgnoreCase(generateKeyColumn.get())) {
+                result.add(new PhysicalColumnMetaData(
+                        entry.getValue().getName(), entry.getValue().getDataType(), entry.getValue().getDataTypeName(), entry.getValue().isPrimaryKey(), true, entry.getValue().isCaseSensitive()));
+            } else {
+                result.add(entry.getValue());
+            }
+        }
+        return result;
+    }
+    
+    private Collection<PhysicalIndexMetaData> getIndexMetaDataList(final PhysicalTableMetaData tableMetaData, final TableRule tableRule) {
+        Collection<PhysicalIndexMetaData> result = new HashSet<>();
+        for (Entry<String, PhysicalIndexMetaData> entry : tableMetaData.getIndexes().entrySet()) {
+            for (DataNode each : tableRule.getActualDataNodes()) {
+                getLogicIndex(entry.getKey(), each.getTableName()).ifPresent(logicIndex -> result.add(new PhysicalIndexMetaData(logicIndex)));
+            }
+        }
+        return result;
+    }
+    
+    private Optional<String> getLogicIndex(final String actualIndexName, final String actualTableName) {
+        String indexNameSuffix = "_" + actualTableName;
+        return actualIndexName.endsWith(indexNameSuffix) ? Optional.of(actualIndexName.replace(indexNameSuffix, "")) : Optional.empty();
+    }
     
     @Override
     public int getOrder() {
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 191cdb2..cd71219 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
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.sharding.rule;
 
+import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
 import lombok.AccessLevel;
@@ -25,6 +26,7 @@ import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmF
 import org.apache.shardingsphere.infra.config.exception.ShardingSphereConfigurationException;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
+import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
 import org.apache.shardingsphere.sharding.algorithm.config.AlgorithmProvidedShardingRuleConfiguration;
@@ -55,7 +57,7 @@ import java.util.stream.Collectors;
  * Sharding rule.
  */
 @Getter
-public final class ShardingRule implements DataNodeContainedRule {
+public final class ShardingRule implements DataNodeContainedRule, TableContainedRule {
     
     static {
         ShardingSphereServiceLoader.register(ShardingAlgorithm.class);
@@ -421,4 +423,9 @@ public final class ShardingRule implements DataNodeContainedRule {
     public Optional<String> findLogicTableByActualTable(final String actualTable) {
         return findTableRuleByActualTable(actualTable).map(TableRule::getLogicTable);
     }
+    
+    @Override
+    public Collection<String> getTables() {
+        return tableRules.stream().map((Function<TableRule, String>) TableRule::getLogicTable).collect(Collectors.toList());
+    }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator
deleted file mode 100644
index d19b922..0000000
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator
+++ /dev/null
@@ -1,18 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-org.apache.shardingsphere.sharding.metadata.ShardingMetaDataDecorator
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
index d29e7aa..e7c0e20 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-common/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
@@ -295,6 +295,11 @@ public final class ShardingRuleTest {
         assertFalse(createMinimumShardingRule().tableRuleExists(Collections.singletonList("table_0")));
     }
     
+    @Test
+    public void assertGetTables() {
+        assertThat(createMaximumShardingRule().getTables(), is(Arrays.asList("logic_table", "sub_logic_table")));
+    }
+    
     private ShardingRule createMaximumShardingRule() {
         ShardingRuleConfiguration shardingRuleConfig = new ShardingRuleConfiguration();
         ShardingTableRuleConfiguration shardingTableRuleConfig = createTableRuleConfiguration("LOGIC_TABLE", "ds_${0..1}.table_${0..2}");
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
index 9cac3e7..8b37796 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
@@ -47,11 +47,11 @@ public final class DataNodes {
      * @return data nodes
      */
     public Collection<DataNode> getDataNodes(final String tableName) {
-        Optional<DataNodeContainedRule> dataNodeRoutedRule = rules.stream().filter(each -> each instanceof DataNodeContainedRule).findFirst().map(rule -> (DataNodeContainedRule) rule);
-        if (!dataNodeRoutedRule.isPresent()) {
+        Optional<DataNodeContainedRule> dataNodeContainedRule = rules.stream().filter(each -> each instanceof DataNodeContainedRule).findFirst().map(rule -> (DataNodeContainedRule) rule);
+        if (!dataNodeContainedRule.isPresent()) {
             return Collections.emptyList();
         }
-        Collection<DataNode> result = new LinkedList<>(dataNodeRoutedRule.get().getAllDataNodes().get(tableName));
+        Collection<DataNode> result = new LinkedList<>(dataNodeContainedRule.get().getAllDataNodes().get(tableName));
         for (ShardingSphereRule each : rules) {
             if (each instanceof DataSourceContainedRule) {
                 for (Entry<String, Collection<String>> entry : ((DataSourceContainedRule) each).getDataSourceMapper().entrySet()) {
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 fe65293..ddbf7ce 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
@@ -22,12 +22,11 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
-import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
 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.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 
@@ -37,7 +36,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.TreeSet;
+import java.util.Optional;
 
 /**
  * Schema meta data loader.
@@ -47,7 +46,6 @@ public final class SchemaMetaDataLoader {
     
     static {
         ShardingSphereServiceLoader.register(ShardingSphereMetaDataLoader.class);
-        ShardingSphereServiceLoader.register(ShardingSphereMetaDataDecorator.class);
     }
     
     /**
@@ -60,30 +58,50 @@ public final class SchemaMetaDataLoader {
      * @return schema meta data
      * @throws SQLException SQL exception
      */
-    @SuppressWarnings({"unchecked", "rawtypes"})
     public static PhysicalSchemaMetaData load(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, 
                                               final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
-        Collection<String> excludedTableNames = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+        PhysicalSchemaMetaData result = loadSchemaMetaData(databaseType, dataSourceMap, rules, props);
+        decorateSchemaMetaData(rules, result);
+        return result;
+    }
+    
+    @SuppressWarnings("rawtypes")
+    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();
+        DataNodes dataNodes = new DataNodes(rules);
         for (Entry<ShardingSphereRule, ShardingSphereMetaDataLoader> entry : OrderedSPIRegistry.getRegisteredServices(rules, ShardingSphereMetaDataLoader.class).entrySet()) {
-            PhysicalSchemaMetaData schemaMetaData = entry.getValue().load(databaseType, dataSourceMap, new DataNodes(rules), entry.getKey(), props, excludedTableNames);
-            excludedTableNames.addAll(schemaMetaData.getAllTableNames());
-            if (entry.getKey() instanceof DataNodeContainedRule) {
-                excludedTableNames.addAll(((DataNodeContainedRule) entry.getKey()).getAllActualTables());
+            if (entry.getKey() instanceof TableContainedRule) {
+                result.getTables().putAll(loadSchemaMetaDataByRule(databaseType, dataSourceMap, (TableContainedRule) entry.getKey(), entry.getValue(), props, dataNodes, result.getAllTableNames()));
+            }
+        }
+        return result;
+    }
+    
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    private static Map<String, PhysicalTableMetaData> loadSchemaMetaDataByRule(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
+                                                                               final TableContainedRule rule, final ShardingSphereMetaDataLoader loader, final ConfigurationProperties props, 
+                                                                               final DataNodes dataNodes, final Collection<String> existedTables) throws SQLException {
+        Collection<String> tables = rule.getTables();
+        Map<String, PhysicalTableMetaData> result = new HashMap<>(tables.size(), 1);
+        for (String each : tables) {
+            if (!existedTables.contains(each)) {
+                Optional<PhysicalTableMetaData> tableMetaData = loader.load(each, databaseType, dataSourceMap, dataNodes, rule, props);
+                tableMetaData.ifPresent(optional -> result.put(each, optional));
             }
-            result.merge(schemaMetaData);
         }
-        decorate(rules, result);
         return result;
     }
     
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private static void decorate(final Collection<ShardingSphereRule> rules, final PhysicalSchemaMetaData schemaMetaData) {
+    private static void decorateSchemaMetaData(final Collection<ShardingSphereRule> rules, final PhysicalSchemaMetaData schemaMetaData) {
         Map<String, PhysicalTableMetaData> tableMetaDataMap = new HashMap<>(schemaMetaData.getAllTableNames().size(), 1);
-        Map<ShardingSphereRule, ShardingSphereMetaDataDecorator> decorators = OrderedSPIRegistry.getRegisteredServices(rules, ShardingSphereMetaDataDecorator.class);
+        Map<ShardingSphereRule, ShardingSphereMetaDataLoader> loaders = OrderedSPIRegistry.getRegisteredServices(rules, ShardingSphereMetaDataLoader.class);
         for (String each : schemaMetaData.getAllTableNames()) {
-            for (Entry<ShardingSphereRule, ShardingSphereMetaDataDecorator> entry : decorators.entrySet()) {
-                tableMetaDataMap.put(each, entry.getValue().decorate(each, tableMetaDataMap.getOrDefault(each, schemaMetaData.get(each)), entry.getKey()));
+            for (Entry<ShardingSphereRule, ShardingSphereMetaDataLoader> entry : loaders.entrySet()) {
+                if (entry.getKey() instanceof TableContainedRule) {
+                    tableMetaDataMap.put(each, entry.getValue().decorate(each, tableMetaDataMap.getOrDefault(each, schemaMetaData.get(each)), (TableContainedRule) entry.getKey()));
+                }
             }
         }
         schemaMetaData.merge(new PhysicalSchemaMetaData(tableMetaDataMap));
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/TableMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/TableMetaDataLoader.java
index 8026c15..f92c047 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/TableMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/TableMetaDataLoader.java
@@ -22,10 +22,10 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
-import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
 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;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
 
@@ -44,7 +44,6 @@ public final class TableMetaDataLoader {
     
     static {
         ShardingSphereServiceLoader.register(ShardingSphereMetaDataLoader.class);
-        ShardingSphereServiceLoader.register(ShardingSphereMetaDataDecorator.class);
     }
     
     /**
@@ -62,9 +61,11 @@ public final class TableMetaDataLoader {
     public static Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap, 
                                                        final Collection<ShardingSphereRule> rules, final ConfigurationProperties props) throws SQLException {
         for (Entry<ShardingSphereRule, ShardingSphereMetaDataLoader> entry : OrderedSPIRegistry.getRegisteredServices(rules, ShardingSphereMetaDataLoader.class).entrySet()) {
-            Optional<PhysicalTableMetaData> result = entry.getValue().load(tableName, databaseType, dataSourceMap, new DataNodes(rules), entry.getKey(), props);
-            if (result.isPresent()) {
-                return Optional.of(decorate(tableName, result.get(), rules));
+            if (entry.getKey() instanceof TableContainedRule) {
+                Optional<PhysicalTableMetaData> result = entry.getValue().load(tableName, databaseType, dataSourceMap, new DataNodes(rules), (TableContainedRule) entry.getKey(), props);
+                if (result.isPresent()) {
+                    return Optional.of(decorate(tableName, result.get(), rules));
+                }
             }
         }
         return Optional.empty();
@@ -72,10 +73,12 @@ public final class TableMetaDataLoader {
     
     @SuppressWarnings({"unchecked", "rawtypes"})
     private static PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final Collection<ShardingSphereRule> rules) {
-        Map<ShardingSphereRule, ShardingSphereMetaDataDecorator> decorators = OrderedSPIRegistry.getRegisteredServices(rules, ShardingSphereMetaDataDecorator.class);
+        Map<ShardingSphereRule, ShardingSphereMetaDataLoader> decorators = OrderedSPIRegistry.getRegisteredServices(rules, ShardingSphereMetaDataLoader.class);
         PhysicalTableMetaData result = null;
-        for (Entry<ShardingSphereRule, ShardingSphereMetaDataDecorator> entry : decorators.entrySet()) {
-            result = entry.getValue().decorate(tableName, null == result ? tableMetaData : result, entry.getKey());
+        for (Entry<ShardingSphereRule, ShardingSphereMetaDataLoader> entry : decorators.entrySet()) {
+            if (entry.getKey() instanceof TableContainedRule) {
+                result = entry.getValue().decorate(tableName, null == result ? tableMetaData : result, (TableContainedRule) entry.getKey());
+            }
         }
         return Optional.ofNullable(result).orElse(tableMetaData);
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataLoader.java
index df66b9a..ab9872d 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataLoader.java
@@ -20,14 +20,12 @@ package org.apache.shardingsphere.infra.metadata.schema.loader.spi;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
-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;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPI;
 
 import javax.sql.DataSource;
 import java.sql.SQLException;
-import java.util.Collection;
 import java.util.Map;
 import java.util.Optional;
 
@@ -36,22 +34,7 @@ import java.util.Optional;
  * 
  * @param <T> type of base rule
  */
-public interface ShardingSphereMetaDataLoader<T extends ShardingSphereRule> extends OrderedSPI<T> {
-    
-    /**
-     * Load schema meta data.
-     * 
-     * @param databaseType database type
-     * @param dataSourceMap data source map
-     * @param dataNodes data nodes
-     * @param rule rule
-     * @param props configuration properties
-     * @param excludedTableNames excluded table names
-     * @return table name and meta data map
-     * @throws SQLException SQL exception
-     */
-    PhysicalSchemaMetaData load(DatabaseType databaseType, Map<String, DataSource> dataSourceMap,
-                                DataNodes dataNodes, T rule, ConfigurationProperties props, Collection<String> excludedTableNames) throws SQLException;
+public interface ShardingSphereMetaDataLoader<T extends TableContainedRule> extends OrderedSPI<T> {
     
     /**
      * Load table meta data.
@@ -67,4 +50,14 @@ public interface ShardingSphereMetaDataLoader<T extends ShardingSphereRule> exte
      */
     Optional<PhysicalTableMetaData> load(String tableName, 
                                          DatabaseType databaseType, Map<String, DataSource> dataSourceMap, DataNodes dataNodes, T rule, ConfigurationProperties props) throws SQLException;
+    
+    /**
+     * Decorate table meta data.
+     *
+     * @param tableName table name
+     * @param tableMetaData table meta data
+     * @param rule rule
+     * @return decorated table meta data
+     */
+    PhysicalTableMetaData decorate(String tableName, PhysicalTableMetaData tableMetaData, T rule);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalSchemaMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalSchemaMetaData.java
index 82bd18a..a744a85 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalSchemaMetaData.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/model/physical/PhysicalSchemaMetaData.java
@@ -17,6 +17,8 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.model.physical;
 
+import lombok.Getter;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -27,6 +29,7 @@ import java.util.concurrent.ConcurrentHashMap;
 /**
  * Physical schema meta data.
  */
+@Getter
 public final class PhysicalSchemaMetaData {
     
     private final Map<String, PhysicalTableMetaData> tables;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataDecorator.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/type/TableContainedRule.java
similarity index 58%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataDecorator.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/type/TableContainedRule.java
index 8323860..a253831 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/loader/spi/ShardingSphereMetaDataDecorator.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/type/TableContainedRule.java
@@ -15,24 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.metadata.schema.loader.spi;
+package org.apache.shardingsphere.infra.rule.type;
 
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.spi.ordered.OrderedSPI;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
+
+import java.util.Collection;
 
 /**
- * ShardingSphere meta data decorator.
+ * ShardingSphere rule contains table.
  */
-public interface ShardingSphereMetaDataDecorator<T extends ShardingSphereRule> extends OrderedSPI<T> {
+public interface TableContainedRule extends ShardingSphereRule {
     
     /**
-     * Decorate table meta data.
+     * Get tables.
      *
-     * @param tableName table name
-     * @param tableMetaData table meta data
-     * @param rule rule
-     * @return decorated table meta data
+     * @return tables
      */
-    PhysicalTableMetaData decorate(String tableName, PhysicalTableMetaData tableMetaData, T rule);
+    Collection<String> getTables();
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/decorator/DataNodeRoutedLogicMetaDataDecoratorFixture.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/decorator/DataNodeRoutedLogicMetaDataDecoratorFixture.java
deleted file mode 100644
index 69ee628..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/decorator/DataNodeRoutedLogicMetaDataDecoratorFixture.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.metadata.schema.fixture.decorator;
-
-import org.apache.shardingsphere.infra.metadata.schema.fixture.rule.DataNodeContainedFixtureRule;
-import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
-
-import java.util.Collections;
-
-public final class DataNodeRoutedLogicMetaDataDecoratorFixture implements ShardingSphereMetaDataDecorator<DataNodeContainedFixtureRule> {
-    
-    @Override
-    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final DataNodeContainedFixtureRule rule) {
-        PhysicalColumnMetaData columnMetaData = new PhysicalColumnMetaData("id", 1, "INT", true, true, false);
-        return new PhysicalTableMetaData(Collections.singletonList(columnMetaData), Collections.emptyList());
-    }
-    
-    @Override
-    public int getOrder() {
-        return 1;
-    }
-    
-    @Override
-    public Class<DataNodeContainedFixtureRule> getTypeClass() {
-        return DataNodeContainedFixtureRule.class;
-    }
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/CommonFixtureLogicMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/CommonFixtureLogicMetaDataLoader.java
index b66243b..020ece1 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/CommonFixtureLogicMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/CommonFixtureLogicMetaDataLoader.java
@@ -22,34 +22,26 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.fixture.rule.CommonFixtureRule;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 
 import javax.sql.DataSource;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
 
 public final class CommonFixtureLogicMetaDataLoader implements ShardingSphereMetaDataLoader<CommonFixtureRule> {
     
     @Override
-    public PhysicalSchemaMetaData load(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                       final DataNodes dataNodes, final CommonFixtureRule rule, final ConfigurationProperties props, final Collection<String> excludedTableNames) {
-        Map<String, PhysicalTableMetaData> tables = new HashMap<>(2, 1);
-        tables.put("common_table_0", new PhysicalTableMetaData(Collections.emptyList(), Collections.emptyList()));
-        tables.put("common_table_1", new PhysicalTableMetaData(Collections.emptyList(), Collections.emptyList()));
-        return new PhysicalSchemaMetaData(tables);
-    }
-    
-    @Override
     public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
                                                 final DataNodes dataNodes, final CommonFixtureRule rule, final ConfigurationProperties props) {
         return Optional.empty();
     }
     
     @Override
+    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final CommonFixtureRule rule) {
+        return tableMetaData;
+    }
+    
+    @Override
     public int getOrder() {
         return 0;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeRoutedFixtureLogicMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeContainedFixtureLogicMetaDataLoader.java
similarity index 72%
rename from shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeRoutedFixtureLogicMetaDataLoader.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeContainedFixtureLogicMetaDataLoader.java
index b7a4163..2c7d010 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeRoutedFixtureLogicMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/loader/DataNodeContainedFixtureLogicMetaDataLoader.java
@@ -22,26 +22,15 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNodes;
 import org.apache.shardingsphere.infra.metadata.schema.fixture.rule.DataNodeContainedFixtureRule;
 import org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader;
-import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalSchemaMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.physical.PhysicalTableMetaData;
 
 import javax.sql.DataSource;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
 
-public final class DataNodeRoutedFixtureLogicMetaDataLoader implements ShardingSphereMetaDataLoader<DataNodeContainedFixtureRule> {
-    
-    @Override
-    public PhysicalSchemaMetaData load(final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
-                                       final DataNodes dataNodes, final DataNodeContainedFixtureRule rule, final ConfigurationProperties props, final Collection<String> excludedTableNames) {
-        Map<String, PhysicalTableMetaData> tables = new HashMap<>(2, 1);
-        tables.put("data_node_routed_table_0", new PhysicalTableMetaData(Collections.emptyList(), Collections.emptyList()));
-        tables.put("data_node_routed_table_1", new PhysicalTableMetaData(Collections.emptyList(), Collections.emptyList()));
-        return new PhysicalSchemaMetaData(tables);
-    }
+public final class DataNodeContainedFixtureLogicMetaDataLoader implements ShardingSphereMetaDataLoader<DataNodeContainedFixtureRule> {
     
     @Override
     public Optional<PhysicalTableMetaData> load(final String tableName, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap,
@@ -51,6 +40,12 @@ public final class DataNodeRoutedFixtureLogicMetaDataLoader implements ShardingS
     }
     
     @Override
+    public PhysicalTableMetaData decorate(final String tableName, final PhysicalTableMetaData tableMetaData, final DataNodeContainedFixtureRule rule) {
+        PhysicalColumnMetaData columnMetaData = new PhysicalColumnMetaData("id", 1, "INT", true, true, false);
+        return new PhysicalTableMetaData(Collections.singletonList(columnMetaData), Collections.emptyList());
+    }
+    
+    @Override
     public int getOrder() {
         return 1;
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/rule/CommonFixtureRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/rule/CommonFixtureRule.java
index 540b0b8..5f5407d 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/rule/CommonFixtureRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/rule/CommonFixtureRule.java
@@ -17,7 +17,15 @@
 
 package org.apache.shardingsphere.infra.metadata.schema.fixture.rule;
 
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 
-public final class CommonFixtureRule implements ShardingSphereRule {
+import java.util.Collection;
+import java.util.Collections;
+
+public final class CommonFixtureRule implements TableContainedRule {
+    
+    @Override
+    public Collection<String> getTables() {
+        return Collections.emptyList();
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/rule/DataNodeContainedFixtureRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/rule/DataNodeContainedFixtureRule.java
index afd4ece..aca92f5 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/rule/DataNodeContainedFixtureRule.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/fixture/rule/DataNodeContainedFixtureRule.java
@@ -19,13 +19,14 @@ package org.apache.shardingsphere.infra.metadata.schema.fixture.rule;
 
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.rule.type.DataNodeContainedRule;
+import org.apache.shardingsphere.infra.rule.type.TableContainedRule;
 
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Map;
 import java.util.Optional;
 
-public final class DataNodeContainedFixtureRule implements DataNodeContainedRule {
+public final class DataNodeContainedFixtureRule implements DataNodeContainedRule, TableContainedRule {
     
     @Override
     public Map<String, Collection<DataNode>> getAllDataNodes() {
@@ -51,4 +52,9 @@ public final class DataNodeContainedFixtureRule implements DataNodeContainedRule
     public Optional<String> findLogicTableByActualTable(final String actualTable) {
         return Optional.empty();
     }
+    
+    @Override
+    public Collection<String> getTables() {
+        return Arrays.asList("data_node_routed_table_0", "data_node_routed_table_1");
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoaderTest.java
index 63ff545..d91ab50 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/loader/SchemaMetaDataLoaderTest.java
@@ -57,9 +57,7 @@ public final class SchemaMetaDataLoaderTest {
     }
     
     private void assertPhysicalSchemaMetaData(final PhysicalSchemaMetaData actual) {
-        assertThat(actual.getAllTableNames().size(), is(4));
-        assertTrue(actual.containsTable("common_table_0"));
-        assertTrue(actual.containsTable("common_table_1"));
+        assertThat(actual.getAllTableNames().size(), is(2));
         assertTrue(actual.containsTable("data_node_routed_table_0"));
         assertTrue(actual.get("data_node_routed_table_0").getColumns().containsKey("id"));
         assertTrue(actual.containsTable("data_node_routed_table_1"));
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator b/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator
deleted file mode 100644
index 3ea5f37..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataDecorator
+++ /dev/null
@@ -1,18 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-org.apache.shardingsphere.infra.metadata.schema.fixture.decorator.DataNodeRoutedLogicMetaDataDecoratorFixture
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader b/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader
index e268e04..0ddcff9 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.loader.spi.ShardingSphereMetaDataLoader
@@ -16,4 +16,4 @@
 #
 
 org.apache.shardingsphere.infra.metadata.schema.fixture.loader.CommonFixtureLogicMetaDataLoader
-org.apache.shardingsphere.infra.metadata.schema.fixture.loader.DataNodeRoutedFixtureLogicMetaDataLoader
+org.apache.shardingsphere.infra.metadata.schema.fixture.loader.DataNodeContainedFixtureLogicMetaDataLoader
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
index ab7600f..e7339b5 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
@@ -208,9 +208,9 @@ public final class ShardingSphereDatabaseMetaData extends AdaptedDatabaseMetaDat
         if (null == tableNamePattern) {
             return null;
         }
-        Optional<DataNodeContainedRule> dataNodeRoutedRule = findDataNodeRoutedRule();
-        if (dataNodeRoutedRule.isPresent()) {
-            return dataNodeRoutedRule.get().findFirstActualTable(tableNamePattern).isPresent() ? "%" + tableNamePattern + "%" : tableNamePattern;
+        Optional<DataNodeContainedRule> dataNodeContainedRule = findDataNodeContainedRule();
+        if (dataNodeContainedRule.isPresent()) {
+            return dataNodeContainedRule.get().findFirstActualTable(tableNamePattern).isPresent() ? "%" + tableNamePattern + "%" : tableNamePattern;
         }
         return tableNamePattern;
     }
@@ -219,11 +219,10 @@ public final class ShardingSphereDatabaseMetaData extends AdaptedDatabaseMetaDat
         if (null == table) {
             return null;
         }
-        Optional<DataNodeContainedRule> dataNodeRoutedRule = findDataNodeRoutedRule();
-        return dataNodeRoutedRule.map(nodeRoutedRule -> nodeRoutedRule.findFirstActualTable(table).orElse(table)).orElse(table);
+        return findDataNodeContainedRule().map(each -> each.findFirstActualTable(table).orElse(table)).orElse(table);
     }
     
-    private Optional<DataNodeContainedRule> findDataNodeRoutedRule() {
+    private Optional<DataNodeContainedRule> findDataNodeContainedRule() {
         return rules.stream().filter(each -> each instanceof DataNodeContainedRule).findFirst().map(rule -> (DataNodeContainedRule) rule);
     }
     
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
index 28a5a2d..39cba2b 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
@@ -102,11 +102,11 @@ public final class DatabaseMetaDataResultSet extends AbstractUnsupportedDatabase
     
     private DatabaseMetaDataObject generateDatabaseMetaDataObject(final int tableNameColumnIndex, final int indexNameColumnIndex, final ResultSet resultSet) throws SQLException {
         DatabaseMetaDataObject result = new DatabaseMetaDataObject(resultSetMetaData.getColumnCount());
-        Optional<DataNodeContainedRule> dataNodeRoutedRule = findDataNodeRoutedRule();
+        Optional<DataNodeContainedRule> dataNodeContainedRule = findDataNodeContainedRule();
         for (int i = 1; i <= columnLabelIndexMap.size(); i++) {
             if (tableNameColumnIndex == i) {
                 String tableName = resultSet.getString(i);
-                Optional<String> logicTableName = dataNodeRoutedRule.isPresent() ? dataNodeRoutedRule.get().findLogicTableByActualTable(tableName) : Optional.empty();
+                Optional<String> logicTableName = dataNodeContainedRule.isPresent() ? dataNodeContainedRule.get().findLogicTableByActualTable(tableName) : Optional.empty();
                 result.addObject(logicTableName.orElse(tableName));
             } else if (indexNameColumnIndex == i) {
                 String tableName = resultSet.getString(tableNameColumnIndex);
@@ -119,7 +119,7 @@ public final class DatabaseMetaDataResultSet extends AbstractUnsupportedDatabase
         return result;
     }
     
-    private Optional<DataNodeContainedRule> findDataNodeRoutedRule() {
+    private Optional<DataNodeContainedRule> findDataNodeContainedRule() {
         return rules.stream().filter(each -> each instanceof DataNodeContainedRule).findFirst().map(rule -> (DataNodeContainedRule) rule);
     }
     
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
index a610840..152a1b1 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
@@ -131,9 +131,9 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
     }
     
     private boolean isNeedAccumulate(final SQLStatementContext<?> sqlStatementContext) {
-        Optional<DataNodeContainedRule> dataNodeRoutedRule = 
+        Optional<DataNodeContainedRule> dataNodeContainedRule = 
                 metaData.getRuleMetaData().getRules().stream().filter(each -> each instanceof DataNodeContainedRule).findFirst().map(rule -> (DataNodeContainedRule) rule);
-        return dataNodeRoutedRule.isPresent() && dataNodeRoutedRule.get().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames());
+        return dataNodeContainedRule.isPresent() && dataNodeContainedRule.get().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames());
     }
     
     private MergedResult mergeQuery(final SQLStatementContext<?> sqlStatementContext, final List<QueryResult> queryResults) throws SQLException {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilder.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilder.java
index 1f0fecf..6f7d656 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilder.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/query/QueryHeaderBuilder.java
@@ -68,12 +68,12 @@ public final class QueryHeaderBuilder {
     private static QueryHeader build(final ResultSetMetaData resultSetMetaData, final ShardingSphereMetaData metaData, final String columnName, final int columnIndex) throws SQLException {
         String schemaName = metaData.getName();
         String actualTableName = resultSetMetaData.getTableName(columnIndex);
-        Optional<DataNodeContainedRule> dataNodeRoutedRule = 
+        Optional<DataNodeContainedRule> dataNodeContainedRule = 
                 metaData.getRuleMetaData().getRules().stream().filter(each -> each instanceof DataNodeContainedRule).findFirst().map(rule -> (DataNodeContainedRule) rule);
         String tableName;
         boolean primaryKey;
-        if (null != actualTableName && dataNodeRoutedRule.isPresent()) {
-            tableName = dataNodeRoutedRule.get().findLogicTableByActualTable(actualTableName).orElse("");
+        if (null != actualTableName && dataNodeContainedRule.isPresent()) {
+            tableName = dataNodeContainedRule.get().findLogicTableByActualTable(actualTableName).orElse("");
             PhysicalTableMetaData tableMetaData = metaData.getSchema().getSchemaMetaData().get(tableName);
             primaryKey = null != tableMetaData && tableMetaData.getColumns().get(columnName.toLowerCase()).isPrimaryKey();
         } else {