You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2023/06/09 10:07:56 UTC

[shardingsphere] branch master updated: Adjust TableContainedRule to distinguish between different table types and move tables init logic to rule init (#26214)

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

panjuan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 64264a510c5 Adjust TableContainedRule to distinguish between different table types and move tables init logic to rule init (#26214)
64264a510c5 is described below

commit 64264a510c557c4678f81c428d54bf0450a12a52
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Fri Jun 9 18:07:48 2023 +0800

    Adjust TableContainedRule to distinguish between different table types and move tables init logic to rule init (#26214)
---
 .../shardingsphere/encrypt/rule/EncryptRule.java   | 27 ++++++--
 .../encrypt/rule/EncryptRuleTest.java              | 13 +---
 .../handler/query/CountEncryptRuleExecutor.java    |  2 +-
 .../query/CountEncryptRuleExecutorTest.java        |  3 +-
 .../apache/shardingsphere/mask/rule/MaskRule.java  | 27 ++++++--
 .../handler/query/CountMaskRuleExecutor.java       |  2 +-
 .../handler/query/CountMaskRuleExecutorTest.java   |  3 +-
 .../shardingsphere/sharding/rule/ShardingRule.java | 60 ++++++++++++------
 .../sharding/rule/ShardingRuleTest.java            |  3 +-
 .../schema/builder/GenericSchemaBuilder.java       | 14 ++--
 .../rule/identifier/type/ColumnContainedRule.java  | 36 -----------
 .../identifier/type/DataNodeContainedRule.java     |  7 --
 .../rule/identifier/type/TableContainedRule.java   | 29 +++++++--
 .../rule/identifier/type/TableNamesMapper.java     | 74 ++++++++++++++++++++++
 .../fixture/rule/TableContainedFixtureRule.java    | 23 +++++--
 .../table/AlterTableStatementSchemaRefresher.java  | 11 ++--
 .../table/CreateTableStatementSchemaRefresher.java | 11 ++--
 .../table/RenameTableStatementSchemaRefresher.java | 11 ++--
 .../view/AlterViewStatementSchemaRefresher.java    | 11 ++--
 .../view/CreateViewStatementSchemaRefresher.java   | 11 ++--
 .../statement/ShardingSpherePreparedStatement.java | 33 ++--------
 .../core/statement/ShardingSphereStatement.java    | 33 ++--------
 .../shardingsphere/single/rule/SingleRule.java     | 44 ++++++++-----
 .../shardingsphere/single/rule/SingleRuleTest.java | 31 +++++----
 .../handler/query/CountSingleTableExecutor.java    |  2 +-
 .../query/CountSingleTableExecutorTest.java        |  3 +-
 .../mode/manager/ContextManager.java               | 11 ++--
 .../standalone/StandaloneModeContextManager.java   | 13 ++--
 .../proxy/backend/connector/DatabaseConnector.java | 33 ++--------
 29 files changed, 327 insertions(+), 254 deletions(-)

diff --git a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
index aff92e634e0..7269393de1e 100644
--- a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
+++ b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/rule/EncryptRule.java
@@ -33,8 +33,8 @@ import org.apache.shardingsphere.encrypt.exception.metadata.MissingEncryptorExce
 import org.apache.shardingsphere.encrypt.spi.EncryptAlgorithm;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
-import org.apache.shardingsphere.infra.rule.identifier.type.ColumnContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableNamesMapper;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
 
@@ -49,7 +49,7 @@ import java.util.Optional;
 /**
  * Encrypt rule.
  */
-public final class EncryptRule implements DatabaseRule, TableContainedRule, ColumnContainedRule {
+public final class EncryptRule implements DatabaseRule, TableContainedRule {
     
     @Getter
     private final RuleConfiguration configuration;
@@ -65,12 +65,15 @@ public final class EncryptRule implements DatabaseRule, TableContainedRule, Colu
     
     private final Map<String, EncryptTable> tables = new LinkedHashMap<>();
     
+    private final TableNamesMapper tableNamesMapper = new TableNamesMapper();
+    
     public EncryptRule(final EncryptRuleConfiguration ruleConfig) {
         configuration = ruleConfig;
         ruleConfig.getEncryptors().forEach((key, value) -> putAllEncryptors(key, TypedSPILoader.getService(EncryptAlgorithm.class, value.getType(), value.getProps())));
         for (EncryptTableRuleConfiguration each : ruleConfig.getTables()) {
             each.getColumns().forEach(this::checkEncryptAlgorithmType);
             tables.put(each.getName().toLowerCase(), new EncryptTable(each));
+            tableNamesMapper.put(each.getName());
         }
     }
     
@@ -86,6 +89,7 @@ public final class EncryptRule implements DatabaseRule, TableContainedRule, Colu
         for (EncryptTableRuleConfiguration each : ruleConfig.getTables()) {
             each.getColumns().forEach(this::checkEncryptAlgorithmType);
             tables.put(each.getName().toLowerCase(), new EncryptTable(each));
+            tableNamesMapper.put(each.getName());
         }
     }
     
@@ -307,8 +311,23 @@ public final class EncryptRule implements DatabaseRule, TableContainedRule, Colu
     }
     
     @Override
-    public Collection<String> getTables() {
-        return tables.keySet();
+    public TableNamesMapper getLogicTableMapper() {
+        return tableNamesMapper;
+    }
+    
+    @Override
+    public TableNamesMapper getActualTableMapper() {
+        return new TableNamesMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getDistributedTableMapper() {
+        return new TableNamesMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getEnhancedTableMapper() {
+        return getLogicTableMapper();
     }
     
     @Override
diff --git a/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java b/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
index 3714e8b1c4f..9c9a5ddc95d 100644
--- a/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
+++ b/features/encrypt/core/src/test/java/org/apache/shardingsphere/encrypt/rule/EncryptRuleTest.java
@@ -32,9 +32,9 @@ import org.junit.jupiter.params.provider.ArgumentsProvider;
 import org.junit.jupiter.params.provider.ArgumentsSource;
 
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -124,19 +124,12 @@ class EncryptRuleTest {
     
     @Test
     void assertGetTables() {
-        assertThat(new EncryptRule(createEncryptRuleConfiguration()).getTables(), is(Collections.singleton("t_encrypt")));
+        assertThat(new LinkedList<>(new EncryptRule(createEncryptRuleConfiguration()).getLogicTableMapper().getTableNames()), is(Collections.singletonList("t_encrypt")));
     }
     
     @Test
     void assertGetTableWithLowercase() {
-        assertThat(new EncryptRule(createEncryptRuleConfigurationWithUpperCaseLogicTable()).getTables(), is(Collections.singleton("t_encrypt")));
-    }
-    
-    @Test
-    void assertTheSameLogicTable() {
-        Collection<String> logicTables = new EncryptRule(createEncryptRuleConfiguration()).getTables();
-        Collection<String> theSameLogicTables = new EncryptRule(createEncryptRuleConfigurationWithUpperCaseLogicTable()).getTables();
-        assertThat(logicTables, is(theSameLogicTables));
+        assertThat(new LinkedList<>(new EncryptRule(createEncryptRuleConfigurationWithUpperCaseLogicTable()).getLogicTableMapper().getTableNames()), is(Collections.singletonList("T_ENCRYPT")));
     }
     
     private EncryptRuleConfiguration createEncryptRuleConfiguration() {
diff --git a/features/encrypt/distsql/handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/query/CountEncryptRuleExecutor.java b/features/encrypt/distsql/handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/query/CountEncryptRuleExecutor.java
index 8ee227dc05d..957cbdd95a5 100644
--- a/features/encrypt/distsql/handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/query/CountEncryptRuleExecutor.java
+++ b/features/encrypt/distsql/handler/src/main/java/org/apache/shardingsphere/encrypt/distsql/handler/query/CountEncryptRuleExecutor.java
@@ -47,7 +47,7 @@ public final class CountEncryptRuleExecutor implements RQLExecutor<CountEncryptR
     }
     
     private void fillRows(final Collection<LocalDataQueryResultRow> result, final EncryptRule rule, final String databaseName) {
-        result.add(new LocalDataQueryResultRow("encrypt", databaseName, rule.getTables().size()));
+        result.add(new LocalDataQueryResultRow("encrypt", databaseName, rule.getLogicTableMapper().getTableNames().size()));
     }
     
     @Override
diff --git a/features/encrypt/distsql/handler/src/test/java/org/apache/shardingsphere/encrypt/distsql/handler/query/CountEncryptRuleExecutorTest.java b/features/encrypt/distsql/handler/src/test/java/org/apache/shardingsphere/encrypt/distsql/handler/query/CountEncryptRuleExecutorTest.java
index 15bd10aeb92..8e63087335d 100644
--- a/features/encrypt/distsql/handler/src/test/java/org/apache/shardingsphere/encrypt/distsql/handler/query/CountEncryptRuleExecutorTest.java
+++ b/features/encrypt/distsql/handler/src/test/java/org/apache/shardingsphere/encrypt/distsql/handler/query/CountEncryptRuleExecutorTest.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableNamesMapper;
 import org.junit.jupiter.api.Test;
 
 import java.util.Collection;
@@ -67,7 +68,7 @@ class CountEncryptRuleExecutorTest {
     
     private EncryptRule mockEncryptRule() {
         EncryptRule result = mock(EncryptRule.class);
-        when(result.getTables()).thenReturn(Collections.singleton("encrypt_table"));
+        when(result.getLogicTableMapper()).thenReturn(new TableNamesMapper(Collections.singleton("encrypt_table")));
         return result;
     }
 }
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskRule.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskRule.java
index 13e2a2bd380..488a0d251d1 100644
--- a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskRule.java
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/rule/MaskRule.java
@@ -20,13 +20,12 @@ package org.apache.shardingsphere.mask.rule;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
-import org.apache.shardingsphere.infra.rule.identifier.type.ColumnContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableNamesMapper;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.mask.api.config.MaskRuleConfiguration;
 import org.apache.shardingsphere.mask.spi.MaskAlgorithm;
 
-import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Optional;
@@ -35,7 +34,7 @@ import java.util.Optional;
  * Mask rule.
  */
 @SuppressWarnings("rawtypes")
-public final class MaskRule implements DatabaseRule, TableContainedRule, ColumnContainedRule {
+public final class MaskRule implements DatabaseRule, TableContainedRule {
     
     @Getter
     private final RuleConfiguration configuration;
@@ -44,10 +43,13 @@ public final class MaskRule implements DatabaseRule, TableContainedRule, ColumnC
     
     private final Map<String, MaskTable> tables = new LinkedHashMap<>();
     
+    private final TableNamesMapper tableNamesMapper = new TableNamesMapper();
+    
     public MaskRule(final MaskRuleConfiguration ruleConfig) {
         configuration = ruleConfig;
         ruleConfig.getMaskAlgorithms().forEach((key, value) -> maskAlgorithms.put(key, TypedSPILoader.getService(MaskAlgorithm.class, value.getType(), value.getProps())));
         ruleConfig.getTables().forEach(each -> tables.put(each.getName().toLowerCase(), new MaskTable(each)));
+        ruleConfig.getTables().forEach(each -> tableNamesMapper.put(each.getName()));
     }
     
     /**
@@ -63,8 +65,23 @@ public final class MaskRule implements DatabaseRule, TableContainedRule, ColumnC
     }
     
     @Override
-    public Collection<String> getTables() {
-        return tables.keySet();
+    public TableNamesMapper getLogicTableMapper() {
+        return tableNamesMapper;
+    }
+    
+    @Override
+    public TableNamesMapper getActualTableMapper() {
+        return new TableNamesMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getDistributedTableMapper() {
+        return new TableNamesMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getEnhancedTableMapper() {
+        return getLogicTableMapper();
     }
     
     @Override
diff --git a/features/mask/distsql/handler/src/main/java/org/apache/shardingsphere/mask/distsql/handler/query/CountMaskRuleExecutor.java b/features/mask/distsql/handler/src/main/java/org/apache/shardingsphere/mask/distsql/handler/query/CountMaskRuleExecutor.java
index 9ec044745d0..53fb72454b0 100644
--- a/features/mask/distsql/handler/src/main/java/org/apache/shardingsphere/mask/distsql/handler/query/CountMaskRuleExecutor.java
+++ b/features/mask/distsql/handler/src/main/java/org/apache/shardingsphere/mask/distsql/handler/query/CountMaskRuleExecutor.java
@@ -47,7 +47,7 @@ public final class CountMaskRuleExecutor implements RQLExecutor<CountMaskRuleSta
     }
     
     private void fillRows(final Collection<LocalDataQueryResultRow> result, final MaskRule rule, final String databaseName) {
-        result.add(new LocalDataQueryResultRow("mask", databaseName, rule.getTables().size()));
+        result.add(new LocalDataQueryResultRow("mask", databaseName, rule.getLogicTableMapper().getTableNames().size()));
     }
     
     @Override
diff --git a/features/mask/distsql/handler/src/test/java/org/apache/shardingsphere/mask/distsql/handler/query/CountMaskRuleExecutorTest.java b/features/mask/distsql/handler/src/test/java/org/apache/shardingsphere/mask/distsql/handler/query/CountMaskRuleExecutorTest.java
index c09824288cc..4c79c76fd46 100644
--- a/features/mask/distsql/handler/src/test/java/org/apache/shardingsphere/mask/distsql/handler/query/CountMaskRuleExecutorTest.java
+++ b/features/mask/distsql/handler/src/test/java/org/apache/shardingsphere/mask/distsql/handler/query/CountMaskRuleExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.mask.distsql.handler.query;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableNamesMapper;
 import org.apache.shardingsphere.mask.distsql.parser.statement.CountMaskRuleStatement;
 import org.apache.shardingsphere.mask.rule.MaskRule;
 import org.junit.jupiter.api.Test;
@@ -67,7 +68,7 @@ class CountMaskRuleExecutorTest {
     
     private MaskRule mockMaskRule() {
         MaskRule result = mock(MaskRule.class);
-        when(result.getTables()).thenReturn(Collections.singleton("mask_table"));
+        when(result.getLogicTableMapper()).thenReturn(new TableNamesMapper(Collections.singletonList("mask_table")));
         return result;
     }
 }
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
index f0e00c90fa4..bc6f8c82ed2 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingRule.java
@@ -25,6 +25,7 @@ import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementConte
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
 import org.apache.shardingsphere.infra.datanode.DataNode;
+import org.apache.shardingsphere.infra.expr.core.InlineExpressionParserFactory;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.instance.InstanceContextAware;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
@@ -32,8 +33,8 @@ import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSp
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableNamesMapper;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
-import org.apache.shardingsphere.infra.expr.core.InlineExpressionParserFactory;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
 import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
@@ -117,6 +118,10 @@ public final class ShardingRule implements DatabaseRule, DataNodeContainedRule,
     
     private final ShardingCache shardingCache;
     
+    private final TableNamesMapper logicalTableMapper;
+    
+    private final TableNamesMapper actualTableMapper;
+    
     public ShardingRule(final ShardingRuleConfiguration ruleConfig, final Collection<String> dataSourceNames, final InstanceContext instanceContext) {
         configuration = ruleConfig;
         this.dataSourceNames = getDataSourceNames(ruleConfig.getTables(), ruleConfig.getAutoTables(), dataSourceNames);
@@ -143,6 +148,24 @@ public final class ShardingRule implements DatabaseRule, DataNodeContainedRule,
             ((InstanceContextAware) defaultKeyGenerateAlgorithm).setInstanceContext(instanceContext);
         }
         shardingCache = null != ruleConfig.getShardingCache() ? new ShardingCache(ruleConfig.getShardingCache(), this) : null;
+        logicalTableMapper = createLogicalTableMapper();
+        actualTableMapper = createActualTableMapper();
+    }
+    
+    private TableNamesMapper createLogicalTableMapper() {
+        TableNamesMapper result = new TableNamesMapper(broadcastTables);
+        tableRules.values().forEach(each -> result.put(each.getLogicTable()));
+        return result;
+    }
+    
+    private TableNamesMapper createActualTableMapper() {
+        TableNamesMapper result = new TableNamesMapper();
+        for (TableRule each : tableRules.values()) {
+            for (DataNode dataNode : each.getActualDataNodes()) {
+                result.put(dataNode.getTableName());
+            }
+        }
+        return result;
     }
     
     private Map<String, Collection<DataNode>> createShardingTableDataNodes(final Map<String, TableRule> tableRules) {
@@ -288,11 +311,23 @@ public final class ShardingRule implements DatabaseRule, DataNodeContainedRule,
     }
     
     @Override
-    public Collection<String> getAllTables() {
-        Collection<String> result = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
-        result.addAll(getTables());
-        result.addAll(getAllActualTables());
-        return result;
+    public TableNamesMapper getLogicTableMapper() {
+        return logicalTableMapper;
+    }
+    
+    @Override
+    public TableNamesMapper getActualTableMapper() {
+        return actualTableMapper;
+    }
+    
+    @Override
+    public TableNamesMapper getDistributedTableMapper() {
+        return getLogicTableMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getEnhancedTableMapper() {
+        return getLogicTableMapper();
     }
     
     /**
@@ -680,11 +715,7 @@ public final class ShardingRule implements DatabaseRule, DataNodeContainedRule,
         }
         return shardingTableDataNodes.getOrDefault(tableName.toLowerCase(), Collections.emptyList());
     }
-    
-    private Collection<String> getAllActualTables() {
-        return tableRules.values().stream().flatMap(each -> each.getActualDataNodes().stream().map(DataNode::getTableName)).collect(Collectors.toSet());
-    }
-    
+
     @Override
     public Optional<String> findFirstActualTable(final String logicTable) {
         return findTableRule(logicTable).map(optional -> optional.getActualDataNodes().get(0).getTableName());
@@ -700,13 +731,6 @@ public final class ShardingRule implements DatabaseRule, DataNodeContainedRule,
         return findTableRuleByActualTable(actualTable).map(TableRule::getLogicTable);
     }
     
-    @Override
-    public Collection<String> getTables() {
-        Collection<String> result = tableRules.values().stream().map(TableRule::getLogicTable).collect(Collectors.toSet());
-        result.addAll(broadcastTables);
-        return result;
-    }
-    
     @Override
     public Optional<String> findActualTableByCatalog(final String catalog, final String logicTable) {
         return findTableRule(logicTable).flatMap(optional -> findActualTableFromActualDataNode(catalog, optional.getActualDataNodes()));
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
index 88934fe6852..99ffacb2edd 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/rule/ShardingRuleTest.java
@@ -64,6 +64,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
@@ -414,7 +415,7 @@ class ShardingRuleTest {
     
     @Test
     void assertGetTables() {
-        assertThat(createMaximumShardingRule().getTables(), is(new LinkedHashSet<>(Arrays.asList("LOGIC_TABLE", "SUB_LOGIC_TABLE", "BROADCAST_TABLE"))));
+        assertThat(new LinkedList<>(createMaximumShardingRule().getLogicTableMapper().getTableNames()), is(Arrays.asList("BROADCAST_TABLE", "LOGIC_TABLE", "SUB_LOGIC_TABLE")));
     }
     
     @Test
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilder.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilder.java
index dfc1609fc1a..47b6ca2a471 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilder.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/builder/GenericSchemaBuilder.java
@@ -22,12 +22,6 @@ import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
-import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereColumn;
-import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereConstraint;
-import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereIndex;
-import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
-import org.apache.shardingsphere.infra.metadata.database.schema.reviser.MetaDataReviseEngine;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.metadata.SchemaMetaDataLoaderEngine;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.metadata.SchemaMetaDataLoaderMaterial;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.ColumnMetaData;
@@ -35,6 +29,12 @@ import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.Con
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.SchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereColumn;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereConstraint;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereIndex;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
+import org.apache.shardingsphere.infra.metadata.database.schema.reviser.MetaDataReviseEngine;
 import org.apache.shardingsphere.infra.metadata.database.schema.util.SchemaMetaDataUtils;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
@@ -93,7 +93,7 @@ public final class GenericSchemaBuilder {
     }
     
     private static Collection<String> getAllTableNames(final Collection<ShardingSphereRule> rules) {
-        return rules.stream().filter(TableContainedRule.class::isInstance).flatMap(each -> ((TableContainedRule) each).getTables().stream()).collect(Collectors.toSet());
+        return rules.stream().filter(TableContainedRule.class::isInstance).flatMap(each -> ((TableContainedRule) each).getLogicTableMapper().getTableNames().stream()).collect(Collectors.toSet());
     }
     
     private static Map<String, SchemaMetaData> loadSchemas(final Collection<String> tableNames, final GenericSchemaBuilderMaterial material) throws SQLException {
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/ColumnContainedRule.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/ColumnContainedRule.java
deleted file mode 100644
index b327f8d23c6..00000000000
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/ColumnContainedRule.java
+++ /dev/null
@@ -1,36 +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.rule.identifier.type;
-
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-
-import java.util.Collection;
-
-//TODO Consider introducing transparent table and enhance table.
-/**
- * ShardingSphere rule which contains column.
- */
-public interface ColumnContainedRule extends ShardingSphereRule {
-    
-    /**
-     * Get tables.
-     *
-     * @return tables
-     */
-    Collection<String> getTables();
-}
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataNodeContainedRule.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataNodeContainedRule.java
index 272a351484c..3526434cc1f 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataNodeContainedRule.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataNodeContainedRule.java
@@ -76,11 +76,4 @@ public interface DataNodeContainedRule extends ShardingSphereRule {
      * @return actual table name
      */
     Optional<String> findActualTableByCatalog(String catalog, String logicTable);
-    
-    /**
-     * Get all tables.
-     *
-     * @return all tables
-     */
-    Collection<String> getAllTables();
 }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/TableContainedRule.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/TableContainedRule.java
index 8dc92e62c05..2e9284ef111 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/TableContainedRule.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/TableContainedRule.java
@@ -19,17 +19,36 @@ package org.apache.shardingsphere.infra.rule.identifier.type;
 
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
-import java.util.Collection;
-
 /**
  * ShardingSphere rule which contains table.
  */
 public interface TableContainedRule extends ShardingSphereRule {
     
     /**
-     * Get tables.
+     * Get logic table mapper.
+     *
+     * @return logic table mapper
+     */
+    TableNamesMapper getLogicTableMapper();
+    
+    /**
+     * Get actual table mapper.
+     *
+     * @return actual table mapper
+     */
+    TableNamesMapper getActualTableMapper();
+    
+    /**
+     * Get distributed table mapper.
+     *
+     * @return distributed table mapper
+     */
+    TableNamesMapper getDistributedTableMapper();
+    
+    /**
+     * Get enhanced table mapper.
      *
-     * @return tables
+     * @return enhanced table mapper
      */
-    Collection<String> getTables();
+    TableNamesMapper getEnhancedTableMapper();
 }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/TableNamesMapper.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/TableNamesMapper.java
new file mode 100644
index 00000000000..44577c48035
--- /dev/null
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/TableNamesMapper.java
@@ -0,0 +1,74 @@
+/*
+ * 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.rule.identifier.type;
+
+import lombok.RequiredArgsConstructor;
+
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Table names mapper.
+ */
+@RequiredArgsConstructor
+public final class TableNamesMapper {
+    
+    private final Map<String, String> lowerCaseTableNames = new LinkedHashMap<>();
+    
+    public TableNamesMapper(final Collection<String> tableNames) {
+        tableNames.forEach(each -> lowerCaseTableNames.put(each.toLowerCase(), each));
+    }
+    
+    /**
+     * Judge whether contains table or not.
+     * 
+     * @param tableName table name
+     * @return whether contains table or not
+     */
+    public boolean contains(final String tableName) {
+        return lowerCaseTableNames.containsKey(tableName.toLowerCase());
+    }
+    
+    /**
+     * Get original table names.
+     * 
+     * @return original table names
+     */
+    public Collection<String> getTableNames() {
+        return lowerCaseTableNames.values();
+    }
+    
+    /**
+     * Put.
+     * 
+     * @param tableName table name
+     */
+    public void put(final String tableName) {
+        lowerCaseTableNames.put(tableName.toLowerCase(), tableName);
+    }
+    
+    /**
+     * Remove.
+     * 
+     * @param tableName table name
+     */
+    public void remove(final String tableName) {
+        lowerCaseTableNames.remove(tableName.toLowerCase());
+    }
+}
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/TableContainedFixtureRule.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/TableContainedFixtureRule.java
index 9b534e60804..dac17fbc6be 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/TableContainedFixtureRule.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/fixture/rule/TableContainedFixtureRule.java
@@ -19,17 +19,30 @@ package org.apache.shardingsphere.infra.metadata.database.schema.fixture.rule;
 
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
-
-import java.util.Collection;
-import java.util.Collections;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableNamesMapper;
 
 import static org.mockito.Mockito.mock;
 
 public final class TableContainedFixtureRule implements TableContainedRule {
     
     @Override
-    public Collection<String> getTables() {
-        return Collections.emptyList();
+    public TableNamesMapper getLogicTableMapper() {
+        return new TableNamesMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getActualTableMapper() {
+        return new TableNamesMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getDistributedTableMapper() {
+        return new TableNamesMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getEnhancedTableMapper() {
+        return new TableNamesMapper();
     }
     
     @Override
diff --git a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/AlterTableStatementSchemaRefresher.java b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/AlterTableStatementSchemaRefresher.java
index 02188516f28..cc2f23e15fc 100644
--- a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/AlterTableStatementSchemaRefresher.java
+++ b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/AlterTableStatementSchemaRefresher.java
@@ -27,16 +27,16 @@ import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericS
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.pojo.AlterSchemaMetaDataPOJO;
-import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
 
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Optional;
-import java.util.LinkedList;
 
 /**
  * Schema refresher for alter table statement.
@@ -61,7 +61,7 @@ public final class AlterTableStatementSchemaRefresher implements MetaDataRefresh
     private ShardingSphereTable getTable(final ShardingSphereDatabase database, final Collection<String> logicDataSourceNames, final String schemaName,
                                          final String tableName, final ConfigurationProperties props) throws SQLException {
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(new LinkedList<>(database.getRuleMetaData().getRules()));
-        if (!containsInImmutableDataNodeContainedRule(tableName, database)) {
+        if (isSingleTable(tableName, database)) {
             ruleMetaData.findRules(MutableDataNodeRule.class).forEach(each -> each.put(logicDataSourceNames.iterator().next(), schemaName, tableName));
         }
         GenericSchemaBuilderMaterial material = new GenericSchemaBuilderMaterial(database.getProtocolType(),
@@ -71,9 +71,8 @@ public final class AlterTableStatementSchemaRefresher implements MetaDataRefresh
                 .orElseGet(() -> new ShardingSphereTable(tableName, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
     }
     
-    private boolean containsInImmutableDataNodeContainedRule(final String tableName, final ShardingSphereDatabase database) {
-        return database.getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .filter(each -> !(each instanceof MutableDataNodeRule)).anyMatch(each -> each.getAllTables().contains(tableName));
+    private boolean isSingleTable(final String tableName, final ShardingSphereDatabase database) {
+        return database.getRuleMetaData().findRules(TableContainedRule.class).stream().noneMatch(each -> each.getDistributedTableMapper().contains(tableName));
     }
     
     @Override
diff --git a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/CreateTableStatementSchemaRefresher.java b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/CreateTableStatementSchemaRefresher.java
index ae4d9597b37..e7691a314d0 100644
--- a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/CreateTableStatementSchemaRefresher.java
+++ b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/CreateTableStatementSchemaRefresher.java
@@ -27,16 +27,16 @@ import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericS
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.pojo.AlterSchemaMetaDataPOJO;
-import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
 
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Optional;
-import java.util.LinkedList;
 
 /**
  * Schema refresher for create table statement.
@@ -48,7 +48,7 @@ public final class CreateTableStatementSchemaRefresher implements MetaDataRefres
                         final String schemaName, final CreateTableStatement sqlStatement, final ConfigurationProperties props) throws SQLException {
         String tableName = sqlStatement.getTable().getTableName().getIdentifier().getValue();
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(new LinkedList<>(database.getRuleMetaData().getRules()));
-        if (!containsInImmutableDataNodeContainedRule(tableName, database)) {
+        if (isSingleTable(tableName, database)) {
             ruleMetaData.findRules(MutableDataNodeRule.class).forEach(each -> each.put(logicDataSourceNames.iterator().next(), schemaName, tableName));
         }
         GenericSchemaBuilderMaterial material = new GenericSchemaBuilderMaterial(database.getProtocolType(),
@@ -62,9 +62,8 @@ public final class CreateTableStatementSchemaRefresher implements MetaDataRefres
         }
     }
     
-    private boolean containsInImmutableDataNodeContainedRule(final String tableName, final ShardingSphereDatabase database) {
-        return database.getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .filter(each -> !(each instanceof MutableDataNodeRule)).anyMatch(each -> each.getAllTables().contains(tableName));
+    private boolean isSingleTable(final String tableName, final ShardingSphereDatabase database) {
+        return database.getRuleMetaData().findRules(TableContainedRule.class).stream().noneMatch(each -> each.getDistributedTableMapper().contains(tableName));
     }
     
     @Override
diff --git a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/RenameTableStatementSchemaRefresher.java b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/RenameTableStatementSchemaRefresher.java
index 1742480bc04..70cdefd575c 100644
--- a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/RenameTableStatementSchemaRefresher.java
+++ b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/table/RenameTableStatementSchemaRefresher.java
@@ -27,17 +27,17 @@ import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericS
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.pojo.AlterSchemaMetaDataPOJO;
-import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.table.RenameTableDefinitionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.RenameTableStatement;
 
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Optional;
-import java.util.LinkedList;
 
 /**
  * Schema refresher for rename table statement.
@@ -58,7 +58,7 @@ public final class RenameTableStatementSchemaRefresher implements MetaDataRefres
     private ShardingSphereTable getTable(final ShardingSphereDatabase database, final Collection<String> logicDataSourceNames, final String schemaName, final String tableName,
                                          final ConfigurationProperties props) throws SQLException {
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(new LinkedList<>(database.getRuleMetaData().getRules()));
-        if (!containsInImmutableDataNodeContainedRule(tableName, database)) {
+        if (isSingleTable(tableName, database)) {
             ruleMetaData.findRules(MutableDataNodeRule.class).forEach(each -> each.put(logicDataSourceNames.iterator().next(), schemaName, tableName));
         }
         GenericSchemaBuilderMaterial material = new GenericSchemaBuilderMaterial(database.getProtocolType(),
@@ -68,9 +68,8 @@ public final class RenameTableStatementSchemaRefresher implements MetaDataRefres
                 .orElseGet(() -> new ShardingSphereTable(tableName, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
     }
     
-    private boolean containsInImmutableDataNodeContainedRule(final String tableName, final ShardingSphereDatabase database) {
-        return database.getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .filter(each -> !(each instanceof MutableDataNodeRule)).anyMatch(each -> each.getAllTables().contains(tableName));
+    private boolean isSingleTable(final String tableName, final ShardingSphereDatabase database) {
+        return database.getRuleMetaData().findRules(TableContainedRule.class).stream().noneMatch(each -> each.getDistributedTableMapper().contains(tableName));
     }
     
     @Override
diff --git a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/view/AlterViewStatementSchemaRefresher.java b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/view/AlterViewStatementSchemaRefresher.java
index 9cfad241e65..16030072ddf 100644
--- a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/view/AlterViewStatementSchemaRefresher.java
+++ b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/view/AlterViewStatementSchemaRefresher.java
@@ -28,8 +28,8 @@ import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSp
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereView;
 import org.apache.shardingsphere.infra.metadata.database.schema.pojo.AlterSchemaMetaDataPOJO;
-import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.AlterViewStatementHandler;
@@ -37,9 +37,9 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.AlterViewSta
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Optional;
-import java.util.LinkedList;
 
 /**
  * Schema refresher for alter view statement.
@@ -73,7 +73,7 @@ public final class AlterViewStatementSchemaRefresher implements MetaDataRefreshe
     private ShardingSphereSchema getSchema(final ShardingSphereDatabase database, final Collection<String> logicDataSourceNames,
                                            final String schemaName, final String viewName, final String viewDefinition, final ConfigurationProperties props) throws SQLException {
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(new LinkedList<>(database.getRuleMetaData().getRules()));
-        if (!containsInImmutableDataNodeContainedRule(viewName, database)) {
+        if (isSingleTable(viewName, database)) {
             ruleMetaData.findRules(MutableDataNodeRule.class).forEach(each -> each.put(logicDataSourceNames.iterator().next(), schemaName, viewName));
         }
         GenericSchemaBuilderMaterial material = new GenericSchemaBuilderMaterial(database.getProtocolType(),
@@ -86,9 +86,8 @@ public final class AlterViewStatementSchemaRefresher implements MetaDataRefreshe
         return result;
     }
     
-    private boolean containsInImmutableDataNodeContainedRule(final String tableName, final ShardingSphereDatabase database) {
-        return database.getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .filter(each -> !(each instanceof MutableDataNodeRule)).anyMatch(each -> each.getAllTables().contains(tableName));
+    private boolean isSingleTable(final String tableName, final ShardingSphereDatabase database) {
+        return database.getRuleMetaData().findRules(TableContainedRule.class).stream().noneMatch(each -> each.getDistributedTableMapper().contains(tableName));
     }
     
     @Override
diff --git a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/view/CreateViewStatementSchemaRefresher.java b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/view/CreateViewStatementSchemaRefresher.java
index 5d806b97697..665b7d0870b 100644
--- a/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/view/CreateViewStatementSchemaRefresher.java
+++ b/infra/context/src/main/java/org/apache/shardingsphere/infra/connection/refresher/type/view/CreateViewStatementSchemaRefresher.java
@@ -28,16 +28,16 @@ import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSp
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereView;
 import org.apache.shardingsphere.infra.metadata.database.schema.pojo.AlterSchemaMetaDataPOJO;
-import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateViewStatement;
 
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.Optional;
-import java.util.LinkedList;
 
 /**
  * Schema refresher for create view statement.
@@ -49,7 +49,7 @@ public final class CreateViewStatementSchemaRefresher implements MetaDataRefresh
                         final String schemaName, final CreateViewStatement sqlStatement, final ConfigurationProperties props) throws SQLException {
         String viewName = sqlStatement.getView().getTableName().getIdentifier().getValue();
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(new LinkedList<>(database.getRuleMetaData().getRules()));
-        if (!containsInImmutableDataNodeContainedRule(viewName, database)) {
+        if (isSingleTable(viewName, database)) {
             ruleMetaData.findRules(MutableDataNodeRule.class).forEach(each -> each.put(logicDataSourceNames.iterator().next(), schemaName, viewName));
         }
         GenericSchemaBuilderMaterial material = new GenericSchemaBuilderMaterial(database.getProtocolType(),
@@ -64,9 +64,8 @@ public final class CreateViewStatementSchemaRefresher implements MetaDataRefresh
         }
     }
     
-    private boolean containsInImmutableDataNodeContainedRule(final String tableName, final ShardingSphereDatabase database) {
-        return database.getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .filter(each -> !(each instanceof MutableDataNodeRule)).anyMatch(each -> each.getAllTables().contains(tableName));
+    private boolean isSingleTable(final String tableName, final ShardingSphereDatabase database) {
+        return database.getRuleMetaData().findRules(TableContainedRule.class).stream().noneMatch(each -> each.getDistributedTableMapper().contains(tableName));
     }
     
     @Override
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 2d9b42e2219..9cd4e09446d 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -34,7 +34,6 @@ import org.apache.shardingsphere.driver.jdbc.core.statement.metadata.ShardingSph
 import org.apache.shardingsphere.driver.jdbc.exception.syntax.EmptySQLException;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
 import org.apache.shardingsphere.infra.binder.aware.ParameterAware;
-import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
@@ -77,16 +76,16 @@ import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRule
 import org.apache.shardingsphere.infra.parser.SQLParserEngine;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.identifier.type.ColumnContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
-import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.RawExecutionRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.StorageConnectorReusableRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
 import org.apache.shardingsphere.traffic.engine.TrafficEngine;
 import org.apache.shardingsphere.traffic.exception.metadata.EmptyTrafficExecutionUnitException;
@@ -215,35 +214,17 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
     }
     
     private boolean isTransparentStatement(final ShardingSphereRuleMetaData ruleMetaData) {
-        Optional<DataNodeContainedRule> dataNodeContainedRule = getDataNodeContainedRuleForShardingRule(ruleMetaData.findRules(DataNodeContainedRule.class));
-        Collection<ColumnContainedRule> columnContainedRules = ruleMetaData.findRules(ColumnContainedRule.class);
+        Collection<TableContainedRule> tableContainedRules = ruleMetaData.findRules(TableContainedRule.class);
         for (String each : sqlStatementContext.getTablesContext().getTableNames()) {
-            if ((!dataNodeContainedRule.isPresent() || !dataNodeContainedRule.get().getAllTables().contains(each)) && !containsInColumnContainedRule(each, columnContainedRules)) {
-                continue;
+            for (TableContainedRule tableContainedRule : tableContainedRules) {
+                if (tableContainedRule.getEnhancedTableMapper().contains(each.toLowerCase())) {
+                    return false;
+                }
             }
-            return false;
         }
         return true;
     }
     
-    private Optional<DataNodeContainedRule> getDataNodeContainedRuleForShardingRule(final Collection<DataNodeContainedRule> dataNodeContainedRules) {
-        for (DataNodeContainedRule each : dataNodeContainedRules) {
-            if (!(each instanceof MutableDataNodeRule)) {
-                return Optional.of(each);
-            }
-        }
-        return Optional.empty();
-    }
-    
-    private boolean containsInColumnContainedRule(final String tableName, final Collection<ColumnContainedRule> columnContainedRules) {
-        for (ColumnContainedRule each : columnContainedRules) {
-            if (each.getTables().contains(tableName)) {
-                return true;
-            }
-        }
-        return false;
-    }
-    
     @Override
     public ResultSet executeQuery() throws SQLException {
         ResultSet result;
diff --git a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index bd6ff57d570..0e891fcfde2 100644
--- a/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ b/jdbc/core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -33,7 +33,6 @@ import org.apache.shardingsphere.driver.jdbc.core.resultset.ShardingSphereResult
 import org.apache.shardingsphere.driver.jdbc.exception.syntax.EmptySQLException;
 import org.apache.shardingsphere.driver.jdbc.exception.transaction.JDBCTransactionAcrossDatabasesException;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
-import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
@@ -73,15 +72,15 @@ import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import org.apache.shardingsphere.infra.rule.identifier.type.ColumnContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
-import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.RawExecutionRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
 import org.apache.shardingsphere.traffic.engine.TrafficEngine;
 import org.apache.shardingsphere.traffic.exception.metadata.EmptyTrafficExecutionUnitException;
@@ -243,35 +242,17 @@ public final class ShardingSphereStatement extends AbstractStatementAdapter {
     }
     
     private boolean isTransparentStatement(final SQLStatementContext sqlStatementContext, final ShardingSphereRuleMetaData ruleMetaData) {
-        Optional<DataNodeContainedRule> dataNodeContainedRule = getDataNodeContainedRuleForShardingRule(ruleMetaData.findRules(DataNodeContainedRule.class));
-        Collection<ColumnContainedRule> columnContainedRules = ruleMetaData.findRules(ColumnContainedRule.class);
+        Collection<TableContainedRule> tableContainedRules = ruleMetaData.findRules(TableContainedRule.class);
         for (String each : sqlStatementContext.getTablesContext().getTableNames()) {
-            if ((!dataNodeContainedRule.isPresent() || !dataNodeContainedRule.get().getAllTables().contains(each)) && !containsInColumnContainedRule(each, columnContainedRules)) {
-                continue;
+            for (TableContainedRule tableContainedRule : tableContainedRules) {
+                if (tableContainedRule.getEnhancedTableMapper().contains(each.toLowerCase())) {
+                    return false;
+                }
             }
-            return false;
         }
         return true;
     }
     
-    private Optional<DataNodeContainedRule> getDataNodeContainedRuleForShardingRule(final Collection<DataNodeContainedRule> dataNodeContainedRules) {
-        for (DataNodeContainedRule each : dataNodeContainedRules) {
-            if (!(each instanceof MutableDataNodeRule)) {
-                return Optional.of(each);
-            }
-        }
-        return Optional.empty();
-    }
-    
-    private boolean containsInColumnContainedRule(final String tableName, final Collection<ColumnContainedRule> columnContainedRules) {
-        for (ColumnContainedRule each : columnContainedRules) {
-            if (each.getTables().contains(tableName)) {
-                return true;
-            }
-        }
-        return false;
-    }
-    
     @Override
     public int executeUpdate(final String sql) throws SQLException {
         try {
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleRule.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleRule.java
index c124ed025e1..14fb92b46bb 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleRule.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleRule.java
@@ -31,6 +31,7 @@ import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRul
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableNamesMapper;
 import org.apache.shardingsphere.infra.rule.identifier.type.exportable.ExportableRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.exportable.constant.ExportableConstants;
 import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
@@ -40,7 +41,6 @@ import javax.sql.DataSource;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
@@ -49,7 +49,6 @@ import java.util.Map.Entry;
 import java.util.Optional;
 import java.util.TreeSet;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.stream.Collectors;
 
 /**
  * Single rule.
@@ -67,7 +66,7 @@ public final class SingleRule implements DatabaseRule, DataNodeContainedRule, Ta
     @Getter
     private final Map<String, Collection<DataNode>> singleTableDataNodes;
     
-    private final Map<String, String> tableNames;
+    private final TableNamesMapper tableNamesMapper = new TableNamesMapper();
     
     public SingleRule(final SingleRuleConfiguration ruleConfig, final String databaseName, final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereRule> builtRules) {
         configuration = ruleConfig;
@@ -76,7 +75,7 @@ public final class SingleRule implements DatabaseRule, DataNodeContainedRule, Ta
         Map<String, DataSource> aggregateDataSourceMap = getAggregateDataSourceMap(enabledDataSources, builtRules);
         dataSourceNames = aggregateDataSourceMap.keySet();
         singleTableDataNodes = SingleTableDataNodeLoader.load(databaseName, DatabaseTypeEngine.getStorageType(enabledDataSources.values()), aggregateDataSourceMap, getLoadedTables(builtRules));
-        tableNames = singleTableDataNodes.entrySet().stream().collect(Collectors.toConcurrentMap(Entry::getKey, entry -> entry.getValue().iterator().next().getTableName()));
+        singleTableDataNodes.forEach((key, value) -> tableNamesMapper.put(value.iterator().next().getTableName()));
     }
     
     private Map<String, DataSource> getAggregateDataSourceMap(final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereRule> builtRules) {
@@ -103,8 +102,15 @@ public final class SingleRule implements DatabaseRule, DataNodeContainedRule, Ta
     }
     
     private Collection<String> getLoadedTables(final Collection<ShardingSphereRule> builtRules) {
-        return builtRules.stream().filter(DataNodeContainedRule.class::isInstance)
-                .flatMap(each -> ((DataNodeContainedRule) each).getAllTables().stream()).collect(Collectors.toCollection(() -> new TreeSet<>(String.CASE_INSENSITIVE_ORDER)));
+        Collection<String> result = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+        for (ShardingSphereRule each : builtRules) {
+            if (!(each instanceof TableContainedRule)) {
+                continue;
+            }
+            result.addAll(((TableContainedRule) each).getDistributedTableMapper().getTableNames());
+            result.addAll(((TableContainedRule) each).getActualTableMapper().getTableNames());
+        }
+        return result;
     }
     
     /**
@@ -196,7 +202,7 @@ public final class SingleRule implements DatabaseRule, DataNodeContainedRule, Ta
             DataNode dataNode = new DataNode(dataSourceName, tableName);
             dataNode.setSchemaName(schemaName);
             dataNodes.add(dataNode);
-            tableNames.put(tableName.toLowerCase(), tableName);
+            tableNamesMapper.put(tableName);
         }
     }
     
@@ -214,7 +220,7 @@ public final class SingleRule implements DatabaseRule, DataNodeContainedRule, Ta
         dataNodes.removeIf(each -> schemaNames.contains(each.getSchemaName().toLowerCase()));
         if (dataNodes.isEmpty()) {
             singleTableDataNodes.remove(tableName.toLowerCase());
-            tableNames.remove(tableName.toLowerCase());
+            tableNamesMapper.remove(tableName);
         }
     }
     
@@ -266,20 +272,28 @@ public final class SingleRule implements DatabaseRule, DataNodeContainedRule, Ta
     }
     
     @Override
-    public Collection<String> getAllTables() {
-        Collection<String> result = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
-        result.addAll(tableNames.values());
-        return result;
+    public TableNamesMapper getLogicTableMapper() {
+        return tableNamesMapper;
+    }
+    
+    @Override
+    public TableNamesMapper getActualTableMapper() {
+        return new TableNamesMapper();
+    }
+    
+    @Override
+    public TableNamesMapper getDistributedTableMapper() {
+        return new TableNamesMapper();
     }
     
     @Override
-    public Collection<String> getTables() {
-        return new HashSet<>(tableNames.values());
+    public TableNamesMapper getEnhancedTableMapper() {
+        return new TableNamesMapper();
     }
     
     @Override
     public Map<String, Object> getExportData() {
-        return Collections.singletonMap(ExportableConstants.EXPORT_SINGLE_TABLES, tableNames.keySet());
+        return Collections.singletonMap(ExportableConstants.EXPORT_SINGLE_TABLES, tableNamesMapper.getTableNames());
     }
     
     @Override
diff --git a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/rule/SingleRuleTest.java b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/rule/SingleRuleTest.java
index 27de1fbc8c8..19c93e10e71 100644
--- a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/rule/SingleRuleTest.java
+++ b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/rule/SingleRuleTest.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.metadata.database.schema.QualifiedTable;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
 import org.apache.shardingsphere.test.fixture.jdbc.MockedDataSource;
 import org.junit.jupiter.api.BeforeEach;
@@ -94,9 +95,10 @@ class SingleRuleTest {
     
     @Test
     void assertGetSingleTableDataNodes() {
-        DataNodeContainedRule dataNodeContainedRule = mock(DataNodeContainedRule.class);
-        when(dataNodeContainedRule.getAllTables()).thenReturn(Arrays.asList("t_order", "t_order_0", "t_order_1"));
-        SingleRule singleRule = new SingleRule(new SingleRuleConfiguration(), DefaultDatabase.LOGIC_NAME, dataSourceMap, Collections.singleton(dataNodeContainedRule));
+        TableContainedRule tableContainedRule = mock(TableContainedRule.class, RETURNS_DEEP_STUBS);
+        when(tableContainedRule.getDistributedTableMapper().getTableNames()).thenReturn(Collections.singletonList("t_order"));
+        when(tableContainedRule.getActualTableMapper().getTableNames()).thenReturn(Arrays.asList("t_order_0", "t_order_1"));
+        SingleRule singleRule = new SingleRule(new SingleRuleConfiguration(), DefaultDatabase.LOGIC_NAME, dataSourceMap, Collections.singleton(tableContainedRule));
         Map<String, Collection<DataNode>> actual = singleRule.getSingleTableDataNodes();
         assertThat(actual.size(), is(2));
         assertTrue(actual.containsKey("employee"));
@@ -105,9 +107,10 @@ class SingleRuleTest {
     
     @Test
     void assertGetSingleTableDataNodesWithUpperCase() {
-        DataNodeContainedRule dataNodeContainedRule = mock(DataNodeContainedRule.class);
-        when(dataNodeContainedRule.getAllTables()).thenReturn(Arrays.asList("T_ORDER", "T_ORDER_0", "T_ORDER_1"));
-        SingleRule singleRule = new SingleRule(new SingleRuleConfiguration(), DefaultDatabase.LOGIC_NAME, dataSourceMap, Collections.singleton(dataNodeContainedRule));
+        TableContainedRule tableContainedRule = mock(TableContainedRule.class, RETURNS_DEEP_STUBS);
+        when(tableContainedRule.getDistributedTableMapper().getTableNames()).thenReturn(Collections.singletonList("T_ORDER"));
+        when(tableContainedRule.getActualTableMapper().getTableNames()).thenReturn(Arrays.asList("T_ORDER_0", "T_ORDER_1"));
+        SingleRule singleRule = new SingleRule(new SingleRuleConfiguration(), DefaultDatabase.LOGIC_NAME, dataSourceMap, Collections.singleton(tableContainedRule));
         Map<String, Collection<DataNode>> actual = singleRule.getSingleTableDataNodes();
         assertThat(actual.size(), is(2));
         assertTrue(actual.containsKey("employee"));
@@ -187,11 +190,11 @@ class SingleRuleTest {
         tableNames.add(new QualifiedTable(DefaultDatabase.LOGIC_NAME, "teacher"));
         assertThat(singleRule.getSingleTableNames(tableNames).iterator().next().getSchemaName(), is(DefaultDatabase.LOGIC_NAME));
         assertThat(singleRule.getSingleTableNames(tableNames).iterator().next().getTableName(), is("teacher"));
-        assertTrue(singleRule.getAllTables().contains("employee"));
-        assertTrue(singleRule.getAllTables().contains("student"));
-        assertTrue(singleRule.getAllTables().contains("t_order_0"));
-        assertTrue(singleRule.getAllTables().contains("t_order_1"));
-        assertTrue(singleRule.getAllTables().contains("teacher"));
+        assertTrue(singleRule.getLogicTableMapper().contains("employee"));
+        assertTrue(singleRule.getLogicTableMapper().contains("student"));
+        assertTrue(singleRule.getLogicTableMapper().contains("t_order_0"));
+        assertTrue(singleRule.getLogicTableMapper().contains("t_order_1"));
+        assertTrue(singleRule.getLogicTableMapper().contains("teacher"));
     }
     
     @Test
@@ -203,9 +206,9 @@ class SingleRuleTest {
         Collection<QualifiedTable> tableNames = new LinkedList<>();
         tableNames.add(new QualifiedTable(DefaultDatabase.LOGIC_NAME, "employee"));
         assertTrue(singleRule.getSingleTableNames(tableNames).isEmpty());
-        assertTrue(singleRule.getAllTables().contains("student"));
-        assertTrue(singleRule.getAllTables().contains("t_order_0"));
-        assertTrue(singleRule.getAllTables().contains("t_order_1"));
+        assertTrue(singleRule.getLogicTableMapper().contains("student"));
+        assertTrue(singleRule.getLogicTableMapper().contains("t_order_0"));
+        assertTrue(singleRule.getLogicTableMapper().contains("t_order_1"));
     }
     
     @Test
diff --git a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/CountSingleTableExecutor.java b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/CountSingleTableExecutor.java
index f0db8687c17..d62f82e2d07 100644
--- a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/CountSingleTableExecutor.java
+++ b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/CountSingleTableExecutor.java
@@ -40,7 +40,7 @@ public final class CountSingleTableExecutor implements RQLExecutor<CountSingleTa
     @Override
     public Collection<LocalDataQueryResultRow> getRows(final ShardingSphereDatabase database, final CountSingleTableStatement sqlStatement) {
         SingleRule rule = database.getRuleMetaData().getSingleRule(SingleRule.class);
-        return Collections.singleton(new LocalDataQueryResultRow(database.getName(), rule.getAllTables().size()));
+        return Collections.singleton(new LocalDataQueryResultRow(database.getName(), rule.getLogicTableMapper().getTableNames().size()));
     }
     
     @Override
diff --git a/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/CountSingleTableExecutorTest.java b/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/CountSingleTableExecutorTest.java
index a0903781299..e17610b21e5 100644
--- a/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/CountSingleTableExecutorTest.java
+++ b/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/CountSingleTableExecutorTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.single.distsql.handler.query;
 import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableNamesMapper;
 import org.apache.shardingsphere.single.distsql.statement.rql.CountSingleTableStatement;
 import org.apache.shardingsphere.single.rule.SingleRule;
 import org.junit.jupiter.api.Test;
@@ -66,7 +67,7 @@ class CountSingleTableExecutorTest {
     
     private SingleRule mockSingleRule() {
         SingleRule result = mock(SingleRule.class);
-        when(result.getAllTables()).thenReturn(Arrays.asList("single_table_1", "single_table_2"));
+        when(result.getLogicTableMapper()).thenReturn(new TableNamesMapper(Arrays.asList("single_table_1", "single_table_2")));
         return result;
     }
 }
diff --git a/mode/core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java b/mode/core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
index a940cf3b15a..fff537ab4b7 100644
--- a/mode/core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
+++ b/mode/core/src/main/java/org/apache/shardingsphere/mode/manager/ContextManager.java
@@ -44,9 +44,9 @@ import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSp
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereView;
 import org.apache.shardingsphere.infra.rule.builder.global.GlobalRulesBuilder;
-import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.ResourceHeldRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.infra.state.cluster.ClusterState;
 import org.apache.shardingsphere.infra.state.cluster.ClusterStateContext;
 import org.apache.shardingsphere.infra.yaml.data.pojo.YamlShardingSphereRowData;
@@ -216,7 +216,7 @@ public final class ContextManager implements AutoCloseable {
     
     private synchronized void alterTable(final String databaseName, final String schemaName, final ShardingSphereTable beBoChangedTable) {
         ShardingSphereDatabase database = metaDataContexts.get().getMetaData().getDatabase(databaseName);
-        if (!containsMutableDataNodeRule(database, beBoChangedTable.getName())) {
+        if (isSingleTable(database, beBoChangedTable.getName())) {
             database.reloadRules(MutableDataNodeRule.class);
         }
         database.getSchema(schemaName).putTable(beBoChangedTable.getName(), beBoChangedTable);
@@ -224,15 +224,14 @@ public final class ContextManager implements AutoCloseable {
     
     private synchronized void alterView(final String databaseName, final String schemaName, final ShardingSphereView beBoChangedView) {
         ShardingSphereDatabase database = metaDataContexts.get().getMetaData().getDatabase(databaseName);
-        if (!containsMutableDataNodeRule(database, beBoChangedView.getName())) {
+        if (isSingleTable(database, beBoChangedView.getName())) {
             database.reloadRules(MutableDataNodeRule.class);
         }
         database.getSchema(schemaName).putView(beBoChangedView.getName(), beBoChangedView);
     }
     
-    private boolean containsMutableDataNodeRule(final ShardingSphereDatabase database, final String tableName) {
-        return database.getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .filter(each -> !(each instanceof MutableDataNodeRule)).anyMatch(each -> each.getAllTables().contains(tableName));
+    private boolean isSingleTable(final ShardingSphereDatabase database, final String tableName) {
+        return database.getRuleMetaData().findRules(TableContainedRule.class).stream().noneMatch(each -> each.getDistributedTableMapper().contains(tableName));
     }
     
     /**
diff --git a/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneModeContextManager.java b/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneModeContextManager.java
index 19dd98d88dc..489b94d59dc 100644
--- a/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneModeContextManager.java
+++ b/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneModeContextManager.java
@@ -27,9 +27,9 @@ import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSp
 import org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereView;
 import org.apache.shardingsphere.infra.metadata.database.schema.pojo.AlterSchemaMetaDataPOJO;
 import org.apache.shardingsphere.infra.metadata.database.schema.pojo.AlterSchemaPOJO;
-import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.ResourceHeldRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.infra.util.spi.type.ordered.cache.OrderedServicesCache;
 import org.apache.shardingsphere.metadata.persist.service.database.DatabaseMetaDataBasedPersistService;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -96,7 +96,7 @@ public final class StandaloneModeContextManager implements ModeContextManager, C
     
     private void addDataNode(final ShardingSphereDatabase database, final String logicDataSourceName, final String schemaName, final Collection<String> tobeAddedTableNames) {
         tobeAddedTableNames.forEach(each -> {
-            if (!Strings.isNullOrEmpty(logicDataSourceName) && !containsInImmutableDataNodeContainedRule(each, database)) {
+            if (!Strings.isNullOrEmpty(logicDataSourceName) && isSingleTable(each, database)) {
                 database.getRuleMetaData().findRules(MutableDataNodeRule.class).forEach(rule -> rule.put(logicDataSourceName, schemaName, each));
             }
         });
@@ -110,7 +110,7 @@ public final class StandaloneModeContextManager implements ModeContextManager, C
     
     private void addTablesToDataNode(final ShardingSphereDatabase database, final String schemaName, final String logicDataSourceName, final Map<String, ShardingSphereTable> toBeAddedTables) {
         for (Entry<String, ShardingSphereTable> entry : toBeAddedTables.entrySet()) {
-            if (!Strings.isNullOrEmpty(logicDataSourceName) && !containsInImmutableDataNodeContainedRule(entry.getKey(), database)) {
+            if (!Strings.isNullOrEmpty(logicDataSourceName) && isSingleTable(entry.getKey(), database)) {
                 database.getRuleMetaData().findRules(MutableDataNodeRule.class).forEach(rule -> rule.put(logicDataSourceName, schemaName, entry.getKey()));
             }
             database.getSchema(schemaName).putTable(entry.getKey(), entry.getValue());
@@ -120,7 +120,7 @@ public final class StandaloneModeContextManager implements ModeContextManager, C
     private void addViewsToDataNode(final ShardingSphereDatabase database, final String schemaName, final String logicDataSourceName,
                                     final Map<String, ShardingSphereTable> toBeAddedTables, final Map<String, ShardingSphereView> toBeAddedViews) {
         for (Entry<String, ShardingSphereView> entry : toBeAddedViews.entrySet()) {
-            if (!Strings.isNullOrEmpty(logicDataSourceName) && !containsInImmutableDataNodeContainedRule(entry.getKey(), database)) {
+            if (!Strings.isNullOrEmpty(logicDataSourceName) && isSingleTable(entry.getKey(), database)) {
                 database.getRuleMetaData().findRules(MutableDataNodeRule.class).forEach(rule -> rule.put(logicDataSourceName, schemaName, entry.getKey()));
             }
             database.getSchema(schemaName).putTable(entry.getKey(), toBeAddedTables.get(entry.getKey().toLowerCase()));
@@ -128,9 +128,8 @@ public final class StandaloneModeContextManager implements ModeContextManager, C
         }
     }
     
-    private boolean containsInImmutableDataNodeContainedRule(final String tableName, final ShardingSphereDatabase database) {
-        return database.getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .filter(each -> !(each instanceof MutableDataNodeRule)).anyMatch(each -> each.getAllTables().contains(tableName));
+    private boolean isSingleTable(final String tableName, final ShardingSphereDatabase database) {
+        return database.getRuleMetaData().findRules(TableContainedRule.class).stream().noneMatch(each -> each.getDistributedTableMapper().contains(tableName));
     }
     
     private void removeSchemaMetaData(final ShardingSphereDatabase database, final String schemaName) {
diff --git a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
index 17c8acbdefc..05b8c1ff6c6 100644
--- a/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
+++ b/proxy/backend/core/src/main/java/org/apache/shardingsphere/proxy/backend/connector/DatabaseConnector.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.proxy.backend.connector;
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.dialect.SQLExceptionTransformEngine;
 import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
-import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKeyContext;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CloseStatementContext;
@@ -47,9 +46,8 @@ import org.apache.shardingsphere.infra.merge.MergeEngine;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.schema.util.SystemSchemaUtils;
-import org.apache.shardingsphere.infra.rule.identifier.type.ColumnContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
-import org.apache.shardingsphere.infra.rule.identifier.type.MutableDataNodeRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.TableContainedRule;
 import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -77,6 +75,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLInsertStatement;
+import org.apache.shardingsphere.sqlfederation.decider.SQLFederationDecideEngine;
 import org.apache.shardingsphere.sqlfederation.rule.SQLFederationRule;
 import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutor;
 import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
@@ -146,35 +145,17 @@ public final class DatabaseConnector implements DatabaseBackendHandler {
     }
     
     private boolean isTransparentStatement(final SQLStatementContext sqlStatementContext) {
-        Optional<DataNodeContainedRule> dataNodeContainedRule = getDataNodeContainedRuleForShardingRule(database.getRuleMetaData().findRules(DataNodeContainedRule.class));
-        Collection<ColumnContainedRule> columnContainedRules = database.getRuleMetaData().findRules(ColumnContainedRule.class);
+        Collection<TableContainedRule> tableContainedRules = database.getRuleMetaData().findRules(TableContainedRule.class);
         for (String each : sqlStatementContext.getTablesContext().getTableNames()) {
-            if ((!dataNodeContainedRule.isPresent() || !dataNodeContainedRule.get().getAllTables().contains(each)) && !containsInColumnContainedRule(each, columnContainedRules)) {
-                continue;
+            for (TableContainedRule tableContainedRule : tableContainedRules) {
+                if (tableContainedRule.getEnhancedTableMapper().contains(each.toLowerCase())) {
+                    return false;
+                }
             }
-            return false;
         }
         return true;
     }
     
-    private Optional<DataNodeContainedRule> getDataNodeContainedRuleForShardingRule(final Collection<DataNodeContainedRule> dataNodeContainedRules) {
-        for (DataNodeContainedRule each : dataNodeContainedRules) {
-            if (!(each instanceof MutableDataNodeRule)) {
-                return Optional.of(each);
-            }
-        }
-        return Optional.empty();
-    }
-    
-    private boolean containsInColumnContainedRule(final String tableName, final Collection<ColumnContainedRule> columnContainedRules) {
-        for (ColumnContainedRule each : columnContainedRules) {
-            if (each.getTables().contains(tableName)) {
-                return true;
-            }
-        }
-        return false;
-    }
-    
     /**
      * Add statement.
      *