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

jianglongtao 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 65e8eab5df9 Refactor DataNodeRule (#30307)
65e8eab5df9 is described below

commit 65e8eab5df97b2fa31159d676135d4fbeb7e9227
Author: Liang Zhang <[email protected]>
AuthorDate: Tue Feb 27 11:36:46 2024 +0800

    Refactor DataNodeRule (#30307)
    
    * Refactor DataNodeRule
    
    * Refactor DataNodeRule
    
    * Refactor DataNodeRule
    
    * Refactor DataNodeRule
---
 .../broadcast/rule/BroadcastDataNodeRule.java      |  82 +++++++++++++++
 .../broadcast/rule/BroadcastRule.java              |  64 +-----------
 .../broadcast/route/BroadcastSqlRouterTest.java    |   2 +-
 .../update/DropReadwriteSplittingRuleExecutor.java |   4 +-
 .../DropReadwriteSplittingRuleExecutorTest.java    |   7 +-
 .../constraint/ShardingConstraintReviser.java      |   2 +-
 .../sharding/rule/ShardingDataNodeRule.java        | 112 +++++++++++++++++++++
 .../shardingsphere/sharding/rule/ShardingRule.java |  51 +---------
 .../metadata/ShardingMetaDataReviseEngineTest.java |   5 +-
 .../sharding/rule/ShardingRuleTest.java            |  12 +--
 .../shardingsphere/infra/datanode/DataNodes.java   |   6 +-
 .../infra/metadata/database/rule/RuleMetaData.java |   5 +-
 .../collector/tables/PgClassTableCollector.java    |   6 +-
 .../type/datanode/DataNodeContainedRule.java       |  33 ++++++
 .../DataNodeRule.java}                             |   7 +-
 .../infra/datanode/DataNodesTest.java              |  16 ++-
 .../schema/util/SchemaMetaDataUtilsTest.java       |  15 +--
 .../handler/fixture/DistSQLHandlerFixtureRule.java |  40 ++------
 .../driver/executor/DriverJDBCExecutor.java        |   4 +-
 .../batch/BatchPreparedStatementExecutor.java      |   4 +-
 .../metadata/ShardingSphereDatabaseMetaData.java   |   7 +-
 .../core/resultset/DatabaseMetaDataResultSet.java  |   4 +-
 .../resultset/ShardingSphereResultSetMetaData.java |   6 +-
 .../statement/ShardingSpherePreparedStatement.java |   4 +-
 .../core/statement/ShardingSphereStatement.java    |   4 +-
 .../batch/BatchPreparedStatementExecutorTest.java  |   4 +-
 .../driver/jdbc/adapter/StatementAdapterTest.java  |   6 +-
 .../resultset/DatabaseMetaDataResultSetTest.java   |   5 +-
 .../data/pipeline/cdc/util/CDCDataNodeUtils.java   |   8 +-
 .../pipeline/cdc/util/CDCDataNodeUtilsTest.java    |  13 +--
 .../single/rule/SingleDataNodeRule.java            |  27 ++---
 .../shardingsphere/single/rule/SingleRule.java     |  42 ++------
 .../single/route/SingleSQLRouterTest.java          |   4 +-
 .../engine/SingleStandardRouteEngineTest.java      |   4 +-
 .../shardingsphere/single/rule/SingleRuleTest.java |  24 ++---
 .../handler/query/ShowSingleTableExecutor.java     |   2 +-
 .../handler/update/UnloadSingleTableExecutor.java  |   2 +-
 .../handler/query/ShowSingleTableExecutorTest.java |   4 +-
 .../proxy/backend/connector/DatabaseConnector.java |   4 +-
 .../header/query/MySQLQueryHeaderBuilder.java      |   4 +-
 .../header/query/MySQLQueryHeaderBuilderTest.java  |  10 +-
 41 files changed, 379 insertions(+), 286 deletions(-)

diff --git 
a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastDataNodeRule.java
 
b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastDataNodeRule.java
new file mode 100644
index 00000000000..508068d293c
--- /dev/null
+++ 
b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastDataNodeRule.java
@@ -0,0 +1,82 @@
+/*
+ * 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.broadcast.rule;
+
+import org.apache.shardingsphere.infra.datanode.DataNode;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Broadcast data node rule.
+ */
+public final class BroadcastDataNodeRule implements DataNodeRule {
+    
+    private final Collection<String> tables;
+    
+    private final Map<String, Collection<DataNode>> tableDataNodes;
+    
+    public BroadcastDataNodeRule(final Collection<String> dataSourceNames, 
final Collection<String> tables) {
+        this.tables = tables;
+        tableDataNodes = 
tables.stream().collect(Collectors.toMap(String::toLowerCase, each -> 
generateDataNodes(each, dataSourceNames)));
+    }
+    
+    private Collection<DataNode> generateDataNodes(final String logicTable, 
final Collection<String> dataSourceNames) {
+        return dataSourceNames.stream().map(each -> new DataNode(each, 
logicTable)).collect(Collectors.toList());
+    }
+    
+    @Override
+    public Map<String, Collection<DataNode>> getAllDataNodes() {
+        return tableDataNodes;
+    }
+    
+    @Override
+    public Collection<DataNode> getDataNodesByTableName(final String 
tableName) {
+        return tableDataNodes.getOrDefault(tableName, Collections.emptyList());
+    }
+    
+    @Override
+    public Optional<String> findFirstActualTable(final String logicTable) {
+        return tableDataNodes.containsKey(logicTable.toLowerCase()) ? 
Optional.of(logicTable) : Optional.empty();
+    }
+    
+    @Override
+    public boolean isNeedAccumulate(final Collection<String> tables) {
+        return tables.isEmpty() || !this.tables.containsAll(tables);
+    }
+    
+    @Override
+    public Optional<String> findLogicTableByActualTable(final String 
actualTable) {
+        return tableDataNodes.containsKey(actualTable.toLowerCase()) ? 
Optional.of(actualTable) : Optional.empty();
+    }
+    
+    @Override
+    public Optional<String> findActualTableByCatalog(final String catalog, 
final String logicTable) {
+        if (!tableDataNodes.containsKey(logicTable.toLowerCase())) {
+            return Optional.empty();
+        }
+        if 
(tableDataNodes.get(logicTable.toLowerCase()).stream().noneMatch(each -> 
each.getDataSourceName().equalsIgnoreCase(catalog))) {
+            return Optional.empty();
+        }
+        return Optional.of(logicTable);
+    }
+}
diff --git 
a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastRule.java
 
b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastRule.java
index 35454ee182e..cfca4a67c10 100644
--- 
a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastRule.java
+++ 
b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/rule/BroadcastRule.java
@@ -19,22 +19,19 @@ package org.apache.shardingsphere.broadcast.rule;
 
 import lombok.Getter;
 import 
org.apache.shardingsphere.broadcast.api.config.BroadcastRuleConfiguration;
-import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 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.DataSourceContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperContainedRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperRule;
 
 import javax.sql.DataSource;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Optional;
 import java.util.TreeSet;
 import java.util.stream.Collectors;
 
@@ -52,7 +49,7 @@ public final class BroadcastRule implements DatabaseRule, 
DataNodeContainedRule,
     
     private final Collection<String> dataSourceNames;
     
-    private final Map<String, Collection<DataNode>> tableDataNodes;
+    private final DataNodeRule dataNodeRule;
     
     private final TableMapperRule tableMapperRule;
     
@@ -61,9 +58,8 @@ public final class BroadcastRule implements DatabaseRule, 
DataNodeContainedRule,
         this.databaseName = databaseName;
         dataSourceNames = getAggregatedDataSourceNames(dataSources, 
builtRules);
         tables = createBroadcastTables(config.getTables());
-        tableDataNodes = createShardingTableDataNodes(dataSourceNames, tables);
+        dataNodeRule = new BroadcastDataNodeRule(dataSourceNames, tables);
         tableMapperRule = new BroadcastTableMapperRule(tables);
-        
     }
     
     private Collection<String> getAggregatedDataSourceNames(final Map<String, 
DataSource> dataSources, final Collection<ShardingSphereRule> builtRules) {
@@ -98,58 +94,6 @@ public final class BroadcastRule implements DatabaseRule, 
DataNodeContainedRule,
         return result;
     }
     
-    private Map<String, Collection<DataNode>> 
createShardingTableDataNodes(final Collection<String> dataSourceNames, final 
Collection<String> tables) {
-        Map<String, Collection<DataNode>> result = new 
HashMap<>(tables.size(), 1F);
-        for (String each : tables) {
-            result.put(each.toLowerCase(), generateDataNodes(each, 
dataSourceNames));
-        }
-        return result;
-    }
-    
-    private Collection<DataNode> generateDataNodes(final String logicTable, 
final Collection<String> dataSourceNames) {
-        Collection<DataNode> result = new LinkedList<>();
-        for (String each : dataSourceNames) {
-            result.add(new DataNode(each, logicTable));
-        }
-        return result;
-    }
-    
-    @Override
-    public Map<String, Collection<DataNode>> getAllDataNodes() {
-        return tableDataNodes;
-    }
-    
-    @Override
-    public Collection<DataNode> getDataNodesByTableName(final String 
tableName) {
-        return tableDataNodes.getOrDefault(tableName, Collections.emptyList());
-    }
-    
-    @Override
-    public Optional<String> findFirstActualTable(final String logicTable) {
-        return tableDataNodes.containsKey(logicTable.toLowerCase()) ? 
Optional.of(logicTable) : Optional.empty();
-    }
-    
-    @Override
-    public boolean isNeedAccumulate(final Collection<String> tables) {
-        return !isAllBroadcastTables(tables);
-    }
-    
-    @Override
-    public Optional<String> findLogicTableByActualTable(final String 
actualTable) {
-        return tableDataNodes.containsKey(actualTable.toLowerCase()) ? 
Optional.of(actualTable) : Optional.empty();
-    }
-    
-    @Override
-    public Optional<String> findActualTableByCatalog(final String catalog, 
final String logicTable) {
-        if (!tableDataNodes.containsKey(logicTable.toLowerCase())) {
-            return Optional.empty();
-        }
-        if 
(tableDataNodes.get(logicTable.toLowerCase()).stream().noneMatch(each -> 
each.getDataSourceName().equalsIgnoreCase(catalog))) {
-            return Optional.empty();
-        }
-        return Optional.of(logicTable);
-    }
-    
     /**
      * Get broadcast rule table names.
      * 
diff --git 
a/features/broadcast/core/src/test/java/org/apache/shardingsphere/broadcast/route/BroadcastSqlRouterTest.java
 
b/features/broadcast/core/src/test/java/org/apache/shardingsphere/broadcast/route/BroadcastSqlRouterTest.java
index 3f640ad4f72..64c594e81ce 100644
--- 
a/features/broadcast/core/src/test/java/org/apache/shardingsphere/broadcast/route/BroadcastSqlRouterTest.java
+++ 
b/features/broadcast/core/src/test/java/org/apache/shardingsphere/broadcast/route/BroadcastSqlRouterTest.java
@@ -88,7 +88,7 @@ class BroadcastSqlRouterTest {
         BroadcastRuleConfiguration currentConfig = 
mock(BroadcastRuleConfiguration.class);
         
when(currentConfig.getTables()).thenReturn(Collections.singleton("t_order"));
         BroadcastRule broadcastRule = new BroadcastRule(currentConfig, 
DefaultDatabase.LOGIC_NAME, Collections.singletonMap("tmp_ds", new 
MockedDataSource(mockConnection())), Collections.emptyList());
-        broadcastRule.getTableDataNodes().put("t_order", 
Collections.singletonList(createDataNode("tmp_ds")));
+        broadcastRule.getDataNodeRule().getAllDataNodes().put("t_order", 
Collections.singletonList(createDataNode("tmp_ds")));
         ShardingSphereDatabase database = mockSingleDatabase();
         RouteContext routeContext = new 
BroadcastSQLRouter().createRouteContext(createQueryContext(), 
mock(RuleMetaData.class), database, broadcastRule, new 
ConfigurationProperties(new Properties()),
                 new ConnectionContext());
diff --git 
a/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutor.java
 
b/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutor.java
index 7dd5f479ecd..bef6caca96f 100644
--- 
a/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutor.java
+++ 
b/features/readwrite-splitting/distsql/handler/src/main/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutor.java
@@ -26,7 +26,7 @@ import 
org.apache.shardingsphere.infra.algorithm.core.config.AlgorithmConfigurat
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import 
org.apache.shardingsphere.infra.exception.core.ShardingSpherePreconditions;
 import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.StaticDataSourceContainedRule;
 import 
org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
@@ -90,7 +90,7 @@ public final class DropReadwriteSplittingRuleExecutor 
implements DatabaseRuleDro
                 continue;
             }
             Collection<DataNode> actualDataNodes = new HashSet<>();
-            each.getAllDataNodes().values().forEach(actualDataNodes::addAll);
+            
each.getDataNodeRule().getAllDataNodes().values().forEach(actualDataNodes::addAll);
             
result.addAll(actualDataNodes.stream().map(DataNode::getDataSourceName).collect(Collectors.toSet()));
         }
         return result;
diff --git 
a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutorTest.java
 
b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutorTest.java
index 8d563c4e9e0..ae1f283f9a9 100644
--- 
a/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutorTest.java
+++ 
b/features/readwrite-splitting/distsql/handler/src/test/java/org/apache/shardingsphere/readwritesplitting/distsql/handler/update/DropReadwriteSplittingRuleExecutorTest.java
@@ -23,7 +23,7 @@ import 
org.apache.shardingsphere.distsql.handler.exception.rule.RuleInUsedExcept
 import 
org.apache.shardingsphere.infra.algorithm.core.config.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import 
org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
 import 
org.apache.shardingsphere.readwritesplitting.api.rule.ReadwriteSplittingDataSourceRuleConfiguration;
@@ -48,6 +48,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -82,8 +83,8 @@ class DropReadwriteSplittingRuleExecutorTest {
         DataSourceContainedRule dataSourceContainedRule = 
mock(DataSourceContainedRule.class);
         
when(dataSourceContainedRule.getDataSourceMapper()).thenReturn(Collections.singletonMap("foo_ds",
 Collections.singleton("readwrite_ds")));
         
when(database.getRuleMetaData().findRules(DataSourceContainedRule.class)).thenReturn(Collections.singleton(dataSourceContainedRule));
-        DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
-        
when(dataNodeContainedRule.getAllDataNodes()).thenReturn(Collections.singletonMap("foo_ds",
 Collections.singleton(new DataNode("readwrite_ds.tbl"))));
+        DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+        
when(dataNodeContainedRule.getDataNodeRule().getAllDataNodes()).thenReturn(Collections.singletonMap("foo_ds",
 Collections.singleton(new DataNode("readwrite_ds.tbl"))));
         
when(database.getRuleMetaData().findRules(DataNodeContainedRule.class)).thenReturn(Collections.singleton(dataNodeContainedRule));
         executor.setDatabase(database);
         ReadwriteSplittingRule rule = mock(ReadwriteSplittingRule.class);
diff --git 
a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/constraint/ShardingConstraintReviser.java
 
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/constraint/ShardingConstraintReviser.java
index b422f4b163f..51c602663d7 100644
--- 
a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/constraint/ShardingConstraintReviser.java
+++ 
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/constraint/ShardingConstraintReviser.java
@@ -40,7 +40,7 @@ public final class ShardingConstraintReviser implements 
ConstraintReviser<Shardi
             String referencedTableName = 
originalMetaData.getReferencedTableName();
             Optional<String> logicIndexName = 
getLogicIndex(originalMetaData.getName(), each.getTableName());
             if (logicIndexName.isPresent()) {
-                return Optional.of(new 
ConstraintMetaData(logicIndexName.get(), 
rule.findLogicTableByActualTable(referencedTableName).orElse(referencedTableName)));
+                return Optional.of(new 
ConstraintMetaData(logicIndexName.get(), 
rule.getDataNodeRule().findLogicTableByActualTable(referencedTableName).orElse(referencedTableName)));
             }
         }
         return Optional.empty();
diff --git 
a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingDataNodeRule.java
 
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingDataNodeRule.java
new file mode 100644
index 00000000000..fe19f0a1bdb
--- /dev/null
+++ 
b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/rule/ShardingDataNodeRule.java
@@ -0,0 +1,112 @@
+/*
+ * 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.rule;
+
+import com.cedarsoftware.util.CaseInsensitiveMap;
+import com.google.common.base.Strings;
+import org.apache.shardingsphere.infra.datanode.DataNode;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Sharding data node rule.
+ */
+public final class ShardingDataNodeRule implements DataNodeRule {
+    
+    private final Map<String, ShardingTable> shardingTables;
+    
+    private final Map<String, Collection<DataNode>> tableDataNodes;
+    
+    public ShardingDataNodeRule(final Map<String, ShardingTable> 
shardingTables) {
+        this.shardingTables = shardingTables;
+        tableDataNodes = createShardingTableDataNodes(shardingTables);
+    }
+    
+    private Map<String, Collection<DataNode>> 
createShardingTableDataNodes(final Map<String, ShardingTable> shardingTables) {
+        Map<String, Collection<DataNode>> result = new 
CaseInsensitiveMap<>(shardingTables.size(), 1F);
+        for (ShardingTable each : shardingTables.values()) {
+            result.put(each.getLogicTable(), each.getActualDataNodes());
+        }
+        return result;
+    }
+    
+    @Override
+    public Map<String, Collection<DataNode>> getAllDataNodes() {
+        return tableDataNodes;
+    }
+    
+    @Override
+    public Collection<DataNode> getDataNodesByTableName(final String 
tableName) {
+        return tableDataNodes.getOrDefault(tableName, Collections.emptyList());
+    }
+    
+    @Override
+    public Optional<String> findFirstActualTable(final String logicTable) {
+        return findShardingTable(logicTable).map(optional -> 
optional.getActualDataNodes().get(0).getTableName());
+    }
+    
+    private Optional<ShardingTable> findShardingTable(final String 
logicTableName) {
+        return Strings.isNullOrEmpty(logicTableName) || 
!shardingTables.containsKey(logicTableName) ? Optional.empty() : 
Optional.of(shardingTables.get(logicTableName));
+    }
+    
+    @Override
+    public boolean isNeedAccumulate(final Collection<String> tables) {
+        return containsShardingTable(tables);
+    }
+    
+    private boolean containsShardingTable(final Collection<String> 
logicTableNames) {
+        for (String each : logicTableNames) {
+            if (isShardingTable(each)) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    private boolean isShardingTable(final String logicTableName) {
+        return shardingTables.containsKey(logicTableName);
+    }
+    
+    @Override
+    public Optional<String> findLogicTableByActualTable(final String 
actualTable) {
+        return 
findShardingTableByActualTable(actualTable).map(ShardingTable::getLogicTable);
+    }
+    
+    private Optional<ShardingTable> findShardingTableByActualTable(final 
String actualTableName) {
+        for (ShardingTable each : shardingTables.values()) {
+            if (each.isExisted(actualTableName)) {
+                return Optional.of(each);
+            }
+        }
+        return Optional.empty();
+    }
+    
+    @Override
+    public Optional<String> findActualTableByCatalog(final String catalog, 
final String logicTable) {
+        return findShardingTable(logicTable).flatMap(optional -> 
findActualTableFromActualDataNode(catalog, optional.getActualDataNodes()));
+    }
+    
+    private Optional<String> findActualTableFromActualDataNode(final String 
catalog, final List<DataNode> actualDataNodes) {
+        return actualDataNodes.stream().filter(each -> 
each.getDataSourceName().equalsIgnoreCase(catalog)).findFirst().map(DataNode::getTableName);
+    }
+}
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 a536b7a2a86..a177dd2bc0d 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
@@ -35,7 +35,8 @@ import 
org.apache.shardingsphere.infra.instance.InstanceContextAware;
 import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
 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.datanode.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperContainedRule;
 import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
@@ -113,10 +114,10 @@ public final class ShardingRule implements DatabaseRule, 
DataNodeContainedRule,
     
     private final String defaultShardingColumn;
     
-    private final Map<String, Collection<DataNode>> shardingTableDataNodes;
-    
     private final ShardingCache shardingCache;
     
+    private final DataNodeRule dataNodeRule;
+    
     private final ShardingTableMapperRule tableMapperRule;
     
     public ShardingRule(final ShardingRuleConfiguration ruleConfig, final 
Map<String, DataSource> dataSources, final InstanceContext instanceContext) {
@@ -136,7 +137,6 @@ public final class ShardingRule implements DatabaseRule, 
DataNodeContainedRule,
                 ? TypedSPILoader.getService(KeyGenerateAlgorithm.class, null)
                 : 
keyGenerators.get(ruleConfig.getDefaultKeyGenerateStrategy().getKeyGeneratorName());
         defaultShardingColumn = ruleConfig.getDefaultShardingColumn();
-        shardingTableDataNodes = createShardingTableDataNodes(shardingTables);
         
ShardingSpherePreconditions.checkState(isValidBindingTableConfiguration(shardingTables,
 new BindingTableCheckedConfiguration(this.dataSourceNames, shardingAlgorithms,
                 ruleConfig.getBindingTableGroups(), 
defaultDatabaseShardingStrategyConfig, defaultTableShardingStrategyConfig, 
defaultShardingColumn)),
                 InvalidBindingTablesException::new);
@@ -145,6 +145,7 @@ public final class ShardingRule implements DatabaseRule, 
DataNodeContainedRule,
             ((InstanceContextAware) 
defaultKeyGenerateAlgorithm).setInstanceContext(instanceContext);
         }
         shardingCache = null == ruleConfig.getShardingCache() ? null : new 
ShardingCache(ruleConfig.getShardingCache(), this);
+        dataNodeRule = new ShardingDataNodeRule(shardingTables);
         tableMapperRule = new ShardingTableMapperRule(shardingTables);
     }
     
@@ -168,14 +169,6 @@ public final class ShardingRule implements DatabaseRule, 
DataNodeContainedRule,
         return null == ruleConfig.getDefaultTableShardingStrategy() ? new 
NoneShardingStrategyConfiguration() : 
ruleConfig.getDefaultTableShardingStrategy();
     }
     
-    private Map<String, Collection<DataNode>> 
createShardingTableDataNodes(final Map<String, ShardingTable> shardingTables) {
-        Map<String, Collection<DataNode>> result = new 
CaseInsensitiveMap<>(shardingTables.size(), 1F);
-        for (ShardingTable each : shardingTables.values()) {
-            result.put(each.getLogicTable(), each.getActualDataNodes());
-        }
-        return result;
-    }
-    
     private Collection<String> getDataSourceNames(final 
Collection<ShardingTableRuleConfiguration> tableRuleConfigs,
                                                   final 
Collection<ShardingAutoTableRuleConfiguration> autoTableRuleConfigs, final 
Collection<String> dataSourceNames) {
         if (tableRuleConfigs.isEmpty() && autoTableRuleConfigs.isEmpty()) {
@@ -679,40 +672,6 @@ public final class ShardingRule implements DatabaseRule, 
DataNodeContainedRule,
         return null != shardingCache;
     }
     
-    @Override
-    public Map<String, Collection<DataNode>> getAllDataNodes() {
-        return shardingTableDataNodes;
-    }
-    
-    @Override
-    public Collection<DataNode> getDataNodesByTableName(final String 
tableName) {
-        return shardingTableDataNodes.getOrDefault(tableName, 
Collections.emptyList());
-    }
-    
-    @Override
-    public Optional<String> findFirstActualTable(final String logicTable) {
-        return findShardingTable(logicTable).map(optional -> 
optional.getActualDataNodes().get(0).getTableName());
-    }
-    
-    @Override
-    public boolean isNeedAccumulate(final Collection<String> tables) {
-        return containsShardingTable(tables);
-    }
-    
-    @Override
-    public Optional<String> findLogicTableByActualTable(final String 
actualTable) {
-        return 
findShardingTableByActualTable(actualTable).map(ShardingTable::getLogicTable);
-    }
-    
-    @Override
-    public Optional<String> findActualTableByCatalog(final String catalog, 
final String logicTable) {
-        return findShardingTable(logicTable).flatMap(optional -> 
findActualTableFromActualDataNode(catalog, optional.getActualDataNodes()));
-    }
-    
-    private Optional<String> findActualTableFromActualDataNode(final String 
catalog, final List<DataNode> actualDataNodes) {
-        return actualDataNodes.stream().filter(each -> 
each.getDataSourceName().equalsIgnoreCase(catalog)).findFirst().map(DataNode::getTableName);
-    }
-    
     private boolean isJoinConditionContainsShardingColumns(final 
ShardingSphereSchema schema, final SelectStatementContext select,
                                                            final 
Collection<String> tableNames, final Collection<WhereSegment> whereSegments) {
         Collection<String> databaseJoinConditionTables = new 
HashSet<>(tableNames.size(), 1F);
diff --git 
a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataReviseEngineTest.java
 
b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataReviseEngineTest.java
index 4724af365f6..b7daa23b04a 100644
--- 
a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataReviseEngineTest.java
+++ 
b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/ShardingMetaDataReviseEngineTest.java
@@ -23,6 +23,7 @@ import 
org.apache.shardingsphere.infra.database.core.metadata.data.model.SchemaM
 import 
org.apache.shardingsphere.infra.database.core.metadata.data.model.TableMetaData;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterial;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.reviser.MetaDataReviseEngine;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.junit.jupiter.api.Test;
 
@@ -56,7 +57,9 @@ class ShardingMetaDataReviseEngineTest {
     
     private ShardingRule mockShardingRule() {
         ShardingRule result = mock(ShardingRule.class);
-        
when(result.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_order"));
+        DataNodeRule dataNodeRule = mock(DataNodeRule.class);
+        
when(dataNodeRule.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_order"));
+        when(result.getDataNodeRule()).thenReturn(dataNodeRule);
         return result;
     }
     
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 c6e21ab543d..8376026a7d5 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
@@ -202,12 +202,12 @@ class ShardingRuleTest {
     
     @Test
     void assertFindLogicTableByActualTable() {
-        
assertTrue(createMaximumShardingRule().findLogicTableByActualTable("table_0").isPresent());
+        
assertTrue(createMaximumShardingRule().getDataNodeRule().findLogicTableByActualTable("table_0").isPresent());
     }
     
     @Test
     void assertNotFindLogicTableByActualTable() {
-        
assertFalse(createMaximumShardingRule().findLogicTableByActualTable("table_3").isPresent());
+        
assertFalse(createMaximumShardingRule().getDataNodeRule().findLogicTableByActualTable("table_3").isPresent());
     }
     
     @Test
@@ -711,7 +711,7 @@ class ShardingRuleTest {
     @Test
     void assertGetDataNodesByTableName() {
         ShardingRule shardingRule = createMinimumShardingRule();
-        Collection<DataNode> actual = 
shardingRule.getDataNodesByTableName("logic_table");
+        Collection<DataNode> actual = 
shardingRule.getDataNodeRule().getDataNodesByTableName("logic_table");
         assertThat(actual.size(), is(6));
         Iterator<DataNode> iterator = actual.iterator();
         DataNode firstDataNode = iterator.next();
@@ -773,7 +773,7 @@ class ShardingRuleTest {
     @Test
     void assertGetAllDataNodes() {
         ShardingRule actual = createMaximumShardingRule();
-        Map<String, Collection<DataNode>> allDataNodes = 
actual.getAllDataNodes();
+        Map<String, Collection<DataNode>> allDataNodes = 
actual.getDataNodeRule().getAllDataNodes();
         assertTrue(allDataNodes.containsKey("logic_table"));
         assertTrue(allDataNodes.containsKey("sub_logic_table"));
         Collection<DataNode> logicTableDataNodes = 
allDataNodes.get("logic_table");
@@ -798,14 +798,14 @@ class ShardingRuleTest {
     @Test
     void assertFindFirstActualTable() {
         ShardingRule actual = createMaximumShardingRule();
-        Optional<String> logicTable = 
actual.findFirstActualTable("logic_table");
+        Optional<String> logicTable = 
actual.getDataNodeRule().findFirstActualTable("logic_table");
         assertThat(logicTable.orElse(""), is("table_0"));
     }
     
     @Test
     void assertFindActualTableByCatalog() {
         ShardingRule actual = createMaximumShardingRule();
-        Optional<String> actualTableByCatalog = 
actual.findActualTableByCatalog("ds_0", "logic_table");
+        Optional<String> actualTableByCatalog = 
actual.getDataNodeRule().findActualTableByCatalog("ds_0", "logic_table");
         assertThat(actualTableByCatalog.orElse(""), is("table_0"));
     }
     
diff --git 
a/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
 
b/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
index 4b593121e9d..0daadd06c8c 100644
--- 
a/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
+++ 
b/infra/common/src/main/java/org/apache/shardingsphere/infra/datanode/DataNodes.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.infra.datanode;
 
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.spi.type.ordered.OrderedSPILoader;
 
 import java.util.Collection;
@@ -57,7 +57,7 @@ public final class DataNodes {
         if (!dataNodeContainedRule.isPresent()) {
             return Collections.emptyList();
         }
-        Collection<DataNode> result = new 
LinkedList<>(dataNodeContainedRule.get().getDataNodesByTableName(tableName));
+        Collection<DataNode> result = new 
LinkedList<>(dataNodeContainedRule.get().getDataNodeRule().getDataNodesByTableName(tableName));
         for (Entry<ShardingSphereRule, DataNodeBuilder> entry : 
dataNodeBuilders.entrySet()) {
             result = entry.getValue().build(result, entry.getKey());
         }
@@ -69,6 +69,6 @@ public final class DataNodes {
     }
     
     private boolean isDataNodeContainedRuleContainsTable(final 
ShardingSphereRule each, final String tableName) {
-        return each instanceof DataNodeContainedRule && 
!((DataNodeContainedRule) each).getDataNodesByTableName(tableName).isEmpty();
+        return each instanceof DataNodeContainedRule && 
!((DataNodeContainedRule) 
each).getDataNodeRule().getDataNodesByTableName(tableName).isEmpty();
     }
 }
diff --git 
a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/RuleMetaData.java
 
b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/RuleMetaData.java
index 5d56abca1a0..0742ffe6baa 100644
--- 
a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/RuleMetaData.java
+++ 
b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/rule/RuleMetaData.java
@@ -22,7 +22,7 @@ import lombok.Getter;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 
 import java.util.Collection;
@@ -131,7 +131,8 @@ public final class RuleMetaData {
     }
     
     private Collection<String> getInUsedStorageUnitNames(final 
DataNodeContainedRule rule) {
-        return rule.getAllDataNodes().values().stream().flatMap(each -> 
each.stream().map(DataNode::getDataSourceName).collect(Collectors.toSet()).stream()).collect(Collectors.toSet());
+        return rule.getDataNodeRule().getAllDataNodes().values().stream()
+                .flatMap(each -> 
each.stream().map(DataNode::getDataSourceName).collect(Collectors.toSet()).stream()).collect(Collectors.toSet());
     }
     
     private void mergeInUsedStorageUnitNameAndRules(final Map<String, 
Collection<Class<? extends ShardingSphereRule>>> storageUnitNameAndRules,
diff --git 
a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/statistics/collector/tables/PgClassTableCollector.java
 
b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/statistics/collector/tables/PgClassTableCollector.java
index 5aa7efeedfc..f2846f1f9dd 100644
--- 
a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/statistics/collector/tables/PgClassTableCollector.java
+++ 
b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/statistics/collector/tables/PgClassTableCollector.java
@@ -24,7 +24,7 @@ import 
org.apache.shardingsphere.infra.metadata.statistics.collector.ShardingSph
 import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -77,8 +77,8 @@ public final class PgClassTableCollector implements 
ShardingSphereStatisticsColl
     
     private String decorateTableName(final Collection<DataNodeContainedRule> 
dataNodeContainedRules, final String actualTableName) {
         for (DataNodeContainedRule each : dataNodeContainedRules) {
-            if (each.findLogicTableByActualTable(actualTableName).isPresent()) 
{
-                return each.findLogicTableByActualTable(actualTableName).get();
+            if 
(each.getDataNodeRule().findLogicTableByActualTable(actualTableName).isPresent())
 {
+                return 
each.getDataNodeRule().findLogicTableByActualTable(actualTableName).get();
             }
         }
         return actualTableName;
diff --git 
a/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeContainedRule.java
 
b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeContainedRule.java
new file mode 100644
index 00000000000..9403edbd6b1
--- /dev/null
+++ 
b/infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeContainedRule.java
@@ -0,0 +1,33 @@
+/*
+ * 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.datanode;
+
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+
+/**
+ * ShardingSphere rule which contains data node.
+ */
+public interface DataNodeContainedRule extends ShardingSphereRule {
+    
+    /**
+     * Get data node rule.
+     * 
+     * @return data node rule
+     */
+    DataNodeRule getDataNodeRule();
+}
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/datanode/DataNodeRule.java
similarity index 90%
rename from 
infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/DataNodeContainedRule.java
rename to 
infra/common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/datanode/DataNodeRule.java
index 3526434cc1f..aa2537e14e1 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/datanode/DataNodeRule.java
@@ -15,19 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.rule.identifier.type;
+package org.apache.shardingsphere.infra.rule.identifier.type.datanode;
 
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
 import java.util.Collection;
 import java.util.Map;
 import java.util.Optional;
 
 /**
- * ShardingSphere rule which contains data node.
+ * Data node rule.
  */
-public interface DataNodeContainedRule extends ShardingSphereRule {
+public interface DataNodeRule {
     
     /**
      * Get all data nodes.
diff --git 
a/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodesTest.java
 
b/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodesTest.java
index e8caf1d4719..7f784f6ceb7 100644
--- 
a/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodesTest.java
+++ 
b/infra/common/src/test/java/org/apache/shardingsphere/infra/datanode/DataNodesTest.java
@@ -19,8 +19,9 @@ package org.apache.shardingsphere.infra.datanode;
 
 import org.apache.shardingsphere.infra.fixture.FixtureRule;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
 import org.junit.jupiter.api.Test;
 
 import java.util.Arrays;
@@ -33,6 +34,7 @@ import java.util.Map;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -147,17 +149,21 @@ class DataNodesTest {
     }
     
     private ShardingSphereRule mockSingleRule() {
-        DataNodeContainedRule result = mock(DataNodeContainedRule.class);
-        
when(result.getDataNodesByTableName("t_single")).thenReturn(Collections.singletonList(new
 DataNode("readwrite_ds", "t_single")));
+        DataNodeRule dataNodeRule = mock(DataNodeRule.class);
+        
when(dataNodeRule.getDataNodesByTableName("t_single")).thenReturn(Collections.singleton(new
 DataNode("readwrite_ds", "t_single")));
+        DataNodeContainedRule result = mock(DataNodeContainedRule.class, 
RETURNS_DEEP_STUBS);
+        when(result.getDataNodeRule()).thenReturn(dataNodeRule);
         return result;
     }
     
     private ShardingSphereRule mockShardingRule() {
-        DataNodeContainedRule result = mock(DataNodeContainedRule.class);
         Collection<DataNode> dataNodes = new LinkedList<>();
         dataNodes.add(new DataNode("readwrite_ds", "t_order_0"));
         dataNodes.add(new DataNode("readwrite_ds", "t_order_1"));
-        when(result.getDataNodesByTableName("t_order")).thenReturn(dataNodes);
+        DataNodeRule dataNodeRule = mock(DataNodeRule.class);
+        
when(dataNodeRule.getDataNodesByTableName("t_order")).thenReturn(dataNodes);
+        DataNodeContainedRule result = mock(DataNodeContainedRule.class);
+        when(result.getDataNodeRule()).thenReturn(dataNodeRule);
         return result;
     }
 }
diff --git 
a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SchemaMetaDataUtilsTest.java
 
b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SchemaMetaDataUtilsTest.java
index f08a63688ba..dfb22e76ee4 100644
--- 
a/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SchemaMetaDataUtilsTest.java
+++ 
b/infra/common/src/test/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SchemaMetaDataUtilsTest.java
@@ -22,7 +22,7 @@ import 
org.apache.shardingsphere.infra.database.core.metadata.data.loader.MetaDa
 import org.apache.shardingsphere.infra.database.core.type.DatabaseType;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterial;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.test.fixture.jdbc.MockedDataSource;
@@ -39,6 +39,7 @@ import java.util.Map;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -46,8 +47,8 @@ class SchemaMetaDataUtilsTest {
     
     @Test
     void assertGetSchemaMetaDataLoaderMaterialsWhenConfigCheckMetaDataEnable() 
{
-        DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
-        
when(dataNodeContainedRule.getDataNodesByTableName("t_order")).thenReturn(mockShardingDataNodes());
+        DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+        
when(dataNodeContainedRule.getDataNodeRule().getDataNodesByTableName("t_order")).thenReturn(mockShardingDataNodes());
         GenericSchemaBuilderMaterial material = new 
GenericSchemaBuilderMaterial(mock(DatabaseType.class), mockStorageTypes(), 
mockDataSourceMap(),
                 Arrays.asList(dataNodeContainedRule, 
mock(DataSourceContainedRule.class)), mock(ConfigurationProperties.class), 
"sharding_db");
         Collection<MetaDataLoaderMaterial> actual = 
SchemaMetaDataUtils.getMetaDataLoaderMaterials(Collections.singleton("t_order"),
 material, true);
@@ -63,8 +64,8 @@ class SchemaMetaDataUtilsTest {
     
     @Test
     void 
assertGetSchemaMetaDataLoaderMaterialsWhenNotConfigCheckMetaDataEnable() {
-        DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
-        
when(dataNodeContainedRule.getDataNodesByTableName("t_order")).thenReturn(mockShardingDataNodes());
+        DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+        
when(dataNodeContainedRule.getDataNodeRule().getDataNodesByTableName("t_order")).thenReturn(mockShardingDataNodes());
         GenericSchemaBuilderMaterial material = new 
GenericSchemaBuilderMaterial(mock(DatabaseType.class), mockStorageTypes(), 
mockDataSourceMap(),
                 Arrays.asList(dataNodeContainedRule, 
mock(DataSourceContainedRule.class)), mock(ConfigurationProperties.class), 
"sharding_db");
         Collection<MetaDataLoaderMaterial> actual = 
SchemaMetaDataUtils.getMetaDataLoaderMaterials(Collections.singleton("t_order"),
 material, false);
@@ -77,8 +78,8 @@ class SchemaMetaDataUtilsTest {
     
     @Test
     void 
assertGetSchemaMetaDataLoaderMaterialsWhenNotConfigCheckMetaDataEnableForSingleTableDataNode()
 {
-        DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
-        
when(dataNodeContainedRule.getDataNodesByTableName("t_single")).thenReturn(mockSingleTableDataNodes());
+        DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+        
when(dataNodeContainedRule.getDataNodeRule().getDataNodesByTableName("t_single")).thenReturn(mockSingleTableDataNodes());
         GenericSchemaBuilderMaterial material = new 
GenericSchemaBuilderMaterial(mock(DatabaseType.class), mockStorageTypes(), 
mockDataSourceMap(),
                 Arrays.asList(dataNodeContainedRule, 
mock(DataSourceContainedRule.class)), mock(ConfigurationProperties.class), 
"public");
         Collection<MetaDataLoaderMaterial> actual = 
SchemaMetaDataUtils.getMetaDataLoaderMaterials(Collections.singleton("t_single"),
 material, false);
diff --git 
a/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
 
b/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
index cde6be16d0c..f43a56109a1 100644
--- 
a/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
+++ 
b/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
@@ -20,13 +20,13 @@ package org.apache.shardingsphere.distsql.handler.fixture;
 import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
 
-import java.util.Collection;
 import java.util.Collections;
-import java.util.Map;
 import java.util.Optional;
 
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -38,34 +38,14 @@ public final class DistSQLHandlerFixtureRule implements 
ShardingSphereRule, Data
     }
     
     @Override
-    public Map<String, Collection<DataNode>> getAllDataNodes() {
+    public DataNodeRule getDataNodeRule() {
+        DataNodeRule result = mock(DataNodeRule.class);
         DataNode dataNode = mock(DataNode.class);
         when(dataNode.getDataSourceName()).thenReturn("foo_ds");
-        return Collections.singletonMap("", Collections.singleton(dataNode));
-    }
-    
-    @Override
-    public Collection<DataNode> getDataNodesByTableName(final String 
tableName) {
-        return null;
-    }
-    
-    @Override
-    public Optional<String> findFirstActualTable(final String logicTable) {
-        return Optional.empty();
-    }
-    
-    @Override
-    public boolean isNeedAccumulate(final Collection<String> tables) {
-        return false;
-    }
-    
-    @Override
-    public Optional<String> findLogicTableByActualTable(final String 
actualTable) {
-        return Optional.empty();
-    }
-    
-    @Override
-    public Optional<String> findActualTableByCatalog(final String catalog, 
final String logicTable) {
-        return Optional.empty();
+        when(result.getAllDataNodes()).thenReturn(Collections.singletonMap("", 
Collections.singleton(dataNode)));
+        when(result.findFirstActualTable(any())).thenReturn(Optional.empty());
+        
when(result.findLogicTableByActualTable(any())).thenReturn(Optional.empty());
+        when(result.findActualTableByCatalog(any(), 
any())).thenReturn(Optional.empty());
+        return result;
     }
 }
diff --git 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
index 2c5308a90c5..5e7fbaac352 100644
--- 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
+++ 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/DriverJDBCExecutor.java
@@ -30,7 +30,7 @@ import 
org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryRe
 import org.apache.shardingsphere.infra.executor.sql.process.ProcessEngine;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 
@@ -105,7 +105,7 @@ public final class DriverJDBCExecutor {
     
     private boolean isNeedAccumulate(final Collection<ShardingSphereRule> 
rules, final SQLStatementContext sqlStatementContext) {
         for (ShardingSphereRule each : rules) {
-            if (each instanceof DataNodeContainedRule && 
((DataNodeContainedRule) 
each).isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames())) 
{
+            if (each instanceof DataNodeContainedRule && 
((DataNodeContainedRule) 
each).getDataNodeRule().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames()))
 {
                 return true;
             }
         }
diff --git 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
index 94c18eccbef..7860c66094e 100644
--- 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
+++ 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutor.java
@@ -31,7 +31,7 @@ import 
org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.J
 import 
org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.JDBCExecutorCallback;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
@@ -159,7 +159,7 @@ public final class BatchPreparedStatementExecutor {
     
     private boolean isNeedAccumulate(final SQLStatementContext 
sqlStatementContext) {
         for (ShardingSphereRule each : 
metaDataContexts.getMetaData().getDatabase(databaseName).getRuleMetaData().getRules())
 {
-            if (each instanceof DataNodeContainedRule && 
((DataNodeContainedRule) 
each).isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames())) 
{
+            if (each instanceof DataNodeContainedRule && 
((DataNodeContainedRule) 
each).getDataNodeRule().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames()))
 {
                 return true;
             }
         }
diff --git 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
index 29f37c764f3..4e71ffcd39a 100644
--- 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
+++ 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/metadata/ShardingSphereDatabaseMetaData.java
@@ -23,7 +23,7 @@ import 
org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConne
 import 
org.apache.shardingsphere.driver.jdbc.core.resultset.DatabaseMetaDataResultSet;
 import 
org.apache.shardingsphere.infra.database.core.connector.ConnectionProperties;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
@@ -201,7 +201,8 @@ public final class ShardingSphereDatabaseMetaData extends 
AdaptedDatabaseMetaDat
     private String getActualTableNamePattern(final String tableNamePattern) {
         return null == tableNamePattern
                 ? null
-                : findDataNodeContainedRule().filter(optional -> 
optional.findFirstActualTable(tableNamePattern).isPresent()).map(optional -> 
"%" + tableNamePattern + "%").orElse(tableNamePattern);
+                : findDataNodeContainedRule()
+                        .filter(optional -> 
optional.getDataNodeRule().findFirstActualTable(tableNamePattern).isPresent()).map(optional
 -> "%" + tableNamePattern + "%").orElse(tableNamePattern);
     }
     
     private String getActualTable(final String catalog, final String table) {
@@ -209,7 +210,7 @@ public final class ShardingSphereDatabaseMetaData extends 
AdaptedDatabaseMetaDat
     }
     
     private Optional<String> findActualTable(final DataNodeContainedRule 
dataNodeContainedRule, final String catalog, final String table) {
-        return Strings.isNullOrEmpty(catalog) ? 
dataNodeContainedRule.findFirstActualTable(table) : 
dataNodeContainedRule.findActualTableByCatalog(catalog, table);
+        return Strings.isNullOrEmpty(catalog) ? 
dataNodeContainedRule.getDataNodeRule().findFirstActualTable(table) : 
dataNodeContainedRule.getDataNodeRule().findActualTableByCatalog(catalog, 
table);
     }
     
     private Optional<DataNodeContainedRule> findDataNodeContainedRule() {
diff --git 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
index 5c44c3f0483..321895a080a 100644
--- 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
+++ 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSet.java
@@ -25,7 +25,7 @@ import 
org.apache.shardingsphere.driver.jdbc.exception.syntax.ColumnLabelNotFoun
 import 
org.apache.shardingsphere.driver.jdbc.unsupported.AbstractUnsupportedDatabaseMetaDataResultSet;
 import 
org.apache.shardingsphere.infra.executor.sql.execute.result.query.impl.driver.jdbc.type.util.ResultSetUtils;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 
 import java.math.BigDecimal;
 import java.net.URL;
@@ -111,7 +111,7 @@ public final class DatabaseMetaDataResultSet extends 
AbstractUnsupportedDatabase
         for (int i = 1; i <= columnLabelIndexMap.size(); i++) {
             if (tableNameColumnIndex == i) {
                 String tableName = resultSet.getString(i);
-                Optional<String> logicTableName = 
dataNodeContainedRule.isPresent() ? 
dataNodeContainedRule.get().findLogicTableByActualTable(tableName) : 
Optional.empty();
+                Optional<String> logicTableName = 
dataNodeContainedRule.isPresent() ? 
dataNodeContainedRule.get().getDataNodeRule().findLogicTableByActualTable(tableName)
 : Optional.empty();
                 result.addObject(logicTableName.orElse(tableName));
             } else if (indexNameColumnIndex == i) {
                 String tableName = resultSet.getString(tableNameColumnIndex);
diff --git 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetMetaData.java
 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetMetaData.java
index a936c0f7367..1e1a097623f 100644
--- 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetMetaData.java
+++ 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/resultset/ShardingSphereResultSetMetaData.java
@@ -25,7 +25,7 @@ import 
org.apache.shardingsphere.infra.binder.context.statement.SQLStatementCont
 import 
org.apache.shardingsphere.infra.binder.context.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.database.core.DefaultDatabase;
 import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
@@ -147,8 +147,8 @@ public final class ShardingSphereResultSetMetaData extends 
WrapperAdapter implem
     
     private String decorateTableName(final Collection<DataNodeContainedRule> 
dataNodeContainedRules, final String actualTableName) {
         for (DataNodeContainedRule each : dataNodeContainedRules) {
-            if (each.findLogicTableByActualTable(actualTableName).isPresent()) 
{
-                return each.findLogicTableByActualTable(actualTableName).get();
+            if 
(each.getDataNodeRule().findLogicTableByActualTable(actualTableName).isPresent())
 {
+                return 
each.getDataNodeRule().findLogicTableByActualTable(actualTableName).get();
             }
         }
         return actualTableName;
diff --git 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 6f289b7aced..81c58612823 100644
--- 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -76,7 +76,7 @@ import org.apache.shardingsphere.infra.metadata.user.Grantee;
 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.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.RawExecutionRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.StorageConnectorReusableRule;
 import org.apache.shardingsphere.infra.session.query.QueryContext;
@@ -761,7 +761,7 @@ public final class ShardingSpherePreparedStatement extends 
AbstractPreparedState
     @Override
     public boolean isAccumulate() {
         return 
metaDataContexts.getMetaData().getDatabase(databaseName).getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .anyMatch(each -> 
each.isNeedAccumulate(executionContext.getSqlStatementContext().getTablesContext().getTableNames()));
+                .anyMatch(each -> 
each.getDataNodeRule().isNeedAccumulate(executionContext.getSqlStatementContext().getTablesContext().getTableNames()));
     }
     
     @Override
diff --git 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
index 83de9d7a266..1bdc9d889ec 100644
--- 
a/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
+++ 
b/jdbc/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSphereStatement.java
@@ -70,7 +70,7 @@ import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.rule.RuleMetaData;
 import org.apache.shardingsphere.infra.metadata.user.Grantee;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.RawExecutionRule;
 import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
@@ -665,7 +665,7 @@ public final class ShardingSphereStatement extends 
AbstractStatementAdapter {
     @Override
     public boolean isAccumulate() {
         return 
metaDataContexts.getMetaData().getDatabase(databaseName).getRuleMetaData().findRules(DataNodeContainedRule.class).stream()
-                .anyMatch(each -> 
each.isNeedAccumulate(executionContext.getSqlStatementContext().getTablesContext().getTableNames()));
+                .anyMatch(each -> 
each.getDataNodeRule().isNeedAccumulate(executionContext.getSqlStatementContext().getTablesContext().getTableNames()));
     }
     
     @Override
diff --git 
a/jdbc/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
 
b/jdbc/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
index 23a6fc48c01..09a15092caf 100644
--- 
a/jdbc/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
+++ 
b/jdbc/src/test/java/org/apache/shardingsphere/driver/executor/batch/BatchPreparedStatementExecutorTest.java
@@ -115,8 +115,8 @@ class BatchPreparedStatementExecutorTest {
     }
     
     private ShardingRule mockShardingRule() {
-        ShardingRule result = mock(ShardingRule.class);
-        when(result.isNeedAccumulate(any())).thenReturn(true);
+        ShardingRule result = mock(ShardingRule.class, RETURNS_DEEP_STUBS);
+        
when(result.getDataNodeRule().isNeedAccumulate(any())).thenReturn(true);
         return result;
     }
     
diff --git 
a/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
 
b/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
index e907a752eac..1622ce2f07a 100644
--- 
a/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
+++ 
b/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/adapter/StatementAdapterTest.java
@@ -25,7 +25,7 @@ import 
org.apache.shardingsphere.infra.database.core.DefaultDatabase;
 import org.apache.shardingsphere.infra.database.core.type.DatabaseType;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.metadata.database.rule.RuleMetaData;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
 import 
org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
@@ -252,8 +252,8 @@ class StatementAdapterTest {
     
     private ShardingSphereStatement 
mockShardingSphereStatementWithNeedAccumulate(final Statement... statements) {
         ShardingSphereConnection connection = 
mock(ShardingSphereConnection.class, RETURNS_DEEP_STUBS);
-        DataNodeContainedRule rule = mock(DataNodeContainedRule.class);
-        when(rule.isNeedAccumulate(any())).thenReturn(true);
+        DataNodeContainedRule rule = mock(DataNodeContainedRule.class, 
RETURNS_DEEP_STUBS);
+        when(rule.getDataNodeRule().isNeedAccumulate(any())).thenReturn(true);
         
when(connection.getContextManager().getMetaDataContexts().getMetaData().getDatabase(DefaultDatabase.LOGIC_NAME).getRuleMetaData().getRules()).thenReturn(Collections.singleton(rule));
         when(connection.getDatabaseName()).thenReturn("db");
         DatabaseType databaseType = 
TypedSPILoader.getService(DatabaseType.class, "FIXTURE");
diff --git 
a/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSetTest.java
 
b/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSetTest.java
index cedcecd0f1f..9cda437e6f2 100644
--- 
a/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSetTest.java
+++ 
b/jdbc/src/test/java/org/apache/shardingsphere/driver/jdbc/core/resultset/DatabaseMetaDataResultSetTest.java
@@ -44,6 +44,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -133,8 +134,8 @@ class DatabaseMetaDataResultSetTest {
     }
     
     private ShardingRule mockShardingRule() {
-        ShardingRule result = mock(ShardingRule.class);
-        
when(result.findLogicTableByActualTable(ACTUAL_TABLE_NAME)).thenReturn(Optional.of(LOGIC_TABLE_NAME));
+        ShardingRule result = mock(ShardingRule.class, RETURNS_DEEP_STUBS);
+        
when(result.getDataNodeRule().findLogicTableByActualTable(ACTUAL_TABLE_NAME)).thenReturn(Optional.of(LOGIC_TABLE_NAME));
         return result;
     }
     
diff --git 
a/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtils.java
 
b/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtils.java
index 614f00d4206..dfa06c1df98 100644
--- 
a/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtils.java
+++ 
b/kernel/data-pipeline/scenario/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtils.java
@@ -56,8 +56,8 @@ public final class CDCDataNodeUtils {
         Map<String, List<DataNode>> result = new HashMap<>();
         // TODO support virtual data source name
         for (String each : tableNames) {
-            if (singleRule.isPresent() && 
singleRule.get().getAllDataNodes().containsKey(each)) {
-                result.put(each, new 
ArrayList<>(singleRule.get().getAllDataNodes().get(each)));
+            if (singleRule.isPresent() && 
singleRule.get().getDataNodeRule().getAllDataNodes().containsKey(each)) {
+                result.put(each, new 
ArrayList<>(singleRule.get().getDataNodeRule().getAllDataNodes().get(each)));
                 continue;
             }
             if (shardingRule.isPresent() && 
shardingRule.get().findShardingTable(each).isPresent()) {
@@ -65,8 +65,8 @@ public final class CDCDataNodeUtils {
                 result.put(each, shardingTable.getActualDataNodes());
                 continue;
             }
-            if (broadcastRule.isPresent() && 
broadcastRule.get().findFirstActualTable(each).isPresent()) {
-                result.put(each, 
Collections.singletonList(broadcastRule.get().getTableDataNodes().get(each).iterator().next()));
+            if (broadcastRule.isPresent() && 
broadcastRule.get().getDataNodeRule().findFirstActualTable(each).isPresent()) {
+                result.put(each, 
Collections.singletonList(broadcastRule.get().getDataNodeRule().getAllDataNodes().get(each).iterator().next()));
                 continue;
             }
             throw new PipelineInvalidParameterException(String.format("Not 
find actual data nodes of `%s`", each));
diff --git 
a/kernel/data-pipeline/scenario/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtilsTest.java
 
b/kernel/data-pipeline/scenario/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtilsTest.java
index 586d23cc208..ca65c6176ce 100644
--- 
a/kernel/data-pipeline/scenario/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtilsTest.java
+++ 
b/kernel/data-pipeline/scenario/cdc/core/src/test/java/org/apache/shardingsphere/data/pipeline/cdc/util/CDCDataNodeUtilsTest.java
@@ -35,6 +35,7 @@ import java.util.Optional;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -50,14 +51,14 @@ class CDCDataNodeUtilsTest {
         
when(mockShardingRule.findShardingTable("t_order")).thenReturn(Optional.of(mockShardingTable));
         
when(mockShardingRule.getShardingTable("t_order")).thenReturn(mockShardingTable);
         
when(mockRuleMetaData.findSingleRule(ShardingRule.class)).thenReturn(Optional.of(mockShardingRule));
-        SingleRule mockSingleRule = mock(SingleRule.class);
+        SingleRule mockSingleRule = mock(SingleRule.class, RETURNS_DEEP_STUBS);
         
when(mockRuleMetaData.findSingleRule(SingleRule.class)).thenReturn(Optional.of(mockSingleRule));
-        
when(mockSingleRule.getAllDataNodes()).thenReturn(Collections.singletonMap("t_order_item",
 Collections.singletonList(new DataNode("single.t_order_item"))));
+        
when(mockSingleRule.getDataNodeRule().getAllDataNodes()).thenReturn(Collections.singletonMap("t_order_item",
 Collections.singletonList(new DataNode("single.t_order_item"))));
         when(mockDatabase.getRuleMetaData()).thenReturn(mockRuleMetaData);
-        BroadcastRule mockBroadcastRule = mock(BroadcastRule.class);
-        
when(mockRuleMetaData.findSingleRule(BroadcastRule.class)).thenReturn(Optional.of(mockBroadcastRule));
-        
when(mockBroadcastRule.findFirstActualTable("t_address")).thenReturn(Optional.of("broadcast.t_address"));
-        
when(mockBroadcastRule.getTableDataNodes()).thenReturn(Collections.singletonMap("t_address",
 Collections.singletonList(new DataNode("broadcast.t_address"))));
+        BroadcastRule broadcastRule = mock(BroadcastRule.class, 
RETURNS_DEEP_STUBS);
+        
when(mockRuleMetaData.findSingleRule(BroadcastRule.class)).thenReturn(Optional.of(broadcastRule));
+        
when(broadcastRule.getDataNodeRule().findFirstActualTable("t_address")).thenReturn(Optional.of("broadcast.t_address"));
+        
when(broadcastRule.getDataNodeRule().getAllDataNodes()).thenReturn(Collections.singletonMap("t_address",
 Collections.singletonList(new DataNode("broadcast.t_address"))));
         Map<String, List<DataNode>> actual = 
CDCDataNodeUtils.buildDataNodesMap(mockDatabase, Arrays.asList("t_order", 
"t_order_item", "t_address"));
         assertTrue(actual.containsKey("t_order"));
         assertTrue(actual.containsKey("t_order_item"));
diff --git 
a/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
 
b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleDataNodeRule.java
similarity index 68%
copy from 
infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
copy to 
kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleDataNodeRule.java
index cde6be16d0c..ab6807ad3a0 100644
--- 
a/infra/distsql-handler/src/test/java/org/apache/shardingsphere/distsql/handler/fixture/DistSQLHandlerFixtureRule.java
+++ 
b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/rule/SingleDataNodeRule.java
@@ -15,38 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.distsql.handler.fixture;
+package org.apache.shardingsphere.single.rule;
 
-import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
+import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.datanode.DataNode;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
 
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Optional;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public final class DistSQLHandlerFixtureRule implements ShardingSphereRule, 
DataNodeContainedRule {
+/**
+ * Single data node rule.
+ */
+@RequiredArgsConstructor
+public final class SingleDataNodeRule implements DataNodeRule {
     
-    @Override
-    public RuleConfiguration getConfiguration() {
-        return mock(RuleConfiguration.class);
-    }
+    private final Map<String, Collection<DataNode>> tableDataNodes;
     
     @Override
     public Map<String, Collection<DataNode>> getAllDataNodes() {
-        DataNode dataNode = mock(DataNode.class);
-        when(dataNode.getDataSourceName()).thenReturn("foo_ds");
-        return Collections.singletonMap("", Collections.singleton(dataNode));
+        return tableDataNodes;
     }
     
     @Override
     public Collection<DataNode> getDataNodesByTableName(final String 
tableName) {
-        return null;
+        return tableDataNodes.getOrDefault(tableName.toLowerCase(), 
Collections.emptyList());
     }
     
     @Override
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 06fef5b4b10..115146d8247 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
@@ -29,8 +29,9 @@ import 
org.apache.shardingsphere.infra.metadata.database.schema.QualifiedTable;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.util.IndexMetaDataUtils;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 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.MutableDataNodeRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeRule;
 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.infra.rule.identifier.type.table.TableMapperContainedRule;
@@ -64,16 +65,18 @@ public final class SingleRule implements DatabaseRule, 
DataNodeContainedRule, Ta
     @Getter
     private final Collection<String> dataSourceNames;
     
-    @Getter
     private final Map<String, Collection<DataNode>> singleTableDataNodes;
     
     private final DatabaseType protocolType;
     
+    @Getter
+    private final DataNodeRule dataNodeRule;
+    
     @Getter
     private final SingleTableMapperRule tableMapperRule;
     
-    public SingleRule(final SingleRuleConfiguration ruleConfig, final String 
databaseName, final DatabaseType protocolType, final Map<String, DataSource> 
dataSourceMap,
-                      final Collection<ShardingSphereRule> builtRules) {
+    public SingleRule(final SingleRuleConfiguration ruleConfig, final String 
databaseName,
+                      final DatabaseType protocolType, final Map<String, 
DataSource> dataSourceMap, final Collection<ShardingSphereRule> builtRules) {
         configuration = ruleConfig;
         defaultDataSource = ruleConfig.getDefaultDataSource().orElse(null);
         Map<String, DataSource> enabledDataSources = 
DataSourceStateManager.getInstance().getEnabledDataSources(databaseName, 
dataSourceMap);
@@ -81,6 +84,7 @@ public final class SingleRule implements DatabaseRule, 
DataNodeContainedRule, Ta
         dataSourceNames = aggregateDataSourceMap.keySet();
         this.protocolType = protocolType;
         singleTableDataNodes = SingleTableDataNodeLoader.load(databaseName, 
protocolType, aggregateDataSourceMap, builtRules, configuration.getTables());
+        dataNodeRule = new SingleDataNodeRule(singleTableDataNodes);
         tableMapperRule = new SingleTableMapperRule(singleTableDataNodes);
     }
     
@@ -255,36 +259,6 @@ public final class SingleRule implements DatabaseRule, 
DataNodeContainedRule, Ta
         return new SingleRule((SingleRuleConfiguration) config, databaseName, 
protocolType, dataSourceMap, builtRules);
     }
     
-    @Override
-    public Map<String, Collection<DataNode>> getAllDataNodes() {
-        return singleTableDataNodes;
-    }
-    
-    @Override
-    public Collection<DataNode> getDataNodesByTableName(final String 
tableName) {
-        return singleTableDataNodes.getOrDefault(tableName.toLowerCase(), 
Collections.emptyList());
-    }
-    
-    @Override
-    public Optional<String> findFirstActualTable(final String logicTable) {
-        return Optional.empty();
-    }
-    
-    @Override
-    public boolean isNeedAccumulate(final Collection<String> tables) {
-        return false;
-    }
-    
-    @Override
-    public Optional<String> findLogicTableByActualTable(final String 
actualTable) {
-        return Optional.empty();
-    }
-    
-    @Override
-    public Optional<String> findActualTableByCatalog(final String catalog, 
final String logicTable) {
-        return Optional.empty();
-    }
-    
     @Override
     public Map<String, Object> getExportData() {
         return 
Collections.singletonMap(ExportableConstants.EXPORT_SINGLE_TABLES, 
tableMapperRule.getLogicTableMapper().getTableNames());
diff --git 
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
 
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
index 233784700fb..4dd3ab5a348 100644
--- 
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
+++ 
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/SingleSQLRouterTest.java
@@ -73,7 +73,7 @@ class SingleSQLRouterTest {
     void assertCreateRouteContextWithSingleDataSource() throws SQLException {
         SingleRule rule = new SingleRule(new SingleRuleConfiguration(),
                 DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), 
Collections.singletonMap("foo_ds", new MockedDataSource(mockConnection())), 
Collections.emptyList());
-        rule.getSingleTableDataNodes().put("t_order", 
Collections.singletonList(createDataNode("foo_ds")));
+        rule.getDataNodeRule().getAllDataNodes().put("t_order", 
Collections.singletonList(createDataNode("foo_ds")));
         ShardingSphereDatabase database = mockSingleDatabase();
         RouteContext actual = new 
SingleSQLRouter().createRouteContext(createQueryContext(),
                 mock(RuleMetaData.class), database, rule, new 
ConfigurationProperties(new Properties()), new ConnectionContext());
@@ -94,7 +94,7 @@ class SingleSQLRouterTest {
     void assertCreateRouteContextWithReadwriteSplittingDataSource() throws 
SQLException {
         SingleRule rule = new SingleRule(new SingleRuleConfiguration(),
                 DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), 
Collections.singletonMap("readwrite_ds", new 
MockedDataSource(mockConnection())), Collections.emptyList());
-        rule.getSingleTableDataNodes().put("t_order", 
Collections.singletonList(createDataNode("write_ds")));
+        rule.getDataNodeRule().getAllDataNodes().put("t_order", 
Collections.singletonList(createDataNode("write_ds")));
         ShardingSphereDatabase database = mockReadwriteSplittingDatabase();
         RouteContext actual = new 
SingleSQLRouter().createRouteContext(createQueryContext(),
                 mock(RuleMetaData.class), database, rule, new 
ConfigurationProperties(new Properties()), new ConnectionContext());
diff --git 
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/engine/SingleStandardRouteEngineTest.java
 
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/engine/SingleStandardRouteEngineTest.java
index a6507575165..48ec99df1bb 100644
--- 
a/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/engine/SingleStandardRouteEngineTest.java
+++ 
b/kernel/single/core/src/test/java/org/apache/shardingsphere/single/route/engine/SingleStandardRouteEngineTest.java
@@ -62,8 +62,8 @@ class SingleStandardRouteEngineTest {
     void assertRouteInSameDataSource() throws SQLException {
         SingleStandardRouteEngine engine = new 
SingleStandardRouteEngine(mockQualifiedTables(), null);
         SingleRule singleRule = new SingleRule(new SingleRuleConfiguration(), 
DefaultDatabase.LOGIC_NAME, new MySQLDatabaseType(), createDataSourceMap(), 
Collections.emptyList());
-        singleRule.getSingleTableDataNodes().put("t_order", 
Collections.singleton(mockDataNode("t_order")));
-        singleRule.getSingleTableDataNodes().put("t_order_item", 
Collections.singleton(mockDataNode("t_order_item")));
+        singleRule.getDataNodeRule().getAllDataNodes().put("t_order", 
Collections.singleton(mockDataNode("t_order")));
+        singleRule.getDataNodeRule().getAllDataNodes().put("t_order_item", 
Collections.singleton(mockDataNode("t_order_item")));
         RouteContext routeContext = new RouteContext();
         engine.route(routeContext, singleRule);
         List<RouteUnit> routeUnits = new 
ArrayList<>(routeContext.getRouteUnits());
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 01fd57e1cce..3ee1eb177b3 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,7 +23,7 @@ import org.apache.shardingsphere.infra.datanode.DataNode;
 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.datanode.DataNodeContainedRule;
 import 
org.apache.shardingsphere.infra.rule.identifier.type.table.TableMapperContainedRule;
 import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
 import org.apache.shardingsphere.test.fixture.jdbc.MockedDataSource;
@@ -104,7 +104,7 @@ class SingleRuleTest {
         
when(tableContainedRule.getTableMapperRule().getDistributedTableMapper().getTableNames()).thenReturn(Collections.singletonList("t_order"));
         
when(tableContainedRule.getTableMapperRule().getActualTableMapper().getTableNames()).thenReturn(Arrays.asList("t_order_0",
 "t_order_1"));
         SingleRule singleRule = new SingleRule(ruleConfig, 
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap, 
Collections.singleton(tableContainedRule));
-        Map<String, Collection<DataNode>> actual = 
singleRule.getSingleTableDataNodes();
+        Map<String, Collection<DataNode>> actual = 
singleRule.getDataNodeRule().getAllDataNodes();
         assertThat(actual.size(), is(2));
         assertTrue(actual.containsKey("employee"));
         assertTrue(actual.containsKey("student"));
@@ -116,7 +116,7 @@ class SingleRuleTest {
         
when(tableContainedRule.getTableMapperRule().getDistributedTableMapper().getTableNames()).thenReturn(Collections.singleton("T_ORDER"));
         
when(tableContainedRule.getTableMapperRule().getActualTableMapper().getTableNames()).thenReturn(Arrays.asList("T_ORDER_0",
 "T_ORDER_1"));
         SingleRule singleRule = new SingleRule(ruleConfig, 
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap, 
Collections.singleton(tableContainedRule));
-        Map<String, Collection<DataNode>> actual = 
singleRule.getSingleTableDataNodes();
+        Map<String, Collection<DataNode>> actual = 
singleRule.getDataNodeRule().getAllDataNodes();
         assertThat(actual.size(), is(2));
         assertTrue(actual.containsKey("employee"));
         assertTrue(actual.containsKey("student"));
@@ -211,17 +211,17 @@ class SingleRuleTest {
     void assertGetAllDataNodes() {
         DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
         SingleRule singleRule = new SingleRule(ruleConfig, 
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap, 
Collections.singleton(dataNodeContainedRule));
-        assertTrue(singleRule.getAllDataNodes().containsKey("employee"));
-        assertTrue(singleRule.getAllDataNodes().containsKey("student"));
-        assertTrue(singleRule.getAllDataNodes().containsKey("t_order_0"));
-        assertTrue(singleRule.getAllDataNodes().containsKey("t_order_1"));
+        
assertTrue(singleRule.getDataNodeRule().getAllDataNodes().containsKey("employee"));
+        
assertTrue(singleRule.getDataNodeRule().getAllDataNodes().containsKey("student"));
+        
assertTrue(singleRule.getDataNodeRule().getAllDataNodes().containsKey("t_order_0"));
+        
assertTrue(singleRule.getDataNodeRule().getAllDataNodes().containsKey("t_order_1"));
     }
     
     @Test
     void assertGetDataNodesByTableName() {
         DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
         SingleRule singleRule = new SingleRule(ruleConfig, 
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap, 
Collections.singleton(dataNodeContainedRule));
-        Collection<DataNode> actual = 
singleRule.getDataNodesByTableName("EMPLOYEE");
+        Collection<DataNode> actual = 
singleRule.getDataNodeRule().getDataNodesByTableName("EMPLOYEE");
         assertThat(actual.size(), is(1));
         DataNode dataNode = actual.iterator().next();
         assertThat(dataNode.getDataSourceName(), is("foo_ds"));
@@ -233,14 +233,14 @@ class SingleRuleTest {
         DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
         SingleRule singleRule = new SingleRule(ruleConfig, 
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap, 
Collections.singleton(dataNodeContainedRule));
         String logicTable = "employee";
-        assertFalse(singleRule.findFirstActualTable(logicTable).isPresent());
+        
assertFalse(singleRule.getDataNodeRule().findFirstActualTable(logicTable).isPresent());
     }
     
     @Test
     void assertIsNeedAccumulate() {
         DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
         SingleRule singleRule = new SingleRule(ruleConfig, 
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap, 
Collections.singleton(dataNodeContainedRule));
-        assertFalse(singleRule.isNeedAccumulate(Collections.emptyList()));
+        
assertFalse(singleRule.getDataNodeRule().isNeedAccumulate(Collections.emptyList()));
     }
     
     @Test
@@ -248,7 +248,7 @@ class SingleRuleTest {
         DataNodeContainedRule dataNodeContainedRule = 
mock(DataNodeContainedRule.class);
         SingleRule singleRule = new SingleRule(ruleConfig, 
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap, 
Collections.singleton(dataNodeContainedRule));
         String actualTable = "student";
-        
assertFalse(singleRule.findLogicTableByActualTable(actualTable).isPresent());
+        
assertFalse(singleRule.getDataNodeRule().findLogicTableByActualTable(actualTable).isPresent());
     }
     
     @Test
@@ -257,6 +257,6 @@ class SingleRuleTest {
         SingleRule singleRule = new SingleRule(ruleConfig, 
DefaultDatabase.LOGIC_NAME, new H2DatabaseType(), dataSourceMap, 
Collections.singleton(dataNodeContainedRule));
         String catalog = "employee";
         String logicTable = "t_order_0";
-        assertFalse(singleRule.findActualTableByCatalog(catalog, 
logicTable).isPresent());
+        
assertFalse(singleRule.getDataNodeRule().findActualTableByCatalog(catalog, 
logicTable).isPresent());
     }
 }
diff --git 
a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutor.java
 
b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutor.java
index 4c3675def05..50a27be73e2 100644
--- 
a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutor.java
+++ 
b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutor.java
@@ -51,7 +51,7 @@ public final class ShowSingleTableExecutor implements 
DistSQLQueryExecutor<ShowS
     @Override
     public Collection<LocalDataQueryResultRow> getRows(final 
ShowSingleTableStatement sqlStatement, final ContextManager contextManager) {
         Collection<DataNode> resultDataNodes = getPattern(sqlStatement)
-                .map(optional -> 
getDataNodesWithLikePattern(rule.getSingleTableDataNodes(), 
optional)).orElseGet(() -> getDataNodes(rule.getSingleTableDataNodes(), 
sqlStatement));
+                .map(optional -> 
getDataNodesWithLikePattern(rule.getDataNodeRule().getAllDataNodes(), 
optional)).orElseGet(() -> 
getDataNodes(rule.getDataNodeRule().getAllDataNodes(), sqlStatement));
         Collection<DataNode> sortedDataNodes = 
resultDataNodes.stream().sorted(Comparator.comparing(DataNode::getTableName)).collect(Collectors.toList());
         return sortedDataNodes.stream().map(each -> new 
LocalDataQueryResultRow(each.getTableName(), 
each.getDataSourceName())).collect(Collectors.toList());
     }
diff --git 
a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/update/UnloadSingleTableExecutor.java
 
b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/update/UnloadSingleTableExecutor.java
index b2299df06e8..6d92ed4a621 100644
--- 
a/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/update/UnloadSingleTableExecutor.java
+++ 
b/kernel/single/distsql/handler/src/main/java/org/apache/shardingsphere/single/distsql/handler/update/UnloadSingleTableExecutor.java
@@ -64,7 +64,7 @@ public final class UnloadSingleTableExecutor implements 
DatabaseRuleAlterExecuto
         for (String each : sqlStatement.getTables()) {
             checkTableExist(allTables, each);
             checkIsSingleTable(singleTables, each);
-            checkTableRuleExist(database.getName(), 
database.getProtocolType(), singleRule.getDataNodesByTableName(each), each);
+            checkTableRuleExist(database.getName(), 
database.getProtocolType(), 
singleRule.getDataNodeRule().getDataNodesByTableName(each), each);
         }
     }
     
diff --git 
a/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutorTest.java
 
b/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutorTest.java
index fba8523ee86..fa95936e169 100644
--- 
a/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutorTest.java
+++ 
b/kernel/single/distsql/handler/src/test/java/org/apache/shardingsphere/single/distsql/handler/query/ShowSingleTableExecutorTest.java
@@ -86,11 +86,11 @@ class ShowSingleTableExecutorTest {
     }
     
     private SingleRule mockSingleRule() {
-        SingleRule result = mock(SingleRule.class);
+        SingleRule result = mock(SingleRule.class, RETURNS_DEEP_STUBS);
         Map<String, Collection<DataNode>> singleTableDataNodeMap = new 
HashMap<>();
         singleTableDataNodeMap.put("t_order", Collections.singleton(new 
DataNode("ds_1", "t_order")));
         singleTableDataNodeMap.put("t_order_item", Collections.singleton(new 
DataNode("ds_2", "t_order_item")));
-        
when(result.getSingleTableDataNodes()).thenReturn(singleTableDataNodeMap);
+        
when(result.getDataNodeRule().getAllDataNodes()).thenReturn(singleTableDataNodeMap);
         return result;
     }
 }
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 ab30083ad4b..6bece73bea9 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
@@ -46,7 +46,7 @@ import 
org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 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.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.session.query.QueryContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
@@ -343,7 +343,7 @@ public final class DatabaseConnector implements 
DatabaseBackendHandler {
     
     private boolean isNeedAccumulate(final SQLStatementContext 
sqlStatementContext) {
         Optional<DataNodeContainedRule> dataNodeContainedRule = 
database.getRuleMetaData().findSingleRule(DataNodeContainedRule.class);
-        return dataNodeContainedRule.isPresent() && 
dataNodeContainedRule.get().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames());
+        return dataNodeContainedRule.isPresent() && 
dataNodeContainedRule.get().getDataNodeRule().isNeedAccumulate(sqlStatementContext.getTablesContext().getTableNames());
     }
     
     /**
diff --git 
a/proxy/backend/type/mysql/src/main/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilder.java
 
b/proxy/backend/type/mysql/src/main/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilder.java
index fe6b745a994..4e74c268947 100644
--- 
a/proxy/backend/type/mysql/src/main/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilder.java
+++ 
b/proxy/backend/type/mysql/src/main/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilder.java
@@ -21,7 +21,7 @@ import 
org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryRe
 import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereColumn;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
-import 
org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import 
org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import 
org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeaderBuilder;
 
@@ -61,7 +61,7 @@ public final class MySQLQueryHeaderBuilder implements 
QueryHeaderBuilder {
     
     private String getLogicTableName(final ShardingSphereDatabase database, 
final String actualTableName) {
         for (DataNodeContainedRule each : 
database.getRuleMetaData().findRules(DataNodeContainedRule.class)) {
-            Optional<String> logicTable = 
each.findLogicTableByActualTable(actualTableName);
+            Optional<String> logicTable = 
each.getDataNodeRule().findLogicTableByActualTable(actualTableName);
             if (logicTable.isPresent()) {
                 return logicTable.get();
             }
diff --git 
a/proxy/backend/type/mysql/src/test/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilderTest.java
 
b/proxy/backend/type/mysql/src/test/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilderTest.java
index c6515658be0..44828a75227 100644
--- 
a/proxy/backend/type/mysql/src/test/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilderTest.java
+++ 
b/proxy/backend/type/mysql/src/test/java/org/apache/shardingsphere/proxy/backend/mysql/response/header/query/MySQLQueryHeaderBuilderTest.java
@@ -24,7 +24,7 @@ import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSp
 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.rule.identifier.type.DataNodeContainedRule;
+import 
org.apache.shardingsphere.infra.rule.identifier.type.datanode.DataNodeContainedRule;
 import 
org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.junit.jupiter.api.Test;
@@ -79,8 +79,8 @@ class MySQLQueryHeaderBuilderTest {
     void assertBuildWithNullSchema() throws SQLException {
         ShardingSphereDatabase database = mock(ShardingSphereDatabase.class, 
RETURNS_DEEP_STUBS);
         when(database.getSchemas()).thenReturn(Collections.emptyMap());
-        DataNodeContainedRule containedRule = 
mock(DataNodeContainedRule.class);
-        
when(containedRule.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_order"));
+        DataNodeContainedRule containedRule = 
mock(DataNodeContainedRule.class, RETURNS_DEEP_STUBS);
+        
when(containedRule.getDataNodeRule().findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_order"));
         
when(database.getRuleMetaData().findRules(DataNodeContainedRule.class)).thenReturn(Collections.singletonList(containedRule));
         QueryResultMetaData queryResultMetaData = createQueryResultMetaData();
         QueryHeader actual = new 
MySQLQueryHeaderBuilder().build(queryResultMetaData, database, 
queryResultMetaData.getColumnName(1), queryResultMetaData.getColumnLabel(1), 1);
@@ -104,8 +104,8 @@ class MySQLQueryHeaderBuilderTest {
         when(schema.getTable("t_logic_order")).thenReturn(
                 new ShardingSphereTable("t_logic_order", 
Collections.singleton(column), Collections.singleton(new 
ShardingSphereIndex("order_id")), Collections.emptyList()));
         when(result.getSchema(DefaultDatabase.LOGIC_NAME)).thenReturn(schema);
-        ShardingRule shardingRule = mock(ShardingRule.class);
-        
when(shardingRule.findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_logic_order"));
+        ShardingRule shardingRule = mock(ShardingRule.class, 
RETURNS_DEEP_STUBS);
+        
when(shardingRule.getDataNodeRule().findLogicTableByActualTable("t_order")).thenReturn(Optional.of("t_logic_order"));
         
when(result.getRuleMetaData().findRules(DataNodeContainedRule.class)).thenReturn(Collections.singleton(shardingRule));
         when(result.getName()).thenReturn(DefaultDatabase.LOGIC_NAME);
         return result;

Reply via email to